SlideShare a Scribd company logo
1 of 52
Download to read offline
Implementing an akka-streams
materializer for big data
The Gearpump Materializer
Kam Kasravi
Technical Presentation
● Familiarity with akka-streams flow and graph DSL’s
● Familiarity with big data and real time streaming platforms
● Familiarity with scala
● Effort between the akka-streams and Gearpump teams started late last year
● Resulted in a number of pull requests into akka-streams to enable different materializers
● Close to completion with good support of the akka-streams DSL (all GraphStages)
● Fairly seamless to switch between local and distributed
Who am I?
● Committer on Apache Gearpump (incubating)
- http://gearpump.apache.org
● Architect on Trusted Analytics Platform (TAP)
- http://trustedanalytics.org
● Lead or Architect across many companies, industries
- NYSE, eBay, PayPal, Yahoo, ...
Title Goes Here
There are many variations of passages
of lorem ipsum available, but the
majority suffered alteration
some form.
What is Apache Gearpump?
● Accepted into Apache incubator last March
● Similar to Apache Beam and Apache Flink (real-time message delivery)
● Heavily leverages the actor model and akka (more so than others)
● Unique features like dynamic DAG
● Excellent runtime visualization tooling of cluster and application DAGs
● One of the best big data performance profiles (both throughput, latency)
Agenda
● Why?
○ Why integrate akka-streams into a big data platform?
● Big Data platform evolving features
○ Functionality big data platforms are embracing
● Prerequisites needed for any Big Data platform
○ Minimal features a big data platform must have
● Big data platform integration challenges
○ What concepts do not map well within big data platforms?
● Object models: akka-streams, Gearpump
● Materialization
○ ActorMaterializer - materializing the module tree
○ GearpumpMaterializer - rewriting the module tree
Why?
● Akka-streams has limitations inherent within a single JVM
○ Throughput and latency are key big data features that require scaling beyond single JVM’s
● Akka-streams DSL is a superset of other big data platform DSLs
○ Has a logical plan (declarative) that can be transformed to an execution plan (runtime)
● Akka-streams programming paradigm is declarative, composable,
extensible*, stackable* and reusable*
* Provides a level of extensibility and functionality beyond most big data platform DSLs
Extensible
● Extend GraphStage
● Extend Source, Sink, Flow or BidiFlow
● All derive from Graph
* Provides a level of extensibility and functionality beyond most big data platform DSLs
Stackable
● Another term for nestable or recursive. Reference to Kleisli (theoretical).
● Source, Sink, Flow or BidiFlow may contain their own topologies
* Provides a level of extensibility and functionality beyond most big data platform DSLs
Reusable
● Graph topologies can be attached anywhere (any Graph)
● Recent akka-streams feature is dynamic attachment via hubs
● Hubs will take advantage of Gearpump dynamic DAG within the
GearpumpMaterializer
* Provides a level of extensibility and functionality beyond most big data platform DSLs
Big Data platform
evolving features
(1)
● Big data platforms are moving to consolidate disparate API’s
○ Too many APIs: Concord, Flink, Heron, Pulsar, Spark, Storm, Samza
○ Common DSL is also an approach being taken by Apache Beam
○ Analogy to SQL - common grammar that different platforms execute
Big Data platform
evolving features
(2)
● Big data platforms will increasingly require dynamic
pipelines that are compositional and reusable
● Examples include:
○ Machine learning
○ IoT sensors
Big Data platform
evolving features
(3)
● Machine learning use cases
○ Replace or update scoring models
○ Model Ensembles
■ concept drift
■ data drift
Big Data platform
evolving features
(4)
● IoT use cases
○ Bring new sensors on line with no interruption
○ Change or update configuration parameters at remote
sensors
Prerequisites
needed for any
Big Data
platform (1)
Downstream must
be able to pull
Upstream must
be able to push
1. Push and Pull
Downstream must
be able to backpressure
all the way to source
2. Backpressure
<< <<
Prerequisites
needed for any
Big Data
platform (2)
3. Parallelization
4. Asynchronous
5. Bidirectional
Big data
platform
integration
challenges (1)
A number of
GraphStages have
completion or
cancellation
semantics. Big data
pipelines are often
infinite streams and
do not complete.
Cancel is often
viewed as a failure.
● Balance[T]
● Completion[T]
● Merge[T]
● Split[T]
Big data
platform
integration
challenges (2)
A number of
GraphStages have
specific upstream and
downstream ordering
and timing directives.
● Batch[T]
● Concat[T]
● Delay[T]
● DelayInitial[T]
● Interleave[T]
Big data
platform
integration
challenges (3)
The async attribute as
well as fusing do not
map cleanly when
distributing GraphStage
functionality across
machines.
● Graph.async
● Fusing
Graph.async
● Collapses multiple operations (GraphStageLogic) into one actor
● Distributed scenarios where one may want actors within the
same JVM or on the same machine
Fusing
● Creates one or more islands delimited by async boundaries
● For distributed scenario no fusing should occur until the
materializer can evaluate and optimize the execution plan
Object Models
● Akka-stream’s GraphStage, Module, Shape
● Gearpump’s Graph, Task, Partitioner
Akka-streams Object Model
↪ Base type is a Graph. Common base type is a GraphStage
↪ Graph contains a
↳ Module contains a
↳ Shape
↪ Only a RunnableGraph can be materialized
↪ A RunnableGraph needs at least one Source and one Sink
Akka-streams Graph[S, M]
● Graph is parameterized by
○ Shape
○ Materialized Value
● Graph contains a Module contains a Shape
○ Module is where the runtime is constructed and manipulated
● Graph’s first level subtypes provide basic functionality
○ Source
○ Sink
○ Flow
○ BidiFlow
S M
Graph
Source
Sink
Flow
BidiFlow
Module
Shape
GraphStage[S <: Shape]
Graph
GraphStageWithMaterializedValue
GraphStage
GraphStageModule
Module
GraphStage[S <: Shape]
subtypes (incomplete)
↳ Balance[T]
↳ Batch[In, Out]
↳ Broadcast[T]
↳ Collect[In, Out]
↳ Concat[T]
↳ DelayInitial[T]
↳ DropWhile[T]
↳ Expand[In, Out]
↳ FlattenMerge[T, M]
↳ Fold[In, Out]
↳ FoldAsync[T]
↳ FutureSource[T]
↳ GroupBy[T, K]
↳ Grouped[T]
↳ GroupedWithin[T]
↳ Interleave[T]
↳ Intersperse[T]
↳ LimitWeighted[T]
↳ Map[In, Out]
↳ MapAsync[In, Out]
↳ Merge[T]
↳ MergePreferred[T]
↳ MergeSorted[T]
↳ OrElse[T]
↳ Partition[T]
↳ PrefixAndTail[T]
↳ Recover[T]
↳ Scan[In, Out]
↳ SimpleLinearGraph[T]
↳ Sliding[T]
What about Module?
● Module is a recursive structure containing a Set[Modules]
● Module is a declarative data structure used as the AST
● Module is used to represent a graph of nodes and edges from the original
GraphStages
● Module contains downstream and upstream ports (edges)
● Materializers walk the module tree to create and run instances of publishers
and subscribers.
● Each publisher and subscriber is an actor (ActorGraphInterpreter)
Gearpump Object Model
↪ Graph[Node, Edge] holds
↳ Tasks (Node)
↳ Partitioners (Edge)
↪ This is a Gearpump Graph, not to be
confused with akka-streams Graph.
Gearpump Graph[N<:Task, E<:Partitioner]
● Graph is parameterized by
○ Node - must be a subtype of Task
○ Edge - must be a subtype of Parititioner
N E
Graph
List[Task]
List[Partitioner]
Task
Task
GraphTask
GraphTask
subtypes (incomplete)
↳ BalanceTask
↳ BatchTask[In, Out]
↳ BroadcastTask[T]
↳ CollectTask[In, Out]
↳ ConcatTask
↳ DelayInitialTask[T]
↳ DropWhileTask[T]
↳ ExpandTask[In, Out]
↳ FlattenMerge[T, M]
↳ FoldTask[In, Out]
↳ FutureSourceTask[T]
↳ GroupByTask[T, K]
↳ GroupedTask[T]
↳ GroupedWithinTask[T]
↳ InterleaveTask[T]
↳ IntersperseTask[T]
↳ LimitWeightedTask[T]
↳ MapTask[In, Out]
↳ MapAsyncTask[In, Out]
↳ MergeTask[T]
↳ OrElseTask[T]
↳ PartitionTask[T]
↳ PrefixAndTailTask[T]
↳ RecoverTask[T]
↳ ScanTask[In, Out]
↳ SlidingTask[T]
Materializer Variations
1. AST (module tree) is matched for every module type
(GearpumpMaterializer)
2. AST (module tree) is matched for certain module types
○ After distribution - local ActorMaterializer is used for
operations on that worker
○ Materializer works more as a distribution coordinator
Example 1
Source
Broadcast
Flow
Merge
Sink
implicit val materializer = ActorMaterializer()
val sinkActor = system.actorOf(Props(new SinkActor())
val source = Source((1 to 5))
val sink = Sink.actorRef(sinkActor, "COMPLETE")
val flowA: Flow[Int, Int, NotUsed] = Flow[Int].map {
x => println(s"processing broadcasted element : $x in flowA"); x
}
val flowB: Flow[Int, Int, NotUsed] = Flow[Int].map {
x => println(s"processing broadcasted element : $x in flowB"); x
}
val graph = RunnableGraph.fromGraph(GraphDSL.create() {
implicit b =>
val broadcast = b.add(Broadcast[Int](2))
val merge = b.add(Broadcast[Int](2))
source ~> broadcast
broadcast ~> flowA ~> merge
broadcast ~> flowB ~> merge
merge ~> sink
ClosedShape
})
graph.run()
Example 1
implicit val materializer = ActorMaterializer()
val sinkActor = system.actorOf(Props(new SinkActor())
val source = Source((1 to 5))
val sink = Sink.actorRef(sinkActor, "COMPLETE")
val flowA: Flow[Int, Int, NotUsed] = Flow[Int].map {
x => println(s"processing broadcasted element : $x in flowA"); x
}
val flowB: Flow[Int, Int, NotUsed] = Flow[Int].map {
x => println(s"processing broadcasted element : $x in flowB"); x
}
val graph = RunnableGraph.fromGraph(GraphDSL.create() {
implicit b =>
val broadcast = b.add(Broadcast[Int](2))
val merge = b.add(Broadcast[Int](2))
source ~> broadcast
broadcast ~> flowA ~> merge
broadcast ~> flowB ~> merge
merge ~> sink
ClosedShape
})
graph.run()
Source Broadcast
Flow
Flow
Merge
GraphStages
Sink
class SinkActor extends Actor {
def receive: Receive = {
case any: Any =>
println(s“Confirm received: $any”)
}
Example 1
Source Broadcast
Flow
Flow
Merge
GraphStages
Sink
Module Tree
GraphStageModule
GraphStageModule
stage=SingleSource
stage=StatefulMapConcat
ActorRefSink
stage=Broadcast
stage=Map
stage=Merge
GraphStageModule
GraphStageModule
GraphStageModule
Example 1
implicit val materializer = ActorMaterializer()
val sinkActor = system.actorOf(Props(new SinkActor())
val source = Source((1 to 5))
val sink = Sink.actorRef(sinkActor, "COMPLETE")
val flowA: Flow[Int, Int, NotUsed] = Flow[Int].map {
x => println(s"processing broadcasted element : $x in flowA"); x
}
val flowB: Flow[Int, Int, NotUsed] = Flow[Int].map {
x => println(s"processing broadcasted element : $x in flowB"); x
}
val graph = RunnableGraph.fromGraph(GraphDSL.create() {
implicit b =>
val broadcast = b.add(Broadcast[Int](2))
val merge = b.add(Broadcast[Int](2))
source ~> broadcast
broadcast ~> flowA ~> merge
broadcast ~> flowB ~> merge
merge ~> sink
ClosedShape
})
graph.run()
source broadcast
flowA
flowB
merge
GraphStages
sink
Example 1
processing broadcasted element : 1 in flowA
processing broadcasted element : 1 in flowB
processing broadcasted element : 2 in flowA
Confirm received: 1
Confirm received: 1
processing broadcasted element : 2 in flowB
Confirm received: 2
Confirm received: 2
processing broadcasted element : 3 in flowA
processing broadcasted element : 3 in flowB
processing broadcasted element : 4 in flowA
processing broadcasted element : 4 in flowB
Confirm received: 3
Confirm received: 3
processing broadcasted element : 5 in flowA
processing broadcasted element : 5 in flowB
Confirm received: 4
Confirm received: 4
Confirm received: 5
Confirm received: 5
Confirm received: COMPLETE
source broadcast
flowA
flowB
merge
GraphStages
sink
ActorMaterializer Output
Example 1
implicit val materializer = GearpumpMaterializer()
val sinkActor = system.actorOf(Props(new SinkActor())
val source = Source((1 to 5))
val sink = Sink.actorRef(sinkActor, "COMPLETE")
val flowA: Flow[Int, Int, NotUsed] = Flow[Int].map {
x => println(s"processing broadcasted element : $x in flowA"); x
}
val flowB: Flow[Int, Int, NotUsed] = Flow[Int].map {
x => println(s"processing broadcasted element : $x in flowB"); x
}
val graph = RunnableGraph.fromGraph(GraphDSL.create() {
implicit b =>
val broadcast = b.add(Broadcast[Int](2))
val merge = b.add(Broadcast[Int](2))
source ~> broadcast
broadcast ~> flowA ~> merge
broadcast ~> flowB ~> merge
merge ~> sink
ClosedShape
})
graph.run()
source broadcast
flowA
flowB
merge
GraphStages
sink
Example 1
processing broadcasted element : 1 in flowA
processing broadcasted element : 1 in flowB
processing broadcasted element : 2 in flowB
processing broadcasted element : 2 in flowA
processing broadcasted element : 3 in flowB
processing broadcasted element : 3 in flowA
processing broadcasted element : 4 in flowB
processing broadcasted element : 4 in flowA
processing broadcasted element : 5 in flowB
Confirm received: 1
processing broadcasted element : 5 in flowA
Confirm received: 1
Confirm received: 2
Confirm received: 2
Confirm received: 3
Confirm received: 3
Confirm received: 4
Confirm received: 4
Confirm received: 5
Confirm received: 5
source broadcast
flowA
flowB
merge
GraphStages
sink
GearpumpMaterializer Output
Demo
GraphStageModule(
stage=SingleSource)
ActorRefSinkGraphStageModule(
stage=StatefulMapConcat)
GraphStageModule(
stage=Broadcast)
GraphStageModule(
stage=Map)
GraphStageModule(
stage=Merge)
ActorMaterializer
GraphStageModule(
stage=SingleSource)
ActorRefSinkGraphStageModule(
stage=StatefulMapConcat)
GraphStageModule(
stage=Broadcast)
GraphStageModule(
stage=Map)
GraphStageModule(
stage=Merge)
1. Traverses the Module Tree
ActorMaterializer
2. Builds a runtime graph of BoundaryPublisher and
BoundarySubscribers (Reactive API).
3. Each Publisher or Subscriber contains an instance of
GraphStageLogic specific to that GraphStage.
4. Each Publisher or Subscriber also contains an
instance of ActorGraphInterpreter - an Actor that
manages the message flow using GraphStageLogic.
GearpumpMaterializer
GraphStageModule(
stage=SingleSource)
ActorRefSink
GraphStageModule(
stage=Broadcast)
GraphStageModule(
stage=Map)
GraphStageModule(
stage=Merge)
1. Rewrites the Module Tree into ‘local’ and ‘remote’
Gearpump Graphs.
GraphStageModule(
stage=StatefulMapConcat)
GearpumpMaterializer
GraphStageModule(
stage=SingleSource)
ActorRefSink
GraphStageModule(
stage=Broadcast)
GraphStageModule(
stage=Map)
GraphStageModule(
stage=Merge)
2. Choice of ‘local’ and ‘remote’ is determined by a
‘Strategy’. The default Strategy is to put Source and Sink
types in local
GraphStageModule(
stage=StatefulMapConcat)
GearpumpMaterializer
ActorRefSink
3. Inserts BridgeModules into both Graphs
SourceBridgeModule
SinkBridgeModule
SinkBridgeModule
GraphStageModule(
stage=Broadcast)
GraphStageModule(
stage=Map)
GraphStageModule(
stage=Merge)GraphStageModule(
stage=StatefulMapConcat)
GraphStageModule(
stage=SingleSource)
SourceBridgeModule
GearpumpMaterializer
ActorRefSink
4. Local graph is passed to a LocalGraphMaterializer
SinkBridgeModule
GraphStageModule(
stage=SingleSource)
SourceBridgeModule
LocalGraphMaterializer is a
variant (subtype) of
ActorMaterializer
GearpumpMaterializer
5. Converts the remote graph’s Modules into Tasks
SourceBridgeTask SinkBridgeTaskBroadcastTask
TransformTask
MergeTaskStatefulMapConcatTask
GearpumpMaterializer
6. Sends this Graph to the Gearpump master
SourceBridgeTask SinkBridgeTaskBroadcastTask
TransformTask
MergeTaskStatefulMapConcatTask
GearpumpMaterializer
7. Materialization is controlled at BridgeTasks
SourceBridgeTask SinkBridgeTaskBroadcastTask
TransformTask
MergeTaskStatefulMapConcatTask
Example 2
No local graph.
More typical of distributed apps.
implicit val materializer = GearpumpMaterializer()
val sink = GearSink.to(new LoggerSink[String]))
val sourceData = new CollectionDataSource(
List("red hat", "yellow sweater", "blue jack", "red
apple", "green plant", "blue sky"))
val source = GearSource.from[String](sourceData)
source.filter(_.startsWith("red")).map("I want to order
item: " + _).runWith(sink)
Example 3
More complex Graph with loops
implicit val materializer = GearpumpMaterializer()
RunnableGraph.fromGraph(GraphDSL.create() {
implicitbuilder =>
val A = builder.add(Source.single(0)).out
val B = builder.add(Broadcast[Int](2))
val C = builder.add(Merge[Int](2))
val D = builder.add(Flow[Int].map(_ + 1))
val E = builder.add(Balance[Int](2))
val F = builder.add(Merge[Int](2))
val G = builder.add(Sink.foreach(println)).in
C <~ F
A ~> B ~> C ~> F
B ~> D ~> E ~> F
E ~> G
ClosedShape
}).run()
Summary
● Akka-streams provides a compelling programming model
that enables declarative pipeline reuse and extensibility.
● Akka-streams allows different materializers to control and
materialize different parts of the module tree.
● It’s possible to provide a seamless (or nearly seamless)
conversion of akka-streams to run in a distributed setting
by merely replacing ActorMaterializer with
GearpumpMaterializer.
● Alternative distributed materializers can be implemented
using a similar approach.
● Distributed akka-streams via Apache Gearpump will be
available in the next release of Apache Gearpump (0.8.2)
or will be made available within an akka specific repo.
Thank you
twitter:
@ApacheGearpump
@kkasravi

More Related Content

What's hot

Back-Pressure in Action: Handling High-Burst Workloads with Akka Streams & Ka...
Back-Pressure in Action: Handling High-Burst Workloads with Akka Streams & Ka...Back-Pressure in Action: Handling High-Burst Workloads with Akka Streams & Ka...
Back-Pressure in Action: Handling High-Burst Workloads with Akka Streams & Ka...Reactivesummit
 
Spark Summit EU talk by Luc Bourlier
Spark Summit EU talk by Luc BourlierSpark Summit EU talk by Luc Bourlier
Spark Summit EU talk by Luc BourlierSpark Summit
 
Flink Forward SF 2017: Chinmay Soman - Real Time Analytics in the real World ...
Flink Forward SF 2017: Chinmay Soman - Real Time Analytics in the real World ...Flink Forward SF 2017: Chinmay Soman - Real Time Analytics in the real World ...
Flink Forward SF 2017: Chinmay Soman - Real Time Analytics in the real World ...Flink Forward
 
Flink Forward SF 2017: David Hardwick, Sean Hester & David Brelloch - Dynami...
Flink Forward SF 2017: David Hardwick, Sean Hester & David Brelloch -  Dynami...Flink Forward SF 2017: David Hardwick, Sean Hester & David Brelloch -  Dynami...
Flink Forward SF 2017: David Hardwick, Sean Hester & David Brelloch - Dynami...Flink Forward
 
Flink Forward SF 2017: James Malone - Make The Cloud Work For You
Flink Forward SF 2017: James Malone - Make The Cloud Work For YouFlink Forward SF 2017: James Malone - Make The Cloud Work For You
Flink Forward SF 2017: James Malone - Make The Cloud Work For YouFlink Forward
 
Introduction to Apache Beam & No Shard Left Behind: APIs for Massive Parallel...
Introduction to Apache Beam & No Shard Left Behind: APIs for Massive Parallel...Introduction to Apache Beam & No Shard Left Behind: APIs for Massive Parallel...
Introduction to Apache Beam & No Shard Left Behind: APIs for Massive Parallel...Dan Halperin
 
Big Data Day LA 2016/ Big Data Track - Portable Stream and Batch Processing w...
Big Data Day LA 2016/ Big Data Track - Portable Stream and Batch Processing w...Big Data Day LA 2016/ Big Data Track - Portable Stream and Batch Processing w...
Big Data Day LA 2016/ Big Data Track - Portable Stream and Batch Processing w...Data Con LA
 
S3, Cassandra or Outer Space? Dumping Time Series Data using Spark - Demi Ben...
S3, Cassandra or Outer Space? Dumping Time Series Data using Spark - Demi Ben...S3, Cassandra or Outer Space? Dumping Time Series Data using Spark - Demi Ben...
S3, Cassandra or Outer Space? Dumping Time Series Data using Spark - Demi Ben...Codemotion Tel Aviv
 
Uber Real Time Data Analytics
Uber Real Time Data AnalyticsUber Real Time Data Analytics
Uber Real Time Data AnalyticsAnkur Bansal
 
Flink Forward SF 2017: Kenneth Knowles - Back to Sessions overview
Flink Forward SF 2017: Kenneth Knowles - Back to Sessions overviewFlink Forward SF 2017: Kenneth Knowles - Back to Sessions overview
Flink Forward SF 2017: Kenneth Knowles - Back to Sessions overviewFlink Forward
 
Kafka Summit NYC 2017 - Building Advanced Streaming Applications using the La...
Kafka Summit NYC 2017 - Building Advanced Streaming Applications using the La...Kafka Summit NYC 2017 - Building Advanced Streaming Applications using the La...
Kafka Summit NYC 2017 - Building Advanced Streaming Applications using the La...confluent
 
Building Data Product Based on Apache Spark at Airbnb with Jingwei Lu and Liy...
Building Data Product Based on Apache Spark at Airbnb with Jingwei Lu and Liy...Building Data Product Based on Apache Spark at Airbnb with Jingwei Lu and Liy...
Building Data Product Based on Apache Spark at Airbnb with Jingwei Lu and Liy...Databricks
 
Akka streams scala italy2015
Akka streams scala italy2015Akka streams scala italy2015
Akka streams scala italy2015mircodotta
 
Flink Forward SF 2017: Bill Liu & Haohui Mai - AthenaX : Uber’s streaming pro...
Flink Forward SF 2017: Bill Liu & Haohui Mai - AthenaX : Uber’s streaming pro...Flink Forward SF 2017: Bill Liu & Haohui Mai - AthenaX : Uber’s streaming pro...
Flink Forward SF 2017: Bill Liu & Haohui Mai - AthenaX : Uber’s streaming pro...Flink Forward
 
Flink Forward SF 2017: Joe Olson - Using Flink and Queryable State to Buffer ...
Flink Forward SF 2017: Joe Olson - Using Flink and Queryable State to Buffer ...Flink Forward SF 2017: Joe Olson - Using Flink and Queryable State to Buffer ...
Flink Forward SF 2017: Joe Olson - Using Flink and Queryable State to Buffer ...Flink Forward
 
Kafka Summit NYC 2017 - Introducing Exactly Once Semantics in Apache Kafka
Kafka Summit NYC 2017 - Introducing Exactly Once Semantics in Apache KafkaKafka Summit NYC 2017 - Introducing Exactly Once Semantics in Apache Kafka
Kafka Summit NYC 2017 - Introducing Exactly Once Semantics in Apache Kafkaconfluent
 
How to manage large amounts of data with akka streams
How to manage large amounts of data with akka streamsHow to manage large amounts of data with akka streams
How to manage large amounts of data with akka streamsIgor Mielientiev
 
Journey into Reactive Streams and Akka Streams
Journey into Reactive Streams and Akka StreamsJourney into Reactive Streams and Akka Streams
Journey into Reactive Streams and Akka StreamsKevin Webber
 
A dive into akka streams: from the basics to a real-world scenario
A dive into akka streams: from the basics to a real-world scenarioA dive into akka streams: from the basics to a real-world scenario
A dive into akka streams: from the basics to a real-world scenarioGioia Ballin
 

What's hot (20)

Back-Pressure in Action: Handling High-Burst Workloads with Akka Streams & Ka...
Back-Pressure in Action: Handling High-Burst Workloads with Akka Streams & Ka...Back-Pressure in Action: Handling High-Burst Workloads with Akka Streams & Ka...
Back-Pressure in Action: Handling High-Burst Workloads with Akka Streams & Ka...
 
Spark Summit EU talk by Luc Bourlier
Spark Summit EU talk by Luc BourlierSpark Summit EU talk by Luc Bourlier
Spark Summit EU talk by Luc Bourlier
 
Flink Forward SF 2017: Chinmay Soman - Real Time Analytics in the real World ...
Flink Forward SF 2017: Chinmay Soman - Real Time Analytics in the real World ...Flink Forward SF 2017: Chinmay Soman - Real Time Analytics in the real World ...
Flink Forward SF 2017: Chinmay Soman - Real Time Analytics in the real World ...
 
Akka streams
Akka streamsAkka streams
Akka streams
 
Flink Forward SF 2017: David Hardwick, Sean Hester & David Brelloch - Dynami...
Flink Forward SF 2017: David Hardwick, Sean Hester & David Brelloch -  Dynami...Flink Forward SF 2017: David Hardwick, Sean Hester & David Brelloch -  Dynami...
Flink Forward SF 2017: David Hardwick, Sean Hester & David Brelloch - Dynami...
 
Flink Forward SF 2017: James Malone - Make The Cloud Work For You
Flink Forward SF 2017: James Malone - Make The Cloud Work For YouFlink Forward SF 2017: James Malone - Make The Cloud Work For You
Flink Forward SF 2017: James Malone - Make The Cloud Work For You
 
Introduction to Apache Beam & No Shard Left Behind: APIs for Massive Parallel...
Introduction to Apache Beam & No Shard Left Behind: APIs for Massive Parallel...Introduction to Apache Beam & No Shard Left Behind: APIs for Massive Parallel...
Introduction to Apache Beam & No Shard Left Behind: APIs for Massive Parallel...
 
Big Data Day LA 2016/ Big Data Track - Portable Stream and Batch Processing w...
Big Data Day LA 2016/ Big Data Track - Portable Stream and Batch Processing w...Big Data Day LA 2016/ Big Data Track - Portable Stream and Batch Processing w...
Big Data Day LA 2016/ Big Data Track - Portable Stream and Batch Processing w...
 
S3, Cassandra or Outer Space? Dumping Time Series Data using Spark - Demi Ben...
S3, Cassandra or Outer Space? Dumping Time Series Data using Spark - Demi Ben...S3, Cassandra or Outer Space? Dumping Time Series Data using Spark - Demi Ben...
S3, Cassandra or Outer Space? Dumping Time Series Data using Spark - Demi Ben...
 
Uber Real Time Data Analytics
Uber Real Time Data AnalyticsUber Real Time Data Analytics
Uber Real Time Data Analytics
 
Flink Forward SF 2017: Kenneth Knowles - Back to Sessions overview
Flink Forward SF 2017: Kenneth Knowles - Back to Sessions overviewFlink Forward SF 2017: Kenneth Knowles - Back to Sessions overview
Flink Forward SF 2017: Kenneth Knowles - Back to Sessions overview
 
Kafka Summit NYC 2017 - Building Advanced Streaming Applications using the La...
Kafka Summit NYC 2017 - Building Advanced Streaming Applications using the La...Kafka Summit NYC 2017 - Building Advanced Streaming Applications using the La...
Kafka Summit NYC 2017 - Building Advanced Streaming Applications using the La...
 
Building Data Product Based on Apache Spark at Airbnb with Jingwei Lu and Liy...
Building Data Product Based on Apache Spark at Airbnb with Jingwei Lu and Liy...Building Data Product Based on Apache Spark at Airbnb with Jingwei Lu and Liy...
Building Data Product Based on Apache Spark at Airbnb with Jingwei Lu and Liy...
 
Akka streams scala italy2015
Akka streams scala italy2015Akka streams scala italy2015
Akka streams scala italy2015
 
Flink Forward SF 2017: Bill Liu & Haohui Mai - AthenaX : Uber’s streaming pro...
Flink Forward SF 2017: Bill Liu & Haohui Mai - AthenaX : Uber’s streaming pro...Flink Forward SF 2017: Bill Liu & Haohui Mai - AthenaX : Uber’s streaming pro...
Flink Forward SF 2017: Bill Liu & Haohui Mai - AthenaX : Uber’s streaming pro...
 
Flink Forward SF 2017: Joe Olson - Using Flink and Queryable State to Buffer ...
Flink Forward SF 2017: Joe Olson - Using Flink and Queryable State to Buffer ...Flink Forward SF 2017: Joe Olson - Using Flink and Queryable State to Buffer ...
Flink Forward SF 2017: Joe Olson - Using Flink and Queryable State to Buffer ...
 
Kafka Summit NYC 2017 - Introducing Exactly Once Semantics in Apache Kafka
Kafka Summit NYC 2017 - Introducing Exactly Once Semantics in Apache KafkaKafka Summit NYC 2017 - Introducing Exactly Once Semantics in Apache Kafka
Kafka Summit NYC 2017 - Introducing Exactly Once Semantics in Apache Kafka
 
How to manage large amounts of data with akka streams
How to manage large amounts of data with akka streamsHow to manage large amounts of data with akka streams
How to manage large amounts of data with akka streams
 
Journey into Reactive Streams and Akka Streams
Journey into Reactive Streams and Akka StreamsJourney into Reactive Streams and Akka Streams
Journey into Reactive Streams and Akka Streams
 
A dive into akka streams: from the basics to a real-world scenario
A dive into akka streams: from the basics to a real-world scenarioA dive into akka streams: from the basics to a real-world scenario
A dive into akka streams: from the basics to a real-world scenario
 

Viewers also liked

Understanding Akka Streams, Back Pressure, and Asynchronous Architectures
Understanding Akka Streams, Back Pressure, and Asynchronous ArchitecturesUnderstanding Akka Streams, Back Pressure, and Asynchronous Architectures
Understanding Akka Streams, Back Pressure, and Asynchronous ArchitecturesLightbend
 
Akka-chan's Survival Guide for the Streaming World
Akka-chan's Survival Guide for the Streaming WorldAkka-chan's Survival Guide for the Streaming World
Akka-chan's Survival Guide for the Streaming WorldKonrad Malawski
 
Automated conflict resolution - enabling masterless data distribution (Rune S...
Automated conflict resolution - enabling masterless data distribution (Rune S...Automated conflict resolution - enabling masterless data distribution (Rune S...
Automated conflict resolution - enabling masterless data distribution (Rune S...Swiss Big Data User Group
 
Risking Everything with Akka Streams
Risking Everything with Akka StreamsRisking Everything with Akka Streams
Risking Everything with Akka Streamsjohofer
 
Spark streaming , Spark SQL
Spark streaming , Spark SQLSpark streaming , Spark SQL
Spark streaming , Spark SQLYousun Jeong
 
Fault Tolerance and Processing Semantics in Apache Apex
Fault Tolerance and Processing Semantics in Apache ApexFault Tolerance and Processing Semantics in Apache Apex
Fault Tolerance and Processing Semantics in Apache ApexApache Apex Organizer
 
Hadoop Summit SJ 2016: Next Gen Big Data Analytics with Apache Apex
Hadoop Summit SJ 2016: Next Gen Big Data Analytics with Apache ApexHadoop Summit SJ 2016: Next Gen Big Data Analytics with Apache Apex
Hadoop Summit SJ 2016: Next Gen Big Data Analytics with Apache ApexApache Apex
 
Small intro to Big Data - Old version
Small intro to Big Data - Old versionSmall intro to Big Data - Old version
Small intro to Big Data - Old versionSoftwareMill
 
Bonnes pratiques anti-DDOS
Bonnes pratiques anti-DDOSBonnes pratiques anti-DDOS
Bonnes pratiques anti-DDOSJulien SIMON
 
End to End Akka Streams / Reactive Streams - from Business to Socket
End to End Akka Streams / Reactive Streams - from Business to SocketEnd to End Akka Streams / Reactive Streams - from Business to Socket
End to End Akka Streams / Reactive Streams - from Business to SocketKonrad Malawski
 
fluent-plugin-norikra #fluentdcasual
fluent-plugin-norikra #fluentdcasualfluent-plugin-norikra #fluentdcasual
fluent-plugin-norikra #fluentdcasualSATOSHI TAGOMORI
 
Impacto de las tic en la música1
Impacto de las tic en la música1Impacto de las tic en la música1
Impacto de las tic en la música1Isa Lopez
 
DigitalDay Verona 13 Aprile 2015 - Evento Ordine Ingegneri VR
DigitalDay Verona 13 Aprile 2015 - Evento Ordine Ingegneri VRDigitalDay Verona 13 Aprile 2015 - Evento Ordine Ingegneri VR
DigitalDay Verona 13 Aprile 2015 - Evento Ordine Ingegneri VRDomenico Barile
 
3 d pie chart circular puzzle with hole in center pieces 6 stages style 4 pow...
3 d pie chart circular puzzle with hole in center pieces 6 stages style 4 pow...3 d pie chart circular puzzle with hole in center pieces 6 stages style 4 pow...
3 d pie chart circular puzzle with hole in center pieces 6 stages style 4 pow...SlideTeam.net
 

Viewers also liked (20)

Understanding Akka Streams, Back Pressure, and Asynchronous Architectures
Understanding Akka Streams, Back Pressure, and Asynchronous ArchitecturesUnderstanding Akka Streams, Back Pressure, and Asynchronous Architectures
Understanding Akka Streams, Back Pressure, and Asynchronous Architectures
 
Patterns of resilience
Patterns of resiliencePatterns of resilience
Patterns of resilience
 
Akka-chan's Survival Guide for the Streaming World
Akka-chan's Survival Guide for the Streaming WorldAkka-chan's Survival Guide for the Streaming World
Akka-chan's Survival Guide for the Streaming World
 
Automated conflict resolution - enabling masterless data distribution (Rune S...
Automated conflict resolution - enabling masterless data distribution (Rune S...Automated conflict resolution - enabling masterless data distribution (Rune S...
Automated conflict resolution - enabling masterless data distribution (Rune S...
 
Boolean operators
Boolean operatorsBoolean operators
Boolean operators
 
Risking Everything with Akka Streams
Risking Everything with Akka StreamsRisking Everything with Akka Streams
Risking Everything with Akka Streams
 
Streaming SQL
Streaming SQLStreaming SQL
Streaming SQL
 
Spark streaming , Spark SQL
Spark streaming , Spark SQLSpark streaming , Spark SQL
Spark streaming , Spark SQL
 
Fault Tolerance and Processing Semantics in Apache Apex
Fault Tolerance and Processing Semantics in Apache ApexFault Tolerance and Processing Semantics in Apache Apex
Fault Tolerance and Processing Semantics in Apache Apex
 
Sensing the world with Data of Things
Sensing the world with Data of ThingsSensing the world with Data of Things
Sensing the world with Data of Things
 
Hadoop Summit SJ 2016: Next Gen Big Data Analytics with Apache Apex
Hadoop Summit SJ 2016: Next Gen Big Data Analytics with Apache ApexHadoop Summit SJ 2016: Next Gen Big Data Analytics with Apache Apex
Hadoop Summit SJ 2016: Next Gen Big Data Analytics with Apache Apex
 
Small intro to Big Data - Old version
Small intro to Big Data - Old versionSmall intro to Big Data - Old version
Small intro to Big Data - Old version
 
Bonnes pratiques anti-DDOS
Bonnes pratiques anti-DDOSBonnes pratiques anti-DDOS
Bonnes pratiques anti-DDOS
 
End to End Akka Streams / Reactive Streams - from Business to Socket
End to End Akka Streams / Reactive Streams - from Business to SocketEnd to End Akka Streams / Reactive Streams - from Business to Socket
End to End Akka Streams / Reactive Streams - from Business to Socket
 
fluent-plugin-norikra #fluentdcasual
fluent-plugin-norikra #fluentdcasualfluent-plugin-norikra #fluentdcasual
fluent-plugin-norikra #fluentdcasual
 
Impacto de las tic en la música1
Impacto de las tic en la música1Impacto de las tic en la música1
Impacto de las tic en la música1
 
What Are the Main Subtypes of Breast Cancer?
What Are the Main Subtypes of Breast Cancer?What Are the Main Subtypes of Breast Cancer?
What Are the Main Subtypes of Breast Cancer?
 
DigitalDay Verona 13 Aprile 2015 - Evento Ordine Ingegneri VR
DigitalDay Verona 13 Aprile 2015 - Evento Ordine Ingegneri VRDigitalDay Verona 13 Aprile 2015 - Evento Ordine Ingegneri VR
DigitalDay Verona 13 Aprile 2015 - Evento Ordine Ingegneri VR
 
3 d pie chart circular puzzle with hole in center pieces 6 stages style 4 pow...
3 d pie chart circular puzzle with hole in center pieces 6 stages style 4 pow...3 d pie chart circular puzzle with hole in center pieces 6 stages style 4 pow...
3 d pie chart circular puzzle with hole in center pieces 6 stages style 4 pow...
 
Proteomexchange
ProteomexchangeProteomexchange
Proteomexchange
 

Similar to Gearpump akka streams

Presto query optimizer: pursuit of performance
Presto query optimizer: pursuit of performancePresto query optimizer: pursuit of performance
Presto query optimizer: pursuit of performanceDataWorks Summit
 
Use C++ and Intel® Threading Building Blocks (Intel® TBB) for Hardware Progra...
Use C++ and Intel® Threading Building Blocks (Intel® TBB) for Hardware Progra...Use C++ and Intel® Threading Building Blocks (Intel® TBB) for Hardware Progra...
Use C++ and Intel® Threading Building Blocks (Intel® TBB) for Hardware Progra...Intel® Software
 
Greg Hogan – To Petascale and Beyond- Apache Flink in the Clouds
Greg Hogan – To Petascale and Beyond- Apache Flink in the CloudsGreg Hogan – To Petascale and Beyond- Apache Flink in the Clouds
Greg Hogan – To Petascale and Beyond- Apache Flink in the CloudsFlink Forward
 
Hadoop and HBase experiences in perf log project
Hadoop and HBase experiences in perf log projectHadoop and HBase experiences in perf log project
Hadoop and HBase experiences in perf log projectMao Geng
 
Cassandra Exports as a Trivially Parallelizable Problem (Emilio Del Tessandor...
Cassandra Exports as a Trivially Parallelizable Problem (Emilio Del Tessandor...Cassandra Exports as a Trivially Parallelizable Problem (Emilio Del Tessandor...
Cassandra Exports as a Trivially Parallelizable Problem (Emilio Del Tessandor...DataStax
 
MACHBASE_NEO
MACHBASE_NEOMACHBASE_NEO
MACHBASE_NEOMACHBASE
 
Scio - A Scala API for Google Cloud Dataflow & Apache Beam
Scio - A Scala API for Google Cloud Dataflow & Apache BeamScio - A Scala API for Google Cloud Dataflow & Apache Beam
Scio - A Scala API for Google Cloud Dataflow & Apache BeamNeville Li
 
Apache Calcite: One Frontend to Rule Them All
Apache Calcite: One Frontend to Rule Them AllApache Calcite: One Frontend to Rule Them All
Apache Calcite: One Frontend to Rule Them AllMichael Mior
 
Spark (Structured) Streaming vs. Kafka Streams - two stream processing platfo...
Spark (Structured) Streaming vs. Kafka Streams - two stream processing platfo...Spark (Structured) Streaming vs. Kafka Streams - two stream processing platfo...
Spark (Structured) Streaming vs. Kafka Streams - two stream processing platfo...Guido Schmutz
 
Replicate from Oracle to Oracle, Oracle to MySQL, and Oracle to Analytics
Replicate from Oracle to Oracle, Oracle to MySQL, and Oracle to AnalyticsReplicate from Oracle to Oracle, Oracle to MySQL, and Oracle to Analytics
Replicate from Oracle to Oracle, Oracle to MySQL, and Oracle to AnalyticsContinuent
 
How to Build a Telegraf Plugin by Noah Crowley
How to Build a Telegraf Plugin by Noah CrowleyHow to Build a Telegraf Plugin by Noah Crowley
How to Build a Telegraf Plugin by Noah CrowleyInfluxData
 
Building a Telegraf Plugin by Noah Crowly | Developer Advocate | InfluxData
Building a Telegraf Plugin by Noah Crowly | Developer Advocate | InfluxDataBuilding a Telegraf Plugin by Noah Crowly | Developer Advocate | InfluxData
Building a Telegraf Plugin by Noah Crowly | Developer Advocate | InfluxDataInfluxData
 
Porting a Streaming Pipeline from Scala to Rust
Porting a Streaming Pipeline from Scala to RustPorting a Streaming Pipeline from Scala to Rust
Porting a Streaming Pipeline from Scala to RustEvan Chan
 
Free GitOps Workshop + Intro to Kubernetes & GitOps
Free GitOps Workshop + Intro to Kubernetes & GitOpsFree GitOps Workshop + Intro to Kubernetes & GitOps
Free GitOps Workshop + Intro to Kubernetes & GitOpsWeaveworks
 
Tungsten Use Case: How Gittigidiyor (a subsidiary of eBay) Replicates Data In...
Tungsten Use Case: How Gittigidiyor (a subsidiary of eBay) Replicates Data In...Tungsten Use Case: How Gittigidiyor (a subsidiary of eBay) Replicates Data In...
Tungsten Use Case: How Gittigidiyor (a subsidiary of eBay) Replicates Data In...Continuent
 
Replicate Oracle to Oracle, Oracle to MySQL, and Oracle to Analytics
Replicate Oracle to Oracle, Oracle to MySQL, and Oracle to AnalyticsReplicate Oracle to Oracle, Oracle to MySQL, and Oracle to Analytics
Replicate Oracle to Oracle, Oracle to MySQL, and Oracle to AnalyticsLinas Virbalas
 
Fast federated SQL with Apache Calcite
Fast federated SQL with Apache CalciteFast federated SQL with Apache Calcite
Fast federated SQL with Apache CalciteChris Baynes
 
Stream Processing in the Cloud - Athens Kubernetes Meetup 16.07.2019
Stream Processing in the Cloud - Athens Kubernetes Meetup 16.07.2019Stream Processing in the Cloud - Athens Kubernetes Meetup 16.07.2019
Stream Processing in the Cloud - Athens Kubernetes Meetup 16.07.2019Rafał Leszko
 
The Flink - Apache Bigtop integration
The Flink - Apache Bigtop integrationThe Flink - Apache Bigtop integration
The Flink - Apache Bigtop integrationMárton Balassi
 
Java High Level Stream API
Java High Level Stream APIJava High Level Stream API
Java High Level Stream APIApache Apex
 

Similar to Gearpump akka streams (20)

Presto query optimizer: pursuit of performance
Presto query optimizer: pursuit of performancePresto query optimizer: pursuit of performance
Presto query optimizer: pursuit of performance
 
Use C++ and Intel® Threading Building Blocks (Intel® TBB) for Hardware Progra...
Use C++ and Intel® Threading Building Blocks (Intel® TBB) for Hardware Progra...Use C++ and Intel® Threading Building Blocks (Intel® TBB) for Hardware Progra...
Use C++ and Intel® Threading Building Blocks (Intel® TBB) for Hardware Progra...
 
Greg Hogan – To Petascale and Beyond- Apache Flink in the Clouds
Greg Hogan – To Petascale and Beyond- Apache Flink in the CloudsGreg Hogan – To Petascale and Beyond- Apache Flink in the Clouds
Greg Hogan – To Petascale and Beyond- Apache Flink in the Clouds
 
Hadoop and HBase experiences in perf log project
Hadoop and HBase experiences in perf log projectHadoop and HBase experiences in perf log project
Hadoop and HBase experiences in perf log project
 
Cassandra Exports as a Trivially Parallelizable Problem (Emilio Del Tessandor...
Cassandra Exports as a Trivially Parallelizable Problem (Emilio Del Tessandor...Cassandra Exports as a Trivially Parallelizable Problem (Emilio Del Tessandor...
Cassandra Exports as a Trivially Parallelizable Problem (Emilio Del Tessandor...
 
MACHBASE_NEO
MACHBASE_NEOMACHBASE_NEO
MACHBASE_NEO
 
Scio - A Scala API for Google Cloud Dataflow & Apache Beam
Scio - A Scala API for Google Cloud Dataflow & Apache BeamScio - A Scala API for Google Cloud Dataflow & Apache Beam
Scio - A Scala API for Google Cloud Dataflow & Apache Beam
 
Apache Calcite: One Frontend to Rule Them All
Apache Calcite: One Frontend to Rule Them AllApache Calcite: One Frontend to Rule Them All
Apache Calcite: One Frontend to Rule Them All
 
Spark (Structured) Streaming vs. Kafka Streams - two stream processing platfo...
Spark (Structured) Streaming vs. Kafka Streams - two stream processing platfo...Spark (Structured) Streaming vs. Kafka Streams - two stream processing platfo...
Spark (Structured) Streaming vs. Kafka Streams - two stream processing platfo...
 
Replicate from Oracle to Oracle, Oracle to MySQL, and Oracle to Analytics
Replicate from Oracle to Oracle, Oracle to MySQL, and Oracle to AnalyticsReplicate from Oracle to Oracle, Oracle to MySQL, and Oracle to Analytics
Replicate from Oracle to Oracle, Oracle to MySQL, and Oracle to Analytics
 
How to Build a Telegraf Plugin by Noah Crowley
How to Build a Telegraf Plugin by Noah CrowleyHow to Build a Telegraf Plugin by Noah Crowley
How to Build a Telegraf Plugin by Noah Crowley
 
Building a Telegraf Plugin by Noah Crowly | Developer Advocate | InfluxData
Building a Telegraf Plugin by Noah Crowly | Developer Advocate | InfluxDataBuilding a Telegraf Plugin by Noah Crowly | Developer Advocate | InfluxData
Building a Telegraf Plugin by Noah Crowly | Developer Advocate | InfluxData
 
Porting a Streaming Pipeline from Scala to Rust
Porting a Streaming Pipeline from Scala to RustPorting a Streaming Pipeline from Scala to Rust
Porting a Streaming Pipeline from Scala to Rust
 
Free GitOps Workshop + Intro to Kubernetes & GitOps
Free GitOps Workshop + Intro to Kubernetes & GitOpsFree GitOps Workshop + Intro to Kubernetes & GitOps
Free GitOps Workshop + Intro to Kubernetes & GitOps
 
Tungsten Use Case: How Gittigidiyor (a subsidiary of eBay) Replicates Data In...
Tungsten Use Case: How Gittigidiyor (a subsidiary of eBay) Replicates Data In...Tungsten Use Case: How Gittigidiyor (a subsidiary of eBay) Replicates Data In...
Tungsten Use Case: How Gittigidiyor (a subsidiary of eBay) Replicates Data In...
 
Replicate Oracle to Oracle, Oracle to MySQL, and Oracle to Analytics
Replicate Oracle to Oracle, Oracle to MySQL, and Oracle to AnalyticsReplicate Oracle to Oracle, Oracle to MySQL, and Oracle to Analytics
Replicate Oracle to Oracle, Oracle to MySQL, and Oracle to Analytics
 
Fast federated SQL with Apache Calcite
Fast federated SQL with Apache CalciteFast federated SQL with Apache Calcite
Fast federated SQL with Apache Calcite
 
Stream Processing in the Cloud - Athens Kubernetes Meetup 16.07.2019
Stream Processing in the Cloud - Athens Kubernetes Meetup 16.07.2019Stream Processing in the Cloud - Athens Kubernetes Meetup 16.07.2019
Stream Processing in the Cloud - Athens Kubernetes Meetup 16.07.2019
 
The Flink - Apache Bigtop integration
The Flink - Apache Bigtop integrationThe Flink - Apache Bigtop integration
The Flink - Apache Bigtop integration
 
Java High Level Stream API
Java High Level Stream APIJava High Level Stream API
Java High Level Stream API
 

Recently uploaded

cybersecurity notes for mca students for learning
cybersecurity notes for mca students for learningcybersecurity notes for mca students for learning
cybersecurity notes for mca students for learningVitsRangannavar
 
What is Binary Language? Computer Number Systems
What is Binary Language?  Computer Number SystemsWhat is Binary Language?  Computer Number Systems
What is Binary Language? Computer Number SystemsJheuzeDellosa
 
Der Spagat zwischen BIAS und FAIRNESS (2024)
Der Spagat zwischen BIAS und FAIRNESS (2024)Der Spagat zwischen BIAS und FAIRNESS (2024)
Der Spagat zwischen BIAS und FAIRNESS (2024)OPEN KNOWLEDGE GmbH
 
Building a General PDE Solving Framework with Symbolic-Numeric Scientific Mac...
Building a General PDE Solving Framework with Symbolic-Numeric Scientific Mac...Building a General PDE Solving Framework with Symbolic-Numeric Scientific Mac...
Building a General PDE Solving Framework with Symbolic-Numeric Scientific Mac...stazi3110
 
Hand gesture recognition PROJECT PPT.pptx
Hand gesture recognition PROJECT PPT.pptxHand gesture recognition PROJECT PPT.pptx
Hand gesture recognition PROJECT PPT.pptxbodapatigopi8531
 
Call Girls in Naraina Delhi 💯Call Us 🔝8264348440🔝
Call Girls in Naraina Delhi 💯Call Us 🔝8264348440🔝Call Girls in Naraina Delhi 💯Call Us 🔝8264348440🔝
Call Girls in Naraina Delhi 💯Call Us 🔝8264348440🔝soniya singh
 
Optimizing AI for immediate response in Smart CCTV
Optimizing AI for immediate response in Smart CCTVOptimizing AI for immediate response in Smart CCTV
Optimizing AI for immediate response in Smart CCTVshikhaohhpro
 
Professional Resume Template for Software Developers
Professional Resume Template for Software DevelopersProfessional Resume Template for Software Developers
Professional Resume Template for Software DevelopersVinodh Ram
 
XpertSolvers: Your Partner in Building Innovative Software Solutions
XpertSolvers: Your Partner in Building Innovative Software SolutionsXpertSolvers: Your Partner in Building Innovative Software Solutions
XpertSolvers: Your Partner in Building Innovative Software SolutionsMehedi Hasan Shohan
 
chapter--4-software-project-planning.ppt
chapter--4-software-project-planning.pptchapter--4-software-project-planning.ppt
chapter--4-software-project-planning.pptkotipi9215
 
Learn the Fundamentals of XCUITest Framework_ A Beginner's Guide.pdf
Learn the Fundamentals of XCUITest Framework_ A Beginner's Guide.pdfLearn the Fundamentals of XCUITest Framework_ A Beginner's Guide.pdf
Learn the Fundamentals of XCUITest Framework_ A Beginner's Guide.pdfkalichargn70th171
 
BATTLEFIELD ORM: TIPS, TACTICS AND STRATEGIES FOR CONQUERING YOUR DATABASE
BATTLEFIELD ORM: TIPS, TACTICS AND STRATEGIES FOR CONQUERING YOUR DATABASEBATTLEFIELD ORM: TIPS, TACTICS AND STRATEGIES FOR CONQUERING YOUR DATABASE
BATTLEFIELD ORM: TIPS, TACTICS AND STRATEGIES FOR CONQUERING YOUR DATABASEOrtus Solutions, Corp
 
What is Fashion PLM and Why Do You Need It
What is Fashion PLM and Why Do You Need ItWhat is Fashion PLM and Why Do You Need It
What is Fashion PLM and Why Do You Need ItWave PLM
 
(Genuine) Escort Service Lucknow | Starting ₹,5K To @25k with A/C 🧑🏽‍❤️‍🧑🏻 89...
(Genuine) Escort Service Lucknow | Starting ₹,5K To @25k with A/C 🧑🏽‍❤️‍🧑🏻 89...(Genuine) Escort Service Lucknow | Starting ₹,5K To @25k with A/C 🧑🏽‍❤️‍🧑🏻 89...
(Genuine) Escort Service Lucknow | Starting ₹,5K To @25k with A/C 🧑🏽‍❤️‍🧑🏻 89...gurkirankumar98700
 
ODSC - Batch to Stream workshop - integration of Apache Spark, Cassandra, Pos...
ODSC - Batch to Stream workshop - integration of Apache Spark, Cassandra, Pos...ODSC - Batch to Stream workshop - integration of Apache Spark, Cassandra, Pos...
ODSC - Batch to Stream workshop - integration of Apache Spark, Cassandra, Pos...Christina Lin
 
Engage Usergroup 2024 - The Good The Bad_The Ugly
Engage Usergroup 2024 - The Good The Bad_The UglyEngage Usergroup 2024 - The Good The Bad_The Ugly
Engage Usergroup 2024 - The Good The Bad_The UglyFrank van der Linden
 
The Evolution of Karaoke From Analog to App.pdf
The Evolution of Karaoke From Analog to App.pdfThe Evolution of Karaoke From Analog to App.pdf
The Evolution of Karaoke From Analog to App.pdfPower Karaoke
 
DNT_Corporate presentation know about us
DNT_Corporate presentation know about usDNT_Corporate presentation know about us
DNT_Corporate presentation know about usDynamic Netsoft
 
Building Real-Time Data Pipelines: Stream & Batch Processing workshop Slide
Building Real-Time Data Pipelines: Stream & Batch Processing workshop SlideBuilding Real-Time Data Pipelines: Stream & Batch Processing workshop Slide
Building Real-Time Data Pipelines: Stream & Batch Processing workshop SlideChristina Lin
 

Recently uploaded (20)

cybersecurity notes for mca students for learning
cybersecurity notes for mca students for learningcybersecurity notes for mca students for learning
cybersecurity notes for mca students for learning
 
What is Binary Language? Computer Number Systems
What is Binary Language?  Computer Number SystemsWhat is Binary Language?  Computer Number Systems
What is Binary Language? Computer Number Systems
 
Der Spagat zwischen BIAS und FAIRNESS (2024)
Der Spagat zwischen BIAS und FAIRNESS (2024)Der Spagat zwischen BIAS und FAIRNESS (2024)
Der Spagat zwischen BIAS und FAIRNESS (2024)
 
Building a General PDE Solving Framework with Symbolic-Numeric Scientific Mac...
Building a General PDE Solving Framework with Symbolic-Numeric Scientific Mac...Building a General PDE Solving Framework with Symbolic-Numeric Scientific Mac...
Building a General PDE Solving Framework with Symbolic-Numeric Scientific Mac...
 
Hand gesture recognition PROJECT PPT.pptx
Hand gesture recognition PROJECT PPT.pptxHand gesture recognition PROJECT PPT.pptx
Hand gesture recognition PROJECT PPT.pptx
 
Call Girls in Naraina Delhi 💯Call Us 🔝8264348440🔝
Call Girls in Naraina Delhi 💯Call Us 🔝8264348440🔝Call Girls in Naraina Delhi 💯Call Us 🔝8264348440🔝
Call Girls in Naraina Delhi 💯Call Us 🔝8264348440🔝
 
Optimizing AI for immediate response in Smart CCTV
Optimizing AI for immediate response in Smart CCTVOptimizing AI for immediate response in Smart CCTV
Optimizing AI for immediate response in Smart CCTV
 
Professional Resume Template for Software Developers
Professional Resume Template for Software DevelopersProfessional Resume Template for Software Developers
Professional Resume Template for Software Developers
 
XpertSolvers: Your Partner in Building Innovative Software Solutions
XpertSolvers: Your Partner in Building Innovative Software SolutionsXpertSolvers: Your Partner in Building Innovative Software Solutions
XpertSolvers: Your Partner in Building Innovative Software Solutions
 
chapter--4-software-project-planning.ppt
chapter--4-software-project-planning.pptchapter--4-software-project-planning.ppt
chapter--4-software-project-planning.ppt
 
Call Girls In Mukherjee Nagar 📱 9999965857 🤩 Delhi 🫦 HOT AND SEXY VVIP 🍎 SE...
Call Girls In Mukherjee Nagar 📱  9999965857  🤩 Delhi 🫦 HOT AND SEXY VVIP 🍎 SE...Call Girls In Mukherjee Nagar 📱  9999965857  🤩 Delhi 🫦 HOT AND SEXY VVIP 🍎 SE...
Call Girls In Mukherjee Nagar 📱 9999965857 🤩 Delhi 🫦 HOT AND SEXY VVIP 🍎 SE...
 
Learn the Fundamentals of XCUITest Framework_ A Beginner's Guide.pdf
Learn the Fundamentals of XCUITest Framework_ A Beginner's Guide.pdfLearn the Fundamentals of XCUITest Framework_ A Beginner's Guide.pdf
Learn the Fundamentals of XCUITest Framework_ A Beginner's Guide.pdf
 
BATTLEFIELD ORM: TIPS, TACTICS AND STRATEGIES FOR CONQUERING YOUR DATABASE
BATTLEFIELD ORM: TIPS, TACTICS AND STRATEGIES FOR CONQUERING YOUR DATABASEBATTLEFIELD ORM: TIPS, TACTICS AND STRATEGIES FOR CONQUERING YOUR DATABASE
BATTLEFIELD ORM: TIPS, TACTICS AND STRATEGIES FOR CONQUERING YOUR DATABASE
 
What is Fashion PLM and Why Do You Need It
What is Fashion PLM and Why Do You Need ItWhat is Fashion PLM and Why Do You Need It
What is Fashion PLM and Why Do You Need It
 
(Genuine) Escort Service Lucknow | Starting ₹,5K To @25k with A/C 🧑🏽‍❤️‍🧑🏻 89...
(Genuine) Escort Service Lucknow | Starting ₹,5K To @25k with A/C 🧑🏽‍❤️‍🧑🏻 89...(Genuine) Escort Service Lucknow | Starting ₹,5K To @25k with A/C 🧑🏽‍❤️‍🧑🏻 89...
(Genuine) Escort Service Lucknow | Starting ₹,5K To @25k with A/C 🧑🏽‍❤️‍🧑🏻 89...
 
ODSC - Batch to Stream workshop - integration of Apache Spark, Cassandra, Pos...
ODSC - Batch to Stream workshop - integration of Apache Spark, Cassandra, Pos...ODSC - Batch to Stream workshop - integration of Apache Spark, Cassandra, Pos...
ODSC - Batch to Stream workshop - integration of Apache Spark, Cassandra, Pos...
 
Engage Usergroup 2024 - The Good The Bad_The Ugly
Engage Usergroup 2024 - The Good The Bad_The UglyEngage Usergroup 2024 - The Good The Bad_The Ugly
Engage Usergroup 2024 - The Good The Bad_The Ugly
 
The Evolution of Karaoke From Analog to App.pdf
The Evolution of Karaoke From Analog to App.pdfThe Evolution of Karaoke From Analog to App.pdf
The Evolution of Karaoke From Analog to App.pdf
 
DNT_Corporate presentation know about us
DNT_Corporate presentation know about usDNT_Corporate presentation know about us
DNT_Corporate presentation know about us
 
Building Real-Time Data Pipelines: Stream & Batch Processing workshop Slide
Building Real-Time Data Pipelines: Stream & Batch Processing workshop SlideBuilding Real-Time Data Pipelines: Stream & Batch Processing workshop Slide
Building Real-Time Data Pipelines: Stream & Batch Processing workshop Slide
 

Gearpump akka streams

  • 1. Implementing an akka-streams materializer for big data The Gearpump Materializer Kam Kasravi
  • 2. Technical Presentation ● Familiarity with akka-streams flow and graph DSL’s ● Familiarity with big data and real time streaming platforms ● Familiarity with scala ● Effort between the akka-streams and Gearpump teams started late last year ● Resulted in a number of pull requests into akka-streams to enable different materializers ● Close to completion with good support of the akka-streams DSL (all GraphStages) ● Fairly seamless to switch between local and distributed
  • 3. Who am I? ● Committer on Apache Gearpump (incubating) - http://gearpump.apache.org ● Architect on Trusted Analytics Platform (TAP) - http://trustedanalytics.org ● Lead or Architect across many companies, industries - NYSE, eBay, PayPal, Yahoo, ... Title Goes Here There are many variations of passages of lorem ipsum available, but the majority suffered alteration some form.
  • 4. What is Apache Gearpump? ● Accepted into Apache incubator last March ● Similar to Apache Beam and Apache Flink (real-time message delivery) ● Heavily leverages the actor model and akka (more so than others) ● Unique features like dynamic DAG ● Excellent runtime visualization tooling of cluster and application DAGs ● One of the best big data performance profiles (both throughput, latency)
  • 5. Agenda ● Why? ○ Why integrate akka-streams into a big data platform? ● Big Data platform evolving features ○ Functionality big data platforms are embracing ● Prerequisites needed for any Big Data platform ○ Minimal features a big data platform must have ● Big data platform integration challenges ○ What concepts do not map well within big data platforms? ● Object models: akka-streams, Gearpump ● Materialization ○ ActorMaterializer - materializing the module tree ○ GearpumpMaterializer - rewriting the module tree
  • 6. Why? ● Akka-streams has limitations inherent within a single JVM ○ Throughput and latency are key big data features that require scaling beyond single JVM’s ● Akka-streams DSL is a superset of other big data platform DSLs ○ Has a logical plan (declarative) that can be transformed to an execution plan (runtime) ● Akka-streams programming paradigm is declarative, composable, extensible*, stackable* and reusable* * Provides a level of extensibility and functionality beyond most big data platform DSLs
  • 7. Extensible ● Extend GraphStage ● Extend Source, Sink, Flow or BidiFlow ● All derive from Graph * Provides a level of extensibility and functionality beyond most big data platform DSLs
  • 8. Stackable ● Another term for nestable or recursive. Reference to Kleisli (theoretical). ● Source, Sink, Flow or BidiFlow may contain their own topologies * Provides a level of extensibility and functionality beyond most big data platform DSLs
  • 9. Reusable ● Graph topologies can be attached anywhere (any Graph) ● Recent akka-streams feature is dynamic attachment via hubs ● Hubs will take advantage of Gearpump dynamic DAG within the GearpumpMaterializer * Provides a level of extensibility and functionality beyond most big data platform DSLs
  • 10. Big Data platform evolving features (1) ● Big data platforms are moving to consolidate disparate API’s ○ Too many APIs: Concord, Flink, Heron, Pulsar, Spark, Storm, Samza ○ Common DSL is also an approach being taken by Apache Beam ○ Analogy to SQL - common grammar that different platforms execute
  • 11. Big Data platform evolving features (2) ● Big data platforms will increasingly require dynamic pipelines that are compositional and reusable ● Examples include: ○ Machine learning ○ IoT sensors
  • 12. Big Data platform evolving features (3) ● Machine learning use cases ○ Replace or update scoring models ○ Model Ensembles ■ concept drift ■ data drift
  • 13. Big Data platform evolving features (4) ● IoT use cases ○ Bring new sensors on line with no interruption ○ Change or update configuration parameters at remote sensors
  • 14. Prerequisites needed for any Big Data platform (1) Downstream must be able to pull Upstream must be able to push 1. Push and Pull Downstream must be able to backpressure all the way to source 2. Backpressure << <<
  • 15. Prerequisites needed for any Big Data platform (2) 3. Parallelization 4. Asynchronous 5. Bidirectional
  • 16. Big data platform integration challenges (1) A number of GraphStages have completion or cancellation semantics. Big data pipelines are often infinite streams and do not complete. Cancel is often viewed as a failure. ● Balance[T] ● Completion[T] ● Merge[T] ● Split[T]
  • 17. Big data platform integration challenges (2) A number of GraphStages have specific upstream and downstream ordering and timing directives. ● Batch[T] ● Concat[T] ● Delay[T] ● DelayInitial[T] ● Interleave[T]
  • 18. Big data platform integration challenges (3) The async attribute as well as fusing do not map cleanly when distributing GraphStage functionality across machines. ● Graph.async ● Fusing
  • 19. Graph.async ● Collapses multiple operations (GraphStageLogic) into one actor ● Distributed scenarios where one may want actors within the same JVM or on the same machine
  • 20. Fusing ● Creates one or more islands delimited by async boundaries ● For distributed scenario no fusing should occur until the materializer can evaluate and optimize the execution plan
  • 21. Object Models ● Akka-stream’s GraphStage, Module, Shape ● Gearpump’s Graph, Task, Partitioner
  • 22. Akka-streams Object Model ↪ Base type is a Graph. Common base type is a GraphStage ↪ Graph contains a ↳ Module contains a ↳ Shape ↪ Only a RunnableGraph can be materialized ↪ A RunnableGraph needs at least one Source and one Sink
  • 23. Akka-streams Graph[S, M] ● Graph is parameterized by ○ Shape ○ Materialized Value ● Graph contains a Module contains a Shape ○ Module is where the runtime is constructed and manipulated ● Graph’s first level subtypes provide basic functionality ○ Source ○ Sink ○ Flow ○ BidiFlow S M Graph Source Sink Flow BidiFlow Module Shape
  • 25. GraphStage[S <: Shape] subtypes (incomplete) ↳ Balance[T] ↳ Batch[In, Out] ↳ Broadcast[T] ↳ Collect[In, Out] ↳ Concat[T] ↳ DelayInitial[T] ↳ DropWhile[T] ↳ Expand[In, Out] ↳ FlattenMerge[T, M] ↳ Fold[In, Out] ↳ FoldAsync[T] ↳ FutureSource[T] ↳ GroupBy[T, K] ↳ Grouped[T] ↳ GroupedWithin[T] ↳ Interleave[T] ↳ Intersperse[T] ↳ LimitWeighted[T] ↳ Map[In, Out] ↳ MapAsync[In, Out] ↳ Merge[T] ↳ MergePreferred[T] ↳ MergeSorted[T] ↳ OrElse[T] ↳ Partition[T] ↳ PrefixAndTail[T] ↳ Recover[T] ↳ Scan[In, Out] ↳ SimpleLinearGraph[T] ↳ Sliding[T]
  • 26. What about Module? ● Module is a recursive structure containing a Set[Modules] ● Module is a declarative data structure used as the AST ● Module is used to represent a graph of nodes and edges from the original GraphStages ● Module contains downstream and upstream ports (edges) ● Materializers walk the module tree to create and run instances of publishers and subscribers. ● Each publisher and subscriber is an actor (ActorGraphInterpreter)
  • 27. Gearpump Object Model ↪ Graph[Node, Edge] holds ↳ Tasks (Node) ↳ Partitioners (Edge) ↪ This is a Gearpump Graph, not to be confused with akka-streams Graph.
  • 28. Gearpump Graph[N<:Task, E<:Partitioner] ● Graph is parameterized by ○ Node - must be a subtype of Task ○ Edge - must be a subtype of Parititioner N E Graph List[Task] List[Partitioner]
  • 30. GraphTask subtypes (incomplete) ↳ BalanceTask ↳ BatchTask[In, Out] ↳ BroadcastTask[T] ↳ CollectTask[In, Out] ↳ ConcatTask ↳ DelayInitialTask[T] ↳ DropWhileTask[T] ↳ ExpandTask[In, Out] ↳ FlattenMerge[T, M] ↳ FoldTask[In, Out] ↳ FutureSourceTask[T] ↳ GroupByTask[T, K] ↳ GroupedTask[T] ↳ GroupedWithinTask[T] ↳ InterleaveTask[T] ↳ IntersperseTask[T] ↳ LimitWeightedTask[T] ↳ MapTask[In, Out] ↳ MapAsyncTask[In, Out] ↳ MergeTask[T] ↳ OrElseTask[T] ↳ PartitionTask[T] ↳ PrefixAndTailTask[T] ↳ RecoverTask[T] ↳ ScanTask[In, Out] ↳ SlidingTask[T]
  • 31. Materializer Variations 1. AST (module tree) is matched for every module type (GearpumpMaterializer) 2. AST (module tree) is matched for certain module types ○ After distribution - local ActorMaterializer is used for operations on that worker ○ Materializer works more as a distribution coordinator
  • 32. Example 1 Source Broadcast Flow Merge Sink implicit val materializer = ActorMaterializer() val sinkActor = system.actorOf(Props(new SinkActor()) val source = Source((1 to 5)) val sink = Sink.actorRef(sinkActor, "COMPLETE") val flowA: Flow[Int, Int, NotUsed] = Flow[Int].map { x => println(s"processing broadcasted element : $x in flowA"); x } val flowB: Flow[Int, Int, NotUsed] = Flow[Int].map { x => println(s"processing broadcasted element : $x in flowB"); x } val graph = RunnableGraph.fromGraph(GraphDSL.create() { implicit b => val broadcast = b.add(Broadcast[Int](2)) val merge = b.add(Broadcast[Int](2)) source ~> broadcast broadcast ~> flowA ~> merge broadcast ~> flowB ~> merge merge ~> sink ClosedShape }) graph.run()
  • 33. Example 1 implicit val materializer = ActorMaterializer() val sinkActor = system.actorOf(Props(new SinkActor()) val source = Source((1 to 5)) val sink = Sink.actorRef(sinkActor, "COMPLETE") val flowA: Flow[Int, Int, NotUsed] = Flow[Int].map { x => println(s"processing broadcasted element : $x in flowA"); x } val flowB: Flow[Int, Int, NotUsed] = Flow[Int].map { x => println(s"processing broadcasted element : $x in flowB"); x } val graph = RunnableGraph.fromGraph(GraphDSL.create() { implicit b => val broadcast = b.add(Broadcast[Int](2)) val merge = b.add(Broadcast[Int](2)) source ~> broadcast broadcast ~> flowA ~> merge broadcast ~> flowB ~> merge merge ~> sink ClosedShape }) graph.run() Source Broadcast Flow Flow Merge GraphStages Sink class SinkActor extends Actor { def receive: Receive = { case any: Any => println(s“Confirm received: $any”) }
  • 34. Example 1 Source Broadcast Flow Flow Merge GraphStages Sink Module Tree GraphStageModule GraphStageModule stage=SingleSource stage=StatefulMapConcat ActorRefSink stage=Broadcast stage=Map stage=Merge GraphStageModule GraphStageModule GraphStageModule
  • 35. Example 1 implicit val materializer = ActorMaterializer() val sinkActor = system.actorOf(Props(new SinkActor()) val source = Source((1 to 5)) val sink = Sink.actorRef(sinkActor, "COMPLETE") val flowA: Flow[Int, Int, NotUsed] = Flow[Int].map { x => println(s"processing broadcasted element : $x in flowA"); x } val flowB: Flow[Int, Int, NotUsed] = Flow[Int].map { x => println(s"processing broadcasted element : $x in flowB"); x } val graph = RunnableGraph.fromGraph(GraphDSL.create() { implicit b => val broadcast = b.add(Broadcast[Int](2)) val merge = b.add(Broadcast[Int](2)) source ~> broadcast broadcast ~> flowA ~> merge broadcast ~> flowB ~> merge merge ~> sink ClosedShape }) graph.run() source broadcast flowA flowB merge GraphStages sink
  • 36. Example 1 processing broadcasted element : 1 in flowA processing broadcasted element : 1 in flowB processing broadcasted element : 2 in flowA Confirm received: 1 Confirm received: 1 processing broadcasted element : 2 in flowB Confirm received: 2 Confirm received: 2 processing broadcasted element : 3 in flowA processing broadcasted element : 3 in flowB processing broadcasted element : 4 in flowA processing broadcasted element : 4 in flowB Confirm received: 3 Confirm received: 3 processing broadcasted element : 5 in flowA processing broadcasted element : 5 in flowB Confirm received: 4 Confirm received: 4 Confirm received: 5 Confirm received: 5 Confirm received: COMPLETE source broadcast flowA flowB merge GraphStages sink ActorMaterializer Output
  • 37. Example 1 implicit val materializer = GearpumpMaterializer() val sinkActor = system.actorOf(Props(new SinkActor()) val source = Source((1 to 5)) val sink = Sink.actorRef(sinkActor, "COMPLETE") val flowA: Flow[Int, Int, NotUsed] = Flow[Int].map { x => println(s"processing broadcasted element : $x in flowA"); x } val flowB: Flow[Int, Int, NotUsed] = Flow[Int].map { x => println(s"processing broadcasted element : $x in flowB"); x } val graph = RunnableGraph.fromGraph(GraphDSL.create() { implicit b => val broadcast = b.add(Broadcast[Int](2)) val merge = b.add(Broadcast[Int](2)) source ~> broadcast broadcast ~> flowA ~> merge broadcast ~> flowB ~> merge merge ~> sink ClosedShape }) graph.run() source broadcast flowA flowB merge GraphStages sink
  • 38. Example 1 processing broadcasted element : 1 in flowA processing broadcasted element : 1 in flowB processing broadcasted element : 2 in flowB processing broadcasted element : 2 in flowA processing broadcasted element : 3 in flowB processing broadcasted element : 3 in flowA processing broadcasted element : 4 in flowB processing broadcasted element : 4 in flowA processing broadcasted element : 5 in flowB Confirm received: 1 processing broadcasted element : 5 in flowA Confirm received: 1 Confirm received: 2 Confirm received: 2 Confirm received: 3 Confirm received: 3 Confirm received: 4 Confirm received: 4 Confirm received: 5 Confirm received: 5 source broadcast flowA flowB merge GraphStages sink GearpumpMaterializer Output
  • 41. ActorMaterializer 2. Builds a runtime graph of BoundaryPublisher and BoundarySubscribers (Reactive API). 3. Each Publisher or Subscriber contains an instance of GraphStageLogic specific to that GraphStage. 4. Each Publisher or Subscriber also contains an instance of ActorGraphInterpreter - an Actor that manages the message flow using GraphStageLogic.
  • 43. GearpumpMaterializer GraphStageModule( stage=SingleSource) ActorRefSink GraphStageModule( stage=Broadcast) GraphStageModule( stage=Map) GraphStageModule( stage=Merge) 2. Choice of ‘local’ and ‘remote’ is determined by a ‘Strategy’. The default Strategy is to put Source and Sink types in local GraphStageModule( stage=StatefulMapConcat)
  • 44. GearpumpMaterializer ActorRefSink 3. Inserts BridgeModules into both Graphs SourceBridgeModule SinkBridgeModule SinkBridgeModule GraphStageModule( stage=Broadcast) GraphStageModule( stage=Map) GraphStageModule( stage=Merge)GraphStageModule( stage=StatefulMapConcat) GraphStageModule( stage=SingleSource) SourceBridgeModule
  • 45. GearpumpMaterializer ActorRefSink 4. Local graph is passed to a LocalGraphMaterializer SinkBridgeModule GraphStageModule( stage=SingleSource) SourceBridgeModule LocalGraphMaterializer is a variant (subtype) of ActorMaterializer
  • 46. GearpumpMaterializer 5. Converts the remote graph’s Modules into Tasks SourceBridgeTask SinkBridgeTaskBroadcastTask TransformTask MergeTaskStatefulMapConcatTask
  • 47. GearpumpMaterializer 6. Sends this Graph to the Gearpump master SourceBridgeTask SinkBridgeTaskBroadcastTask TransformTask MergeTaskStatefulMapConcatTask
  • 48. GearpumpMaterializer 7. Materialization is controlled at BridgeTasks SourceBridgeTask SinkBridgeTaskBroadcastTask TransformTask MergeTaskStatefulMapConcatTask
  • 49. Example 2 No local graph. More typical of distributed apps. implicit val materializer = GearpumpMaterializer() val sink = GearSink.to(new LoggerSink[String])) val sourceData = new CollectionDataSource( List("red hat", "yellow sweater", "blue jack", "red apple", "green plant", "blue sky")) val source = GearSource.from[String](sourceData) source.filter(_.startsWith("red")).map("I want to order item: " + _).runWith(sink)
  • 50. Example 3 More complex Graph with loops implicit val materializer = GearpumpMaterializer() RunnableGraph.fromGraph(GraphDSL.create() { implicitbuilder => val A = builder.add(Source.single(0)).out val B = builder.add(Broadcast[Int](2)) val C = builder.add(Merge[Int](2)) val D = builder.add(Flow[Int].map(_ + 1)) val E = builder.add(Balance[Int](2)) val F = builder.add(Merge[Int](2)) val G = builder.add(Sink.foreach(println)).in C <~ F A ~> B ~> C ~> F B ~> D ~> E ~> F E ~> G ClosedShape }).run()
  • 51. Summary ● Akka-streams provides a compelling programming model that enables declarative pipeline reuse and extensibility. ● Akka-streams allows different materializers to control and materialize different parts of the module tree. ● It’s possible to provide a seamless (or nearly seamless) conversion of akka-streams to run in a distributed setting by merely replacing ActorMaterializer with GearpumpMaterializer. ● Alternative distributed materializers can be implemented using a similar approach. ● Distributed akka-streams via Apache Gearpump will be available in the next release of Apache Gearpump (0.8.2) or will be made available within an akka specific repo.