SlideShare a Scribd company logo
1 of 89
Download to read offline
MANCHESTER LONDON NEW YORK
Petr Zapletal @petr_zapletal
@spark_summit
@cakesolutions
Distributed Real-Time Stream Processing:
Why and How
Agenda
● Motivation
● Stream Processing
● Available Frameworks
● Systems Comparison
● Recommendations
The Data Deluge
● New Sources and New Use Cases
● 8 Zettabytes (1 ZB = 1 trillion GB) created in 2015
● Stream Processing to the Rescue
● Continuous processing, aggregation and analysis of unbounded data
Distributed Stream Processing
Points of Interest
➜ Runtime and Programming Model
Points of Interest
➜ Runtime and Programming Model
➜ Primitives
Points of Interest
➜ Runtime and Programming Model
➜ Primitives
➜ State Management
Points of Interest
➜ Runtime and Programming Model
➜ Primitives
➜ State Management
➜ Message Delivery Guarantees
Points of Interest
➜ Runtime and Programming Model
➜ Primitives
➜ State Management
➜ Message Delivery Guarantees
➜ Fault Tolerance & Low Overhead Recovery
Points of Interest
➜ Runtime and Programming Model
➜ Primitives
➜ State Management
➜ Message Delivery Guarantees
➜ Fault Tolerance & Low Overhead Recovery
➜ Latency, Throughput & Scalability
Points of Interest
➜ Runtime and Programming Model
➜ Primitives
➜ State Management
➜ Message Delivery Guarantees
➜ Fault Tolerance & Low Overhead Recovery
➜ Latency, Throughput & Scalability
➜ Maturity and Adoption Level
Points of Interest
➜ Runtime and Programming Model
➜ Primitives
➜ State Management
➜ Message Delivery Guarantees
➜ Fault Tolerance & Low Overhead Recovery
➜ Latency, Throughput & Scalability
➜ Maturity and Adoption Level
➜ Ease of Development and Operability
● Most important trait of stream processing system
● Defines expressiveness, possible operations and its limitations
● Therefore defines systems capabilities and its use cases
Runtime and Programming Model
Native Streaming
records
Sink
Operator
Source
Operator
Processing
Operator
records processed one at a time
Processing
Operator
records processed in short batches
Processing
Operator
Receiver
records
Processing
Operator
Micro-batches
Sink
Operator
Micro-batching
Native Streaming
● Records are processed as they arrive
Pros
⟹ Expressiveness
⟹ Low-latency
⟹ Stateful operations
Cons
⟹ Fault-tolerance is expensive
⟹ Load-balancing
Micro-batching
Cons
⟹ Lower latency, depends on
batch interval
⟹ Limited expressivity
⟹ Harder stateful operations
● Splits incoming stream into small batches
Pros
⟹ High-throughput
⟹ Easier fault tolerance
⟹ Simpler load-balancing
Programming Model
Compositional
⟹ Provides basic building blocks as
operators or sources
⟹ Custom component definition
⟹ Manual Topology definition &
optimization
⟹ Advanced functionality often
missing
Declarative
⟹ High-level API
⟹ Operators as higher order functions
⟹ Abstract data types
⟹ Advance operations like state
management or windowing supported
out of the box
⟹ Advanced optimizers
Apache Streaming Landscape
TRIDENT
Storm
Stor
● Pioneer in large scale stream processing
● Higher level micro-batching system build atop Storm
Trident
● Unified batch and stream processing over a batch runtime
Spark Streaming
input data
stream Spark
Streaming
Spark
Engine
batches of
input data
batches of
processed data
Samza
● Builds heavily on Kafka’s log based philosophy
Task 1
Task 2
Task 3
Kafka Streams
● Simple streaming library on top of Apache Kafka
Apex
● Processes massive amounts of real-time events natively in Hadoop
Operator
Operator
OperatorOperator Operator Operator
Output
Stream
Stream
Stream
Stream
Stream
Tuple
Flink
Stream Data
Batch Data
Kafka, RabbitMQ, ...
HDFS, JDBC, ...
● Native streaming & High level API
Counting Words
Spark Summit 2017
Apache Apache Spark
Storm Apache Trident
Flink Streaming Samza
Scala 2017 Streaming
(Apache, 3)
(Streaming, 2)
(2017, 2)
(Spark, 2)
(Storm, 1)
(Trident, 1)
(Flink, 1)
(Samza, 1)
(Scala, 1)
(Summit, 1)
Storm
TopologyBuilder builder = new TopologyBuilder();
builder.setSpout("spout", new RandomSentenceSpout(), 5);
builder.setBolt("split", new Split(), 8).shuffleGrouping("spout");
builder.setBolt("count", new WordCount(), 12).fieldsGrouping("split", new Fields("word"));
...
Map<String, Integer> counts = new HashMap<String, Integer>();
public void execute(Tuple tuple, BasicOutputCollector collector) {
String word = tuple.getString(0);
Integer count = counts.containsKey(word) ? counts.get(word) + 1 : 1;
counts.put(word, count);
collector.emit(new Values(word, count));
}
Storm
TopologyBuilder builder = new TopologyBuilder();
builder.setSpout("spout", new RandomSentenceSpout(), 5);
builder.setBolt("split", new Split(), 8).shuffleGrouping("spout");
builder.setBolt("count", new WordCount(), 12).fieldsGrouping("split", new Fields("word"));
...
Map<String, Integer> counts = new HashMap<String, Integer>();
public void execute(Tuple tuple, BasicOutputCollector collector) {
String word = tuple.getString(0);
Integer count = counts.containsKey(word) ? counts.get(word) + 1 : 1;
counts.put(word, count);
collector.emit(new Values(word, count));
}
Storm
TopologyBuilder builder = new TopologyBuilder();
builder.setSpout("spout", new RandomSentenceSpout(), 5);
builder.setBolt("split", new Split(), 8).shuffleGrouping("spout");
builder.setBolt("count", new WordCount(), 12).fieldsGrouping("split", new Fields("word"));
...
Map<String, Integer> counts = new HashMap<String, Integer>();
public void execute(Tuple tuple, BasicOutputCollector collector) {
String word = tuple.getString(0);
Integer count = counts.containsKey(word) ? counts.get(word) + 1 : 1;
counts.put(word, count);
collector.emit(new Values(word, count));
}
Trident
public static StormTopology buildTopology(LocalDRPC drpc) {
FixedBatchSpout spout = ...
TridentTopology topology = new TridentTopology();
TridentState wordCounts = topology.newStream("spout1", spout)
.each(new Fields("sentence"),new Split(), new Fields("word"))
.groupBy(new Fields("word"))
.persistentAggregate(new MemoryMapState.Factory(),
new Count(), new Fields("count"));
...
}
Trident
public static StormTopology buildTopology(LocalDRPC drpc) {
FixedBatchSpout spout = ...
TridentTopology topology = new TridentTopology();
TridentState wordCounts = topology.newStream("spout1", spout)
.each(new Fields("sentence"),new Split(), new Fields("word"))
.groupBy(new Fields("word"))
.persistentAggregate(new MemoryMapState.Factory(),
new Count(), new Fields("count"));
...
}
Trident
public static StormTopology buildTopology(LocalDRPC drpc) {
FixedBatchSpout spout = ...
TridentTopology topology = new TridentTopology();
TridentState wordCounts = topology.newStream("spout1", spout)
.each(new Fields("sentence"),new Split(), new Fields("word"))
.groupBy(new Fields("word"))
.persistentAggregate(new MemoryMapState.Factory(),
new Count(), new Fields("count"));
...
}
Spark Streaming
val conf = new SparkConf().setAppName("wordcount")
val ssc = new StreamingContext(conf, Seconds(1))
val text = ...
val counts = text.flatMap(line => line.split(" "))
.map(word => (word, 1))
.reduceByKey(_ + _)
counts.print()
ssc.start()
ssc.awaitTermination()
Spark Streaming
val conf = new SparkConf().setAppName("wordcount")
val ssc = new StreamingContext(conf, Seconds(1))
val text = ...
val counts = text.flatMap(line => line.split(" "))
.map(word => (word, 1))
.reduceByKey(_ + _)
counts.print()
ssc.start()
ssc.awaitTermination()
val conf = new SparkConf().setAppName("wordcount")
val ssc = new StreamingContext(conf, Seconds(1))
val text = ...
val counts = text.flatMap(line => line.split(" "))
.map(word => (word, 1))
.reduceByKey(_ + _)
counts.print()
ssc.start()
ssc.awaitTermination()
Spark Streaming
val conf = new SparkConf().setAppName("wordcount")
val ssc = new StreamingContext(conf, Seconds(1))
val text = ...
val counts = text.flatMap(line => line.split(" "))
.map(word => (word, 1))
.reduceByKey(_ + _)
counts.print()
ssc.start()
ssc.awaitTermination()
Spark Streaming
Samza
class WordCountTask extends StreamTask {
override def process(envelope: IncomingMessageEnvelope,
collector: MessageCollector, coordinator: TaskCoordinator) {
val text = envelope.getMessage.asInstanceOf[String]
val counts = text.split(" ")
.foldLeft(Map.empty[String, Int]) {
(count, word) => count + (word -> (count.getOrElse(word, 0) + 1))
}
collector.send(new OutgoingMessageEnvelope(new SystemStream("kafka", "wordcount"),
counts))
}
Samza
class WordCountTask extends StreamTask {
override def process(envelope: IncomingMessageEnvelope,
collector: MessageCollector, coordinator: TaskCoordinator) {
val text = envelope.getMessage.asInstanceOf[String]
val counts = text.split(" ")
.foldLeft(Map.empty[String, Int]) {
(count, word) => count + (word -> (count.getOrElse(word, 0) + 1))
}
collector.send(new OutgoingMessageEnvelope(new SystemStream("kafka", "wordcount"),
counts))
}
class WordCountTask extends StreamTask {
override def process(envelope: IncomingMessageEnvelope,
collector: MessageCollector, coordinator: TaskCoordinator) {
val text = envelope.getMessage.asInstanceOf[String]
val counts = text.split(" ")
.foldLeft(Map.empty[String, Int]) {
(count, word) => count + (word -> (count.getOrElse(word, 0) + 1))
}
collector.send(new OutgoingMessageEnvelope(new SystemStream("kafka", "wordcount"),
counts))
}
Samza
Kafka Streams
final Serde<String> stringSerde = Serdes.String();
final Serde<Long> longSerde = Serdes.Long();
KStream<String, String> textLines = builder.stream(stringSerde, stringSerde,
"streams-file-input");
KStream<String, Long> wordCounts = textLines
.flatMapValues(value -> Arrays.asList(value.toLowerCase().split("W+")))
.groupBy((key, value) -> value)
.count("Counts")
.toStream();
wordCounts.to(stringSerde, longSerde, "streams-wordcount-output");
Kafka Streams
final Serde<String> stringSerde = Serdes.String();
final Serde<Long> longSerde = Serdes.Long();
KStream<String, String> textLines = builder.stream(stringSerde, stringSerde,
"streams-file-input");
KStream<String, Long> wordCounts = textLines
.flatMapValues(value -> Arrays.asList(value.toLowerCase().split("W+")))
.groupBy((key, value) -> value)
.count("Counts")
.toStream();
wordCounts.to(stringSerde, longSerde, "streams-wordcount-output");
Apex
val input = dag.addOperator("input", new LineReader)
val parser = dag.addOperator("parser", new Parser)
val out = dag.addOperator("console", new ConsoleOutputOperator)
dag.addStream[String]("lines", input.out, parser.in)
dag.addStream[String]("words", parser.out, counter.data)
class Parser extends BaseOperator {
@transient
val out = new DefaultOutputPort[String]()
@transient
val in = new DefaultInputPort[String]()
override def process(t: String): Unit = {
for(w <- t.split(" ")) out.emit(w)
}
}
Apex
val input = dag.addOperator("input", new LineReader)
val parser = dag.addOperator("parser", new Parser)
val out = dag.addOperator("console", new ConsoleOutputOperator)
dag.addStream[String]("lines", input.out, parser.in)
dag.addStream[String]("words", parser.out, counter.data)
class Parser extends BaseOperator {
@transient
val out = new DefaultOutputPort[String]()
@transient
val in = new DefaultInputPort[String]()
override def process(t: String): Unit = {
for(w <- t.split(" ")) out.emit(w)
}
}
val input = dag.addOperator("input", new LineReader)
val parser = dag.addOperator("parser", new Parser)
val out = dag.addOperator("console", new ConsoleOutputOperator)
dag.addStream[String]("lines", input.out, parser.in)
dag.addStream[String]("words", parser.out, counter.data)
class Parser extends BaseOperator {
@transient
val out = new DefaultOutputPort[String]()
@transient
val in = new DefaultInputPort[String]()
override def process(t: String): Unit = {
for(w <- t.split(" ")) out.emit(w)
}
}
Apex
Flink
val env = ExecutionEnvironment.getExecutionEnvironment
val text = env.fromElements(...)
val counts = text.flatMap ( _.split(" ") )
.map ( (_, 1) )
.groupBy(0)
.sum(1)
counts.print()
env.execute("wordcount")
Flink
val env = ExecutionEnvironment.getExecutionEnvironment
val text = env.fromElements(...)
val counts = text.flatMap ( _.split(" ") )
.map ( (_, 1) )
.groupBy(0)
.sum(1)
counts.print()
env.execute("wordcount")
Summary
Native Micro-batching Native Native
Compositional Declarative Compositional Declarative
At-least-once Exactly-once At-least-once Exactly-once
Record ACKs
RDD based
Checkpointing
Log-based Checkpointing
Not build-in
Dedicated
Operators
Stateful
Operators
Stateful
Operators
Very Low Medium Low Low
Low Medium High High
High High Medium Medium
Micro-batching
Exactly-once*
Dedicated
DStream
Medium
High
Streaming
Model
API
Guarantees
Fault
Tolerance
State
Management
Latency
Throughput
Maturity
TRIDENT
Hybrid
Compositional*
Exactly-once
Checkpointing
Stateful
Operators
Very Low
High
Medium
Native
Declarative*
At-least-once
Low
Log-based
Stateful
Operators
Low
High
General Guidelines
As always, It depends
General Guidelines
➜ Evaluate particular application needs
General Guidelines
➜ Evaluate particular application needs
➜ Programming model
General Guidelines
➜ Evaluate particular application needs
➜ Programming model
➜ Available delivery guarantees
General Guidelines
➜ Evaluate particular application needs
➜ Programming model
➜ Available delivery guarantees
➜ Almost all non-trivial jobs have state
General Guidelines
➜ Evaluate particular application needs
➜ Programming model
➜ Available delivery guarantees
➜ Almost all non-trivial jobs have state
➜ Fast recovery is critical
Recommendations [Storm & Trident]
● Fits for small and fast tasks
● Very low (tens of milliseconds) latency
● State & Fault tolerance degrades performance significantly
● Potential update to Heron
○ Keeps the API, according to Twitter better in every single way
○ Open-sourced recently
Recommendations [Spark Streaming]
● Spark Ecosystem
● Data Exploration
● Latency is not critical
● Micro-batching limitations
Recommendations [Samza]
● Kafka is a cornerstone of your architecture
● Application requires large states
● Don’t need exactly once
Recommendations [Kafka Streams]
● Similar functionality like Samza with nicer APIs
● Operated by Kafka itself
● Great learning curve
● At-least once delivery
● May not support more advanced functionality
Recommendations [Apex]
● Prefer compositional approach
● Hadoop
● Great performance
● Dynamic DAG changes
Recommendations [Flink]
● Conceptually great, fits very most use cases
● Take advantage of batch processing capabilities
● Need a functionality which is hard to implement in micro-batch
● Enough courage to use emerging project
Dataflow and Apache Beam
Dataflow
Model & SDKs
Apache Flink
Apache Spark
Direct Pipeline
Google Cloud
Dataflow
Stream Processing
Batch Processing
Multiple Modes One Pipeline Many Runtimes
Local
or
cloud
Local
Cloud
Questions
MANCHESTER LONDON NEW YORK
MANCHESTER LONDON NEW YORK
@petr_zapletal @cakesolutions
347 708 1518
enquiries@cakesolutions.net
We are hiring
http://www.cakesolutions.net/careers
References
● http://storm.apache.org/
● http://spark.apache.org/streaming/
● http://samza.apache.org/
● https://apex.apache.org/
● https://flink.apache.org/
● http://beam.incubator.apache.org/
● http://www.cakesolutions.net/teamblogs/comparison-of-apache-stream-processing-frameworks-part-1
● http://data-artisans.com/blog/
● http://www.confluent.io/blog/introducing-kafka-streams-stream-processing-made-simple
● https://yahooeng.tumblr.com/post/135321837876/benchmarking-streaming-computation-engines-at
● http://www.slideshare.net/GyulaFra/largescale-stream-processing-in-the-hadoop-ecosystem
● https://www.oreilly.com/ideas/the-world-beyond-batch-streaming-101
● http://stackoverflow.com/questions/29111549/where-do-apache-samza-and-apache-storm-differ-in-their-use-cases
● https://www.dropbox.com/s/1s8pnjwgkkvik4v/GearPump%20Edge%20Topologies%20and%20Deployment.pptx?dl=0
● ...
List of Figures
● https://databricks.com/blog/2015/07/30/diving-into-apache-spark-streamings-execution-model.html
● http://www.slideshare.net/ptgoetz/storm-hadoop-summit2014
● https://databricks.com/wp-content/uploads/2015/07/image41-1024x602.png
● http://data-artisans.com/wp-content/uploads/2015/08/microbatching.png
● http://samza.apache.org/img/0.9/learn/documentation/container/stateful_job
● http://data-artisans.com/wp-content/uploads/2015/08/streambarrier.png
● https://cwiki.apache.org/confluence/display/FLINK/Stateful+Stream+Processing
● https://raw.githubusercontent.com/tweise/apex-samples/kafka-count-jdbc/exactly-once/docs/images/image00.png
● https://4.bp.blogspot.com/-RlLeDymI_mU/Vp-1cb3AxNI/AAAAAAAACSQ/5TphliHJA4w/s1600/dataflow%2BASF.pn
g
Backup Slides
MANCHESTER LONDON NEW YORK
Processing Architecture Evolution
Batch Pipeline
Serving DBHDFS
Query
Processing Architecture Evolution
Lambda Architecture
Batch Layer Serving
Layer
Stream layer
Query
Allyourdata
Oozie
Query
Processing Architecture Evolution
Standalone Stream Processing
Stream
Processing
Processing Architecture Evolution
Kappa Architecture
Query
Stream
Processing
ETL Operations
● Transformations, joining or filtering of incoming
data
Streaming Applications
Windowing
● Trends in bounded interval, like tweets or sales
Streaming Applications
Streaming Applications
Machine Learning
● Clustering, Trend fitting,
Classification
Streaming Applications
Pattern Recognition
● Fraud detection, Signal
triggering, Anomaly detection
Fault tolerance in streaming systems is
inherently harder that in batch
Fault Tolerance
Managing State
f: (input, state) => (output, state’)
Performance
Hard to design not biased test, lots
of variables
Performance
➜ Latency vs. Throughput
Performance
➜ Latency vs. Throughput
➜ Costs of Delivery Guarantees, Fault-tolerance & State Management
Performance
➜ Latency vs. Throughput
➜ Costs of Delivery Guarantees, Fault-tolerance & State Management
➜ Tuning
Performance
➜ Latency vs. Throughput
➜ Costs of Delivery Guarantees, Fault-tolerance & State Management
➜ Tuning
➜ Network operations, Data locality & Serialization
Project Maturity
When picking up the framework, you
should always consider its maturity
For a long time de-facto industrial
standard
Project Maturity [Storm & Trident]
Project Maturity [Spark Streaming]
The most trending Scala repository
these days and one of the engines
behind Scala’s popularity
Project Maturity [Samza]
Used by LinkedIn and also by tens of
other companies
Project Maturity [Kafka Streams]
???
Project Maturity [Apex]
Graduated very recently, adopted by
a couple of corporate clients already
Project Maturity [Flink]
Still an emerging project, but we can
see its first production deployments

More Related Content

What's hot

Apache Beam: A unified model for batch and stream processing data
Apache Beam: A unified model for batch and stream processing dataApache Beam: A unified model for batch and stream processing data
Apache Beam: A unified model for batch and stream processing dataDataWorks Summit/Hadoop Summit
 
Bellevue Big Data meetup: Dive Deep into Spark Streaming
Bellevue Big Data meetup: Dive Deep into Spark StreamingBellevue Big Data meetup: Dive Deep into Spark Streaming
Bellevue Big Data meetup: Dive Deep into Spark StreamingSantosh Sahoo
 
Reactive programming on Android
Reactive programming on AndroidReactive programming on Android
Reactive programming on AndroidTomáš Kypta
 
Gelly-Stream: Single-Pass Graph Streaming Analytics with Apache Flink
Gelly-Stream: Single-Pass Graph Streaming Analytics with Apache FlinkGelly-Stream: Single-Pass Graph Streaming Analytics with Apache Flink
Gelly-Stream: Single-Pass Graph Streaming Analytics with Apache FlinkVasia Kalavri
 
Deep Dive Into Catalyst: Apache Spark 2.0’s Optimizer
Deep Dive Into Catalyst: Apache Spark 2.0’s OptimizerDeep Dive Into Catalyst: Apache Spark 2.0’s Optimizer
Deep Dive Into Catalyst: Apache Spark 2.0’s OptimizerDatabricks
 
Delta Lake Streaming: Under the Hood
Delta Lake Streaming: Under the HoodDelta Lake Streaming: Under the Hood
Delta Lake Streaming: Under the HoodDatabricks
 
Spark Streaming: Pushing the throughput limits by Francois Garillot and Gerar...
Spark Streaming: Pushing the throughput limits by Francois Garillot and Gerar...Spark Streaming: Pushing the throughput limits by Francois Garillot and Gerar...
Spark Streaming: Pushing the throughput limits by Francois Garillot and Gerar...Spark Summit
 
Deep dive into stateful stream processing in structured streaming by Tathaga...
Deep dive into stateful stream processing in structured streaming  by Tathaga...Deep dive into stateful stream processing in structured streaming  by Tathaga...
Deep dive into stateful stream processing in structured streaming by Tathaga...Databricks
 
Advanced Streaming Analytics with Apache Flink and Apache Kafka, Stephan Ewen
Advanced Streaming Analytics with Apache Flink and Apache Kafka, Stephan EwenAdvanced Streaming Analytics with Apache Flink and Apache Kafka, Stephan Ewen
Advanced Streaming Analytics with Apache Flink and Apache Kafka, Stephan Ewenconfluent
 
Reactive mistakes reactive nyc
Reactive mistakes   reactive nycReactive mistakes   reactive nyc
Reactive mistakes reactive nycPetr Zapletal
 
Time-evolving Graph Processing on Commodity Clusters: Spark Summit East talk ...
Time-evolving Graph Processing on Commodity Clusters: Spark Summit East talk ...Time-evolving Graph Processing on Commodity Clusters: Spark Summit East talk ...
Time-evolving Graph Processing on Commodity Clusters: Spark Summit East talk ...Spark Summit
 
Large-scale graph processing with Apache Flink @GraphDevroom FOSDEM'15
Large-scale graph processing with Apache Flink @GraphDevroom FOSDEM'15Large-scale graph processing with Apache Flink @GraphDevroom FOSDEM'15
Large-scale graph processing with Apache Flink @GraphDevroom FOSDEM'15Vasia Kalavri
 
Multi dimension aggregations using spark and dataframes
Multi dimension aggregations using spark and dataframesMulti dimension aggregations using spark and dataframes
Multi dimension aggregations using spark and dataframesRomi Kuntsman
 
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
 
Streaming SQL (at FlinkForward, Berlin, 2016/09/12)
Streaming SQL (at FlinkForward, Berlin, 2016/09/12)Streaming SQL (at FlinkForward, Berlin, 2016/09/12)
Streaming SQL (at FlinkForward, Berlin, 2016/09/12)Julian Hyde
 
Unified Stream & Batch Processing with Apache Flink (Hadoop Summit Dublin 2016)
Unified Stream & Batch Processing with Apache Flink (Hadoop Summit Dublin 2016)Unified Stream & Batch Processing with Apache Flink (Hadoop Summit Dublin 2016)
Unified Stream & Batch Processing with Apache Flink (Hadoop Summit Dublin 2016)ucelebi
 
Predictive Datacenter Analytics with Strymon
Predictive Datacenter Analytics with StrymonPredictive Datacenter Analytics with Strymon
Predictive Datacenter Analytics with StrymonVasia Kalavri
 
Machine Learning with Apache Flink at Stockholm Machine Learning Group
Machine Learning with Apache Flink at Stockholm Machine Learning GroupMachine Learning with Apache Flink at Stockholm Machine Learning Group
Machine Learning with Apache Flink at Stockholm Machine Learning GroupTill Rohrmann
 
Batch and Stream Graph Processing with Apache Flink
Batch and Stream Graph Processing with Apache FlinkBatch and Stream Graph Processing with Apache Flink
Batch and Stream Graph Processing with Apache FlinkVasia Kalavri
 

What's hot (20)

Apache Beam: A unified model for batch and stream processing data
Apache Beam: A unified model for batch and stream processing dataApache Beam: A unified model for batch and stream processing data
Apache Beam: A unified model for batch and stream processing data
 
Bellevue Big Data meetup: Dive Deep into Spark Streaming
Bellevue Big Data meetup: Dive Deep into Spark StreamingBellevue Big Data meetup: Dive Deep into Spark Streaming
Bellevue Big Data meetup: Dive Deep into Spark Streaming
 
Reactive programming on Android
Reactive programming on AndroidReactive programming on Android
Reactive programming on Android
 
Gelly-Stream: Single-Pass Graph Streaming Analytics with Apache Flink
Gelly-Stream: Single-Pass Graph Streaming Analytics with Apache FlinkGelly-Stream: Single-Pass Graph Streaming Analytics with Apache Flink
Gelly-Stream: Single-Pass Graph Streaming Analytics with Apache Flink
 
Deep Dive Into Catalyst: Apache Spark 2.0’s Optimizer
Deep Dive Into Catalyst: Apache Spark 2.0’s OptimizerDeep Dive Into Catalyst: Apache Spark 2.0’s Optimizer
Deep Dive Into Catalyst: Apache Spark 2.0’s Optimizer
 
Delta Lake Streaming: Under the Hood
Delta Lake Streaming: Under the HoodDelta Lake Streaming: Under the Hood
Delta Lake Streaming: Under the Hood
 
Spark Streaming: Pushing the throughput limits by Francois Garillot and Gerar...
Spark Streaming: Pushing the throughput limits by Francois Garillot and Gerar...Spark Streaming: Pushing the throughput limits by Francois Garillot and Gerar...
Spark Streaming: Pushing the throughput limits by Francois Garillot and Gerar...
 
So you think you can stream.pptx
So you think you can stream.pptxSo you think you can stream.pptx
So you think you can stream.pptx
 
Deep dive into stateful stream processing in structured streaming by Tathaga...
Deep dive into stateful stream processing in structured streaming  by Tathaga...Deep dive into stateful stream processing in structured streaming  by Tathaga...
Deep dive into stateful stream processing in structured streaming by Tathaga...
 
Advanced Streaming Analytics with Apache Flink and Apache Kafka, Stephan Ewen
Advanced Streaming Analytics with Apache Flink and Apache Kafka, Stephan EwenAdvanced Streaming Analytics with Apache Flink and Apache Kafka, Stephan Ewen
Advanced Streaming Analytics with Apache Flink and Apache Kafka, Stephan Ewen
 
Reactive mistakes reactive nyc
Reactive mistakes   reactive nycReactive mistakes   reactive nyc
Reactive mistakes reactive nyc
 
Time-evolving Graph Processing on Commodity Clusters: Spark Summit East talk ...
Time-evolving Graph Processing on Commodity Clusters: Spark Summit East talk ...Time-evolving Graph Processing on Commodity Clusters: Spark Summit East talk ...
Time-evolving Graph Processing on Commodity Clusters: Spark Summit East talk ...
 
Large-scale graph processing with Apache Flink @GraphDevroom FOSDEM'15
Large-scale graph processing with Apache Flink @GraphDevroom FOSDEM'15Large-scale graph processing with Apache Flink @GraphDevroom FOSDEM'15
Large-scale graph processing with Apache Flink @GraphDevroom FOSDEM'15
 
Multi dimension aggregations using spark and dataframes
Multi dimension aggregations using spark and dataframesMulti dimension aggregations using spark and dataframes
Multi dimension aggregations using spark and dataframes
 
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...
 
Streaming SQL (at FlinkForward, Berlin, 2016/09/12)
Streaming SQL (at FlinkForward, Berlin, 2016/09/12)Streaming SQL (at FlinkForward, Berlin, 2016/09/12)
Streaming SQL (at FlinkForward, Berlin, 2016/09/12)
 
Unified Stream & Batch Processing with Apache Flink (Hadoop Summit Dublin 2016)
Unified Stream & Batch Processing with Apache Flink (Hadoop Summit Dublin 2016)Unified Stream & Batch Processing with Apache Flink (Hadoop Summit Dublin 2016)
Unified Stream & Batch Processing with Apache Flink (Hadoop Summit Dublin 2016)
 
Predictive Datacenter Analytics with Strymon
Predictive Datacenter Analytics with StrymonPredictive Datacenter Analytics with Strymon
Predictive Datacenter Analytics with Strymon
 
Machine Learning with Apache Flink at Stockholm Machine Learning Group
Machine Learning with Apache Flink at Stockholm Machine Learning GroupMachine Learning with Apache Flink at Stockholm Machine Learning Group
Machine Learning with Apache Flink at Stockholm Machine Learning Group
 
Batch and Stream Graph Processing with Apache Flink
Batch and Stream Graph Processing with Apache FlinkBatch and Stream Graph Processing with Apache Flink
Batch and Stream Graph Processing with Apache Flink
 

Viewers also liked

Making Structured Streaming Ready for Production
Making Structured Streaming Ready for ProductionMaking Structured Streaming Ready for Production
Making Structured Streaming Ready for ProductionDatabricks
 
Top Mistakes When Writing Reactive Applications - Scala by the Bay 2016
Top Mistakes When Writing Reactive Applications - Scala by the Bay 2016Top Mistakes When Writing Reactive Applications - Scala by the Bay 2016
Top Mistakes When Writing Reactive Applications - Scala by the Bay 2016Petr Zapletal
 
ALLUXIO (formerly Tachyon): Unify Data at Memory Speed - Effective using Spar...
ALLUXIO (formerly Tachyon): Unify Data at Memory Speed - Effective using Spar...ALLUXIO (formerly Tachyon): Unify Data at Memory Speed - Effective using Spar...
ALLUXIO (formerly Tachyon): Unify Data at Memory Speed - Effective using Spar...Alluxio, Inc.
 
Apache kafka-a distributed streaming platform
Apache kafka-a distributed streaming platformApache kafka-a distributed streaming platform
Apache kafka-a distributed streaming platformconfluent
 
Introduction To Streaming Data and Stream Processing with Apache Kafka
Introduction To Streaming Data and Stream Processing with Apache KafkaIntroduction To Streaming Data and Stream Processing with Apache Kafka
Introduction To Streaming Data and Stream Processing with Apache Kafkaconfluent
 
The Power of Both Choices: Practical Load Balancing for Distributed Stream Pr...
The Power of Both Choices: Practical Load Balancing for Distributed Stream Pr...The Power of Both Choices: Practical Load Balancing for Distributed Stream Pr...
The Power of Both Choices: Practical Load Balancing for Distributed Stream Pr...Anis Nasir
 
Hive Poster
Hive PosterHive Poster
Hive Posterragho
 
Building occasionally connected applications using event sourcing
Building occasionally connected applications using event sourcingBuilding occasionally connected applications using event sourcing
Building occasionally connected applications using event sourcingDennis Doomen
 
Streaming Data Flow with Apache Flink @ Paris Flink Meetup 2015
Streaming Data Flow with Apache Flink @ Paris Flink Meetup 2015Streaming Data Flow with Apache Flink @ Paris Flink Meetup 2015
Streaming Data Flow with Apache Flink @ Paris Flink Meetup 2015Till Rohrmann
 
KamonとDatadogによるリアクティブアプリケーションの監視の事例
KamonとDatadogによるリアクティブアプリケーションの監視の事例KamonとDatadogによるリアクティブアプリケーションの監視の事例
KamonとDatadogによるリアクティブアプリケーションの監視の事例Ikuo Matsumura
 
China-Sweden Business Summit 2017 in Stockholm Sweden
China-Sweden Business Summit 2017 in Stockholm SwedenChina-Sweden Business Summit 2017 in Stockholm Sweden
China-Sweden Business Summit 2017 in Stockholm SwedenLydia Liu
 
Diving Through The Layers: Investigating runc, containerd, and the Docker eng...
Diving Through The Layers: Investigating runc, containerd, and the Docker eng...Diving Through The Layers: Investigating runc, containerd, and the Docker eng...
Diving Through The Layers: Investigating runc, containerd, and the Docker eng...Phil Estes
 
Smart Data Conference: DL4J and DataVec
Smart Data Conference: DL4J and DataVecSmart Data Conference: DL4J and DataVec
Smart Data Conference: DL4J and DataVecJosh Patterson
 
Data Streaming (in a Nutshell) ... and Spark's window operations
Data Streaming (in a Nutshell) ... and Spark's window operationsData Streaming (in a Nutshell) ... and Spark's window operations
Data Streaming (in a Nutshell) ... and Spark's window operationsVincenzo Gulisano
 
RISELab:Enabling Intelligent Real-Time Decisions
RISELab:Enabling Intelligent Real-Time DecisionsRISELab:Enabling Intelligent Real-Time Decisions
RISELab:Enabling Intelligent Real-Time DecisionsJen Aman
 
Stream Processing Everywhere - What to use?
Stream Processing Everywhere - What to use?Stream Processing Everywhere - What to use?
Stream Processing Everywhere - What to use?MapR Technologies
 
Digital foundations - Paving the road to cloud solutions
Digital foundations - Paving the road to cloud solutionsDigital foundations - Paving the road to cloud solutions
Digital foundations - Paving the road to cloud solutionsEric D. Schabell
 
User Focused Security at Netflix: Stethoscope
User Focused Security at Netflix: StethoscopeUser Focused Security at Netflix: Stethoscope
User Focused Security at Netflix: StethoscopeJesse Kriss
 
Taking the friction out of microservice frameworks with Lagom
Taking the friction out of microservice frameworks with LagomTaking the friction out of microservice frameworks with Lagom
Taking the friction out of microservice frameworks with LagomMarkus Eisele
 

Viewers also liked (20)

Making Structured Streaming Ready for Production
Making Structured Streaming Ready for ProductionMaking Structured Streaming Ready for Production
Making Structured Streaming Ready for Production
 
Top Mistakes When Writing Reactive Applications - Scala by the Bay 2016
Top Mistakes When Writing Reactive Applications - Scala by the Bay 2016Top Mistakes When Writing Reactive Applications - Scala by the Bay 2016
Top Mistakes When Writing Reactive Applications - Scala by the Bay 2016
 
ALLUXIO (formerly Tachyon): Unify Data at Memory Speed - Effective using Spar...
ALLUXIO (formerly Tachyon): Unify Data at Memory Speed - Effective using Spar...ALLUXIO (formerly Tachyon): Unify Data at Memory Speed - Effective using Spar...
ALLUXIO (formerly Tachyon): Unify Data at Memory Speed - Effective using Spar...
 
Apache kafka-a distributed streaming platform
Apache kafka-a distributed streaming platformApache kafka-a distributed streaming platform
Apache kafka-a distributed streaming platform
 
Introduction To Streaming Data and Stream Processing with Apache Kafka
Introduction To Streaming Data and Stream Processing with Apache KafkaIntroduction To Streaming Data and Stream Processing with Apache Kafka
Introduction To Streaming Data and Stream Processing with Apache Kafka
 
The Power of Both Choices: Practical Load Balancing for Distributed Stream Pr...
The Power of Both Choices: Practical Load Balancing for Distributed Stream Pr...The Power of Both Choices: Practical Load Balancing for Distributed Stream Pr...
The Power of Both Choices: Practical Load Balancing for Distributed Stream Pr...
 
Hive Poster
Hive PosterHive Poster
Hive Poster
 
Building occasionally connected applications using event sourcing
Building occasionally connected applications using event sourcingBuilding occasionally connected applications using event sourcing
Building occasionally connected applications using event sourcing
 
Streaming Data Flow with Apache Flink @ Paris Flink Meetup 2015
Streaming Data Flow with Apache Flink @ Paris Flink Meetup 2015Streaming Data Flow with Apache Flink @ Paris Flink Meetup 2015
Streaming Data Flow with Apache Flink @ Paris Flink Meetup 2015
 
KamonとDatadogによるリアクティブアプリケーションの監視の事例
KamonとDatadogによるリアクティブアプリケーションの監視の事例KamonとDatadogによるリアクティブアプリケーションの監視の事例
KamonとDatadogによるリアクティブアプリケーションの監視の事例
 
MessageBus vs MessageBus
MessageBus vs MessageBusMessageBus vs MessageBus
MessageBus vs MessageBus
 
China-Sweden Business Summit 2017 in Stockholm Sweden
China-Sweden Business Summit 2017 in Stockholm SwedenChina-Sweden Business Summit 2017 in Stockholm Sweden
China-Sweden Business Summit 2017 in Stockholm Sweden
 
Diving Through The Layers: Investigating runc, containerd, and the Docker eng...
Diving Through The Layers: Investigating runc, containerd, and the Docker eng...Diving Through The Layers: Investigating runc, containerd, and the Docker eng...
Diving Through The Layers: Investigating runc, containerd, and the Docker eng...
 
Smart Data Conference: DL4J and DataVec
Smart Data Conference: DL4J and DataVecSmart Data Conference: DL4J and DataVec
Smart Data Conference: DL4J and DataVec
 
Data Streaming (in a Nutshell) ... and Spark's window operations
Data Streaming (in a Nutshell) ... and Spark's window operationsData Streaming (in a Nutshell) ... and Spark's window operations
Data Streaming (in a Nutshell) ... and Spark's window operations
 
RISELab:Enabling Intelligent Real-Time Decisions
RISELab:Enabling Intelligent Real-Time DecisionsRISELab:Enabling Intelligent Real-Time Decisions
RISELab:Enabling Intelligent Real-Time Decisions
 
Stream Processing Everywhere - What to use?
Stream Processing Everywhere - What to use?Stream Processing Everywhere - What to use?
Stream Processing Everywhere - What to use?
 
Digital foundations - Paving the road to cloud solutions
Digital foundations - Paving the road to cloud solutionsDigital foundations - Paving the road to cloud solutions
Digital foundations - Paving the road to cloud solutions
 
User Focused Security at Netflix: Stethoscope
User Focused Security at Netflix: StethoscopeUser Focused Security at Netflix: Stethoscope
User Focused Security at Netflix: Stethoscope
 
Taking the friction out of microservice frameworks with Lagom
Taking the friction out of microservice frameworks with LagomTaking the friction out of microservice frameworks with Lagom
Taking the friction out of microservice frameworks with Lagom
 

Similar to STREAM PROCESSING FRAMEWORKS COMPARED

Distributed real time stream processing- why and how
Distributed real time stream processing- why and howDistributed real time stream processing- why and how
Distributed real time stream processing- why and howPetr Zapletal
 
Spark what's new what's coming
Spark what's new what's comingSpark what's new what's coming
Spark what's new what's comingDatabricks
 
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
 
Spark SQL Deep Dive @ Melbourne Spark Meetup
Spark SQL Deep Dive @ Melbourne Spark MeetupSpark SQL Deep Dive @ Melbourne Spark Meetup
Spark SQL Deep Dive @ Melbourne Spark MeetupDatabricks
 
Flink Forward SF 2017: Timo Walther - Table & SQL API – unified APIs for bat...
Flink Forward SF 2017: Timo Walther -  Table & SQL API – unified APIs for bat...Flink Forward SF 2017: Timo Walther -  Table & SQL API – unified APIs for bat...
Flink Forward SF 2017: Timo Walther - Table & SQL API – unified APIs for bat...Flink Forward
 
Introduction to Spark with Scala
Introduction to Spark with ScalaIntroduction to Spark with Scala
Introduction to Spark with ScalaHimanshu Gupta
 
Structuring Spark: DataFrames, Datasets, and Streaming by Michael Armbrust
Structuring Spark: DataFrames, Datasets, and Streaming by Michael ArmbrustStructuring Spark: DataFrames, Datasets, and Streaming by Michael Armbrust
Structuring Spark: DataFrames, Datasets, and Streaming by Michael ArmbrustSpark Summit
 
Apache Flink Stream Processing
Apache Flink Stream ProcessingApache Flink Stream Processing
Apache Flink Stream ProcessingSuneel Marthi
 
Introduction to Structured Streaming
Introduction to Structured StreamingIntroduction to Structured Streaming
Introduction to Structured StreamingKnoldus Inc.
 
Large-Scale Stream Processing in the Hadoop Ecosystem
Large-Scale Stream Processing in the Hadoop EcosystemLarge-Scale Stream Processing in the Hadoop Ecosystem
Large-Scale Stream Processing in the Hadoop EcosystemGyula Fóra
 
AI與大數據數據處理 Spark實戰(20171216)
AI與大數據數據處理 Spark實戰(20171216)AI與大數據數據處理 Spark實戰(20171216)
AI與大數據數據處理 Spark實戰(20171216)Paul Chao
 
Continuous Application with Structured Streaming 2.0
Continuous Application with Structured Streaming 2.0Continuous Application with Structured Streaming 2.0
Continuous Application with Structured Streaming 2.0Anyscale
 
Spark: Taming Big Data
Spark: Taming Big DataSpark: Taming Big Data
Spark: Taming Big DataLeonardo Gamas
 
SnappyData at Spark Summit 2017
SnappyData at Spark Summit 2017SnappyData at Spark Summit 2017
SnappyData at Spark Summit 2017Jags Ramnarayan
 
SnappyData, the Spark Database. A unified cluster for streaming, transactions...
SnappyData, the Spark Database. A unified cluster for streaming, transactions...SnappyData, the Spark Database. A unified cluster for streaming, transactions...
SnappyData, the Spark Database. A unified cluster for streaming, transactions...SnappyData
 
Structuring Apache Spark 2.0: SQL, DataFrames, Datasets And Streaming - by Mi...
Structuring Apache Spark 2.0: SQL, DataFrames, Datasets And Streaming - by Mi...Structuring Apache Spark 2.0: SQL, DataFrames, Datasets And Streaming - by Mi...
Structuring Apache Spark 2.0: SQL, DataFrames, Datasets And Streaming - by Mi...Databricks
 
Fast federated SQL with Apache Calcite
Fast federated SQL with Apache CalciteFast federated SQL with Apache Calcite
Fast federated SQL with Apache CalciteChris Baynes
 

Similar to STREAM PROCESSING FRAMEWORKS COMPARED (20)

Distributed real time stream processing- why and how
Distributed real time stream processing- why and howDistributed real time stream processing- why and how
Distributed real time stream processing- why and how
 
Spark what's new what's coming
Spark what's new what's comingSpark what's new what's coming
Spark what's new what's coming
 
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
 
Spark SQL Deep Dive @ Melbourne Spark Meetup
Spark SQL Deep Dive @ Melbourne Spark MeetupSpark SQL Deep Dive @ Melbourne Spark Meetup
Spark SQL Deep Dive @ Melbourne Spark Meetup
 
Flink Forward SF 2017: Timo Walther - Table & SQL API – unified APIs for bat...
Flink Forward SF 2017: Timo Walther -  Table & SQL API – unified APIs for bat...Flink Forward SF 2017: Timo Walther -  Table & SQL API – unified APIs for bat...
Flink Forward SF 2017: Timo Walther - Table & SQL API – unified APIs for bat...
 
Introduction to Spark with Scala
Introduction to Spark with ScalaIntroduction to Spark with Scala
Introduction to Spark with Scala
 
Structuring Spark: DataFrames, Datasets, and Streaming by Michael Armbrust
Structuring Spark: DataFrames, Datasets, and Streaming by Michael ArmbrustStructuring Spark: DataFrames, Datasets, and Streaming by Michael Armbrust
Structuring Spark: DataFrames, Datasets, and Streaming by Michael Armbrust
 
Apache Flink Stream Processing
Apache Flink Stream ProcessingApache Flink Stream Processing
Apache Flink Stream Processing
 
cb streams - gavin pickin
cb streams - gavin pickincb streams - gavin pickin
cb streams - gavin pickin
 
Introduction to Structured Streaming
Introduction to Structured StreamingIntroduction to Structured Streaming
Introduction to Structured Streaming
 
Yahoo compares Storm and Spark
Yahoo compares Storm and SparkYahoo compares Storm and Spark
Yahoo compares Storm and Spark
 
Large-Scale Stream Processing in the Hadoop Ecosystem
Large-Scale Stream Processing in the Hadoop EcosystemLarge-Scale Stream Processing in the Hadoop Ecosystem
Large-Scale Stream Processing in the Hadoop Ecosystem
 
AI與大數據數據處理 Spark實戰(20171216)
AI與大數據數據處理 Spark實戰(20171216)AI與大數據數據處理 Spark實戰(20171216)
AI與大數據數據處理 Spark實戰(20171216)
 
Continuous Application with Structured Streaming 2.0
Continuous Application with Structured Streaming 2.0Continuous Application with Structured Streaming 2.0
Continuous Application with Structured Streaming 2.0
 
Flink internals web
Flink internals web Flink internals web
Flink internals web
 
Spark: Taming Big Data
Spark: Taming Big DataSpark: Taming Big Data
Spark: Taming Big Data
 
SnappyData at Spark Summit 2017
SnappyData at Spark Summit 2017SnappyData at Spark Summit 2017
SnappyData at Spark Summit 2017
 
SnappyData, the Spark Database. A unified cluster for streaming, transactions...
SnappyData, the Spark Database. A unified cluster for streaming, transactions...SnappyData, the Spark Database. A unified cluster for streaming, transactions...
SnappyData, the Spark Database. A unified cluster for streaming, transactions...
 
Structuring Apache Spark 2.0: SQL, DataFrames, Datasets And Streaming - by Mi...
Structuring Apache Spark 2.0: SQL, DataFrames, Datasets And Streaming - by Mi...Structuring Apache Spark 2.0: SQL, DataFrames, Datasets And Streaming - by Mi...
Structuring Apache Spark 2.0: SQL, DataFrames, Datasets And Streaming - by Mi...
 
Fast federated SQL with Apache Calcite
Fast federated SQL with Apache CalciteFast federated SQL with Apache Calcite
Fast federated SQL with Apache Calcite
 

More from Petr Zapletal

Change Data Capture - Scale by the Bay 2019
Change Data Capture - Scale by the Bay 2019Change Data Capture - Scale by the Bay 2019
Change Data Capture - Scale by the Bay 2019Petr Zapletal
 
Adopting GraalVM - NE Scala 2019
Adopting GraalVM - NE Scala 2019Adopting GraalVM - NE Scala 2019
Adopting GraalVM - NE Scala 2019Petr Zapletal
 
Adopting GraalVM - Scala eXchange London 2018
Adopting GraalVM - Scala eXchange London 2018Adopting GraalVM - Scala eXchange London 2018
Adopting GraalVM - Scala eXchange London 2018Petr Zapletal
 
Adopting GraalVM - Scale by the Bay 2018
Adopting GraalVM - Scale by the Bay 2018Adopting GraalVM - Scale by the Bay 2018
Adopting GraalVM - Scale by the Bay 2018Petr Zapletal
 
Real World Serverless
Real World ServerlessReal World Serverless
Real World ServerlessPetr Zapletal
 
Reactive mistakes - ScalaDays Chicago 2017
Reactive mistakes -  ScalaDays Chicago 2017Reactive mistakes -  ScalaDays Chicago 2017
Reactive mistakes - ScalaDays Chicago 2017Petr Zapletal
 
Spark Concepts - Spark SQL, Graphx, Streaming
Spark Concepts - Spark SQL, Graphx, StreamingSpark Concepts - Spark SQL, Graphx, Streaming
Spark Concepts - Spark SQL, Graphx, StreamingPetr Zapletal
 
MLlib and Machine Learning on Spark
MLlib and Machine Learning on SparkMLlib and Machine Learning on Spark
MLlib and Machine Learning on SparkPetr Zapletal
 

More from Petr Zapletal (8)

Change Data Capture - Scale by the Bay 2019
Change Data Capture - Scale by the Bay 2019Change Data Capture - Scale by the Bay 2019
Change Data Capture - Scale by the Bay 2019
 
Adopting GraalVM - NE Scala 2019
Adopting GraalVM - NE Scala 2019Adopting GraalVM - NE Scala 2019
Adopting GraalVM - NE Scala 2019
 
Adopting GraalVM - Scala eXchange London 2018
Adopting GraalVM - Scala eXchange London 2018Adopting GraalVM - Scala eXchange London 2018
Adopting GraalVM - Scala eXchange London 2018
 
Adopting GraalVM - Scale by the Bay 2018
Adopting GraalVM - Scale by the Bay 2018Adopting GraalVM - Scale by the Bay 2018
Adopting GraalVM - Scale by the Bay 2018
 
Real World Serverless
Real World ServerlessReal World Serverless
Real World Serverless
 
Reactive mistakes - ScalaDays Chicago 2017
Reactive mistakes -  ScalaDays Chicago 2017Reactive mistakes -  ScalaDays Chicago 2017
Reactive mistakes - ScalaDays Chicago 2017
 
Spark Concepts - Spark SQL, Graphx, Streaming
Spark Concepts - Spark SQL, Graphx, StreamingSpark Concepts - Spark SQL, Graphx, Streaming
Spark Concepts - Spark SQL, Graphx, Streaming
 
MLlib and Machine Learning on Spark
MLlib and Machine Learning on SparkMLlib and Machine Learning on Spark
MLlib and Machine Learning on Spark
 

Recently uploaded

Hand gesture recognition PROJECT PPT.pptx
Hand gesture recognition PROJECT PPT.pptxHand gesture recognition PROJECT PPT.pptx
Hand gesture recognition PROJECT PPT.pptxbodapatigopi8531
 
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
 
Short Story: Unveiling the Reasoning Abilities of Large Language Models by Ke...
Short Story: Unveiling the Reasoning Abilities of Large Language Models by Ke...Short Story: Unveiling the Reasoning Abilities of Large Language Models by Ke...
Short Story: Unveiling the Reasoning Abilities of Large Language Models by Ke...kellynguyen01
 
Professional Resume Template for Software Developers
Professional Resume Template for Software DevelopersProfessional Resume Template for Software Developers
Professional Resume Template for Software DevelopersVinodh Ram
 
EY_Graph Database Powered Sustainability
EY_Graph Database Powered SustainabilityEY_Graph Database Powered Sustainability
EY_Graph Database Powered SustainabilityNeo4j
 
Unit 1.1 Excite Part 1, class 9, cbse...
Unit 1.1 Excite Part 1, class 9, cbse...Unit 1.1 Excite Part 1, class 9, cbse...
Unit 1.1 Excite Part 1, class 9, cbse...aditisharan08
 
Unveiling the Tech Salsa of LAMs with Janus in Real-Time Applications
Unveiling the Tech Salsa of LAMs with Janus in Real-Time ApplicationsUnveiling the Tech Salsa of LAMs with Janus in Real-Time Applications
Unveiling the Tech Salsa of LAMs with Janus in Real-Time ApplicationsAlberto González Trastoy
 
Try MyIntelliAccount Cloud Accounting Software As A Service Solution Risk Fre...
Try MyIntelliAccount Cloud Accounting Software As A Service Solution Risk Fre...Try MyIntelliAccount Cloud Accounting Software As A Service Solution Risk Fre...
Try MyIntelliAccount Cloud Accounting Software As A Service Solution Risk Fre...MyIntelliSource, Inc.
 
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
 
DNT_Corporate presentation know about us
DNT_Corporate presentation know about usDNT_Corporate presentation know about us
DNT_Corporate presentation know about usDynamic Netsoft
 
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
 
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
 
why an Opensea Clone Script might be your perfect match.pdf
why an Opensea Clone Script might be your perfect match.pdfwhy an Opensea Clone Script might be your perfect match.pdf
why an Opensea Clone Script might be your perfect match.pdfjoe51371421
 
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
 
Cloud Management Software Platforms: OpenStack
Cloud Management Software Platforms: OpenStackCloud Management Software Platforms: OpenStack
Cloud Management Software Platforms: OpenStackVICTOR MAESTRE RAMIREZ
 
Adobe Marketo Engage Deep Dives: Using Webhooks to Transfer Data
Adobe Marketo Engage Deep Dives: Using Webhooks to Transfer DataAdobe Marketo Engage Deep Dives: Using Webhooks to Transfer Data
Adobe Marketo Engage Deep Dives: Using Webhooks to Transfer DataBradBedford3
 
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
 
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
 
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
 

Recently uploaded (20)

Hand gesture recognition PROJECT PPT.pptx
Hand gesture recognition PROJECT PPT.pptxHand gesture recognition PROJECT PPT.pptx
Hand gesture recognition PROJECT PPT.pptx
 
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
 
Short Story: Unveiling the Reasoning Abilities of Large Language Models by Ke...
Short Story: Unveiling the Reasoning Abilities of Large Language Models by Ke...Short Story: Unveiling the Reasoning Abilities of Large Language Models by Ke...
Short Story: Unveiling the Reasoning Abilities of Large Language Models by Ke...
 
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...
 
Professional Resume Template for Software Developers
Professional Resume Template for Software DevelopersProfessional Resume Template for Software Developers
Professional Resume Template for Software Developers
 
EY_Graph Database Powered Sustainability
EY_Graph Database Powered SustainabilityEY_Graph Database Powered Sustainability
EY_Graph Database Powered Sustainability
 
Unit 1.1 Excite Part 1, class 9, cbse...
Unit 1.1 Excite Part 1, class 9, cbse...Unit 1.1 Excite Part 1, class 9, cbse...
Unit 1.1 Excite Part 1, class 9, cbse...
 
Unveiling the Tech Salsa of LAMs with Janus in Real-Time Applications
Unveiling the Tech Salsa of LAMs with Janus in Real-Time ApplicationsUnveiling the Tech Salsa of LAMs with Janus in Real-Time Applications
Unveiling the Tech Salsa of LAMs with Janus in Real-Time Applications
 
Try MyIntelliAccount Cloud Accounting Software As A Service Solution Risk Fre...
Try MyIntelliAccount Cloud Accounting Software As A Service Solution Risk Fre...Try MyIntelliAccount Cloud Accounting Software As A Service Solution Risk Fre...
Try MyIntelliAccount Cloud Accounting Software As A Service Solution Risk Fre...
 
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
 
DNT_Corporate presentation know about us
DNT_Corporate presentation know about usDNT_Corporate presentation know about us
DNT_Corporate presentation know about us
 
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
 
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...
 
why an Opensea Clone Script might be your perfect match.pdf
why an Opensea Clone Script might be your perfect match.pdfwhy an Opensea Clone Script might be your perfect match.pdf
why an Opensea Clone Script might be your perfect match.pdf
 
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)
 
Cloud Management Software Platforms: OpenStack
Cloud Management Software Platforms: OpenStackCloud Management Software Platforms: OpenStack
Cloud Management Software Platforms: OpenStack
 
Adobe Marketo Engage Deep Dives: Using Webhooks to Transfer Data
Adobe Marketo Engage Deep Dives: Using Webhooks to Transfer DataAdobe Marketo Engage Deep Dives: Using Webhooks to Transfer Data
Adobe Marketo Engage Deep Dives: Using Webhooks to Transfer Data
 
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
 
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
 
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
 

STREAM PROCESSING FRAMEWORKS COMPARED

  • 3. Agenda ● Motivation ● Stream Processing ● Available Frameworks ● Systems Comparison ● Recommendations
  • 4. The Data Deluge ● New Sources and New Use Cases ● 8 Zettabytes (1 ZB = 1 trillion GB) created in 2015 ● Stream Processing to the Rescue
  • 5. ● Continuous processing, aggregation and analysis of unbounded data Distributed Stream Processing
  • 6. Points of Interest ➜ Runtime and Programming Model
  • 7. Points of Interest ➜ Runtime and Programming Model ➜ Primitives
  • 8. Points of Interest ➜ Runtime and Programming Model ➜ Primitives ➜ State Management
  • 9. Points of Interest ➜ Runtime and Programming Model ➜ Primitives ➜ State Management ➜ Message Delivery Guarantees
  • 10. Points of Interest ➜ Runtime and Programming Model ➜ Primitives ➜ State Management ➜ Message Delivery Guarantees ➜ Fault Tolerance & Low Overhead Recovery
  • 11. Points of Interest ➜ Runtime and Programming Model ➜ Primitives ➜ State Management ➜ Message Delivery Guarantees ➜ Fault Tolerance & Low Overhead Recovery ➜ Latency, Throughput & Scalability
  • 12. Points of Interest ➜ Runtime and Programming Model ➜ Primitives ➜ State Management ➜ Message Delivery Guarantees ➜ Fault Tolerance & Low Overhead Recovery ➜ Latency, Throughput & Scalability ➜ Maturity and Adoption Level
  • 13. Points of Interest ➜ Runtime and Programming Model ➜ Primitives ➜ State Management ➜ Message Delivery Guarantees ➜ Fault Tolerance & Low Overhead Recovery ➜ Latency, Throughput & Scalability ➜ Maturity and Adoption Level ➜ Ease of Development and Operability
  • 14. ● Most important trait of stream processing system ● Defines expressiveness, possible operations and its limitations ● Therefore defines systems capabilities and its use cases Runtime and Programming Model
  • 16. records processed in short batches Processing Operator Receiver records Processing Operator Micro-batches Sink Operator Micro-batching
  • 17. Native Streaming ● Records are processed as they arrive Pros ⟹ Expressiveness ⟹ Low-latency ⟹ Stateful operations Cons ⟹ Fault-tolerance is expensive ⟹ Load-balancing
  • 18. Micro-batching Cons ⟹ Lower latency, depends on batch interval ⟹ Limited expressivity ⟹ Harder stateful operations ● Splits incoming stream into small batches Pros ⟹ High-throughput ⟹ Easier fault tolerance ⟹ Simpler load-balancing
  • 19. Programming Model Compositional ⟹ Provides basic building blocks as operators or sources ⟹ Custom component definition ⟹ Manual Topology definition & optimization ⟹ Advanced functionality often missing Declarative ⟹ High-level API ⟹ Operators as higher order functions ⟹ Abstract data types ⟹ Advance operations like state management or windowing supported out of the box ⟹ Advanced optimizers
  • 21. Storm Stor ● Pioneer in large scale stream processing
  • 22. ● Higher level micro-batching system build atop Storm Trident
  • 23. ● Unified batch and stream processing over a batch runtime Spark Streaming input data stream Spark Streaming Spark Engine batches of input data batches of processed data
  • 24. Samza ● Builds heavily on Kafka’s log based philosophy Task 1 Task 2 Task 3
  • 25. Kafka Streams ● Simple streaming library on top of Apache Kafka
  • 26. Apex ● Processes massive amounts of real-time events natively in Hadoop Operator Operator OperatorOperator Operator Operator Output Stream Stream Stream Stream Stream Tuple
  • 27. Flink Stream Data Batch Data Kafka, RabbitMQ, ... HDFS, JDBC, ... ● Native streaming & High level API
  • 28. Counting Words Spark Summit 2017 Apache Apache Spark Storm Apache Trident Flink Streaming Samza Scala 2017 Streaming (Apache, 3) (Streaming, 2) (2017, 2) (Spark, 2) (Storm, 1) (Trident, 1) (Flink, 1) (Samza, 1) (Scala, 1) (Summit, 1)
  • 29. Storm TopologyBuilder builder = new TopologyBuilder(); builder.setSpout("spout", new RandomSentenceSpout(), 5); builder.setBolt("split", new Split(), 8).shuffleGrouping("spout"); builder.setBolt("count", new WordCount(), 12).fieldsGrouping("split", new Fields("word")); ... Map<String, Integer> counts = new HashMap<String, Integer>(); public void execute(Tuple tuple, BasicOutputCollector collector) { String word = tuple.getString(0); Integer count = counts.containsKey(word) ? counts.get(word) + 1 : 1; counts.put(word, count); collector.emit(new Values(word, count)); }
  • 30. Storm TopologyBuilder builder = new TopologyBuilder(); builder.setSpout("spout", new RandomSentenceSpout(), 5); builder.setBolt("split", new Split(), 8).shuffleGrouping("spout"); builder.setBolt("count", new WordCount(), 12).fieldsGrouping("split", new Fields("word")); ... Map<String, Integer> counts = new HashMap<String, Integer>(); public void execute(Tuple tuple, BasicOutputCollector collector) { String word = tuple.getString(0); Integer count = counts.containsKey(word) ? counts.get(word) + 1 : 1; counts.put(word, count); collector.emit(new Values(word, count)); }
  • 31. Storm TopologyBuilder builder = new TopologyBuilder(); builder.setSpout("spout", new RandomSentenceSpout(), 5); builder.setBolt("split", new Split(), 8).shuffleGrouping("spout"); builder.setBolt("count", new WordCount(), 12).fieldsGrouping("split", new Fields("word")); ... Map<String, Integer> counts = new HashMap<String, Integer>(); public void execute(Tuple tuple, BasicOutputCollector collector) { String word = tuple.getString(0); Integer count = counts.containsKey(word) ? counts.get(word) + 1 : 1; counts.put(word, count); collector.emit(new Values(word, count)); }
  • 32. Trident public static StormTopology buildTopology(LocalDRPC drpc) { FixedBatchSpout spout = ... TridentTopology topology = new TridentTopology(); TridentState wordCounts = topology.newStream("spout1", spout) .each(new Fields("sentence"),new Split(), new Fields("word")) .groupBy(new Fields("word")) .persistentAggregate(new MemoryMapState.Factory(), new Count(), new Fields("count")); ... }
  • 33. Trident public static StormTopology buildTopology(LocalDRPC drpc) { FixedBatchSpout spout = ... TridentTopology topology = new TridentTopology(); TridentState wordCounts = topology.newStream("spout1", spout) .each(new Fields("sentence"),new Split(), new Fields("word")) .groupBy(new Fields("word")) .persistentAggregate(new MemoryMapState.Factory(), new Count(), new Fields("count")); ... }
  • 34. Trident public static StormTopology buildTopology(LocalDRPC drpc) { FixedBatchSpout spout = ... TridentTopology topology = new TridentTopology(); TridentState wordCounts = topology.newStream("spout1", spout) .each(new Fields("sentence"),new Split(), new Fields("word")) .groupBy(new Fields("word")) .persistentAggregate(new MemoryMapState.Factory(), new Count(), new Fields("count")); ... }
  • 35. Spark Streaming val conf = new SparkConf().setAppName("wordcount") val ssc = new StreamingContext(conf, Seconds(1)) val text = ... val counts = text.flatMap(line => line.split(" ")) .map(word => (word, 1)) .reduceByKey(_ + _) counts.print() ssc.start() ssc.awaitTermination()
  • 36. Spark Streaming val conf = new SparkConf().setAppName("wordcount") val ssc = new StreamingContext(conf, Seconds(1)) val text = ... val counts = text.flatMap(line => line.split(" ")) .map(word => (word, 1)) .reduceByKey(_ + _) counts.print() ssc.start() ssc.awaitTermination()
  • 37. val conf = new SparkConf().setAppName("wordcount") val ssc = new StreamingContext(conf, Seconds(1)) val text = ... val counts = text.flatMap(line => line.split(" ")) .map(word => (word, 1)) .reduceByKey(_ + _) counts.print() ssc.start() ssc.awaitTermination() Spark Streaming
  • 38. val conf = new SparkConf().setAppName("wordcount") val ssc = new StreamingContext(conf, Seconds(1)) val text = ... val counts = text.flatMap(line => line.split(" ")) .map(word => (word, 1)) .reduceByKey(_ + _) counts.print() ssc.start() ssc.awaitTermination() Spark Streaming
  • 39. Samza class WordCountTask extends StreamTask { override def process(envelope: IncomingMessageEnvelope, collector: MessageCollector, coordinator: TaskCoordinator) { val text = envelope.getMessage.asInstanceOf[String] val counts = text.split(" ") .foldLeft(Map.empty[String, Int]) { (count, word) => count + (word -> (count.getOrElse(word, 0) + 1)) } collector.send(new OutgoingMessageEnvelope(new SystemStream("kafka", "wordcount"), counts)) }
  • 40. Samza class WordCountTask extends StreamTask { override def process(envelope: IncomingMessageEnvelope, collector: MessageCollector, coordinator: TaskCoordinator) { val text = envelope.getMessage.asInstanceOf[String] val counts = text.split(" ") .foldLeft(Map.empty[String, Int]) { (count, word) => count + (word -> (count.getOrElse(word, 0) + 1)) } collector.send(new OutgoingMessageEnvelope(new SystemStream("kafka", "wordcount"), counts)) }
  • 41. class WordCountTask extends StreamTask { override def process(envelope: IncomingMessageEnvelope, collector: MessageCollector, coordinator: TaskCoordinator) { val text = envelope.getMessage.asInstanceOf[String] val counts = text.split(" ") .foldLeft(Map.empty[String, Int]) { (count, word) => count + (word -> (count.getOrElse(word, 0) + 1)) } collector.send(new OutgoingMessageEnvelope(new SystemStream("kafka", "wordcount"), counts)) } Samza
  • 42. Kafka Streams final Serde<String> stringSerde = Serdes.String(); final Serde<Long> longSerde = Serdes.Long(); KStream<String, String> textLines = builder.stream(stringSerde, stringSerde, "streams-file-input"); KStream<String, Long> wordCounts = textLines .flatMapValues(value -> Arrays.asList(value.toLowerCase().split("W+"))) .groupBy((key, value) -> value) .count("Counts") .toStream(); wordCounts.to(stringSerde, longSerde, "streams-wordcount-output");
  • 43. Kafka Streams final Serde<String> stringSerde = Serdes.String(); final Serde<Long> longSerde = Serdes.Long(); KStream<String, String> textLines = builder.stream(stringSerde, stringSerde, "streams-file-input"); KStream<String, Long> wordCounts = textLines .flatMapValues(value -> Arrays.asList(value.toLowerCase().split("W+"))) .groupBy((key, value) -> value) .count("Counts") .toStream(); wordCounts.to(stringSerde, longSerde, "streams-wordcount-output");
  • 44. Apex val input = dag.addOperator("input", new LineReader) val parser = dag.addOperator("parser", new Parser) val out = dag.addOperator("console", new ConsoleOutputOperator) dag.addStream[String]("lines", input.out, parser.in) dag.addStream[String]("words", parser.out, counter.data) class Parser extends BaseOperator { @transient val out = new DefaultOutputPort[String]() @transient val in = new DefaultInputPort[String]() override def process(t: String): Unit = { for(w <- t.split(" ")) out.emit(w) } }
  • 45. Apex val input = dag.addOperator("input", new LineReader) val parser = dag.addOperator("parser", new Parser) val out = dag.addOperator("console", new ConsoleOutputOperator) dag.addStream[String]("lines", input.out, parser.in) dag.addStream[String]("words", parser.out, counter.data) class Parser extends BaseOperator { @transient val out = new DefaultOutputPort[String]() @transient val in = new DefaultInputPort[String]() override def process(t: String): Unit = { for(w <- t.split(" ")) out.emit(w) } }
  • 46. val input = dag.addOperator("input", new LineReader) val parser = dag.addOperator("parser", new Parser) val out = dag.addOperator("console", new ConsoleOutputOperator) dag.addStream[String]("lines", input.out, parser.in) dag.addStream[String]("words", parser.out, counter.data) class Parser extends BaseOperator { @transient val out = new DefaultOutputPort[String]() @transient val in = new DefaultInputPort[String]() override def process(t: String): Unit = { for(w <- t.split(" ")) out.emit(w) } } Apex
  • 47. Flink val env = ExecutionEnvironment.getExecutionEnvironment val text = env.fromElements(...) val counts = text.flatMap ( _.split(" ") ) .map ( (_, 1) ) .groupBy(0) .sum(1) counts.print() env.execute("wordcount")
  • 48. Flink val env = ExecutionEnvironment.getExecutionEnvironment val text = env.fromElements(...) val counts = text.flatMap ( _.split(" ") ) .map ( (_, 1) ) .groupBy(0) .sum(1) counts.print() env.execute("wordcount")
  • 49. Summary Native Micro-batching Native Native Compositional Declarative Compositional Declarative At-least-once Exactly-once At-least-once Exactly-once Record ACKs RDD based Checkpointing Log-based Checkpointing Not build-in Dedicated Operators Stateful Operators Stateful Operators Very Low Medium Low Low Low Medium High High High High Medium Medium Micro-batching Exactly-once* Dedicated DStream Medium High Streaming Model API Guarantees Fault Tolerance State Management Latency Throughput Maturity TRIDENT Hybrid Compositional* Exactly-once Checkpointing Stateful Operators Very Low High Medium Native Declarative* At-least-once Low Log-based Stateful Operators Low High
  • 51. General Guidelines ➜ Evaluate particular application needs
  • 52. General Guidelines ➜ Evaluate particular application needs ➜ Programming model
  • 53. General Guidelines ➜ Evaluate particular application needs ➜ Programming model ➜ Available delivery guarantees
  • 54. General Guidelines ➜ Evaluate particular application needs ➜ Programming model ➜ Available delivery guarantees ➜ Almost all non-trivial jobs have state
  • 55. General Guidelines ➜ Evaluate particular application needs ➜ Programming model ➜ Available delivery guarantees ➜ Almost all non-trivial jobs have state ➜ Fast recovery is critical
  • 56. Recommendations [Storm & Trident] ● Fits for small and fast tasks ● Very low (tens of milliseconds) latency ● State & Fault tolerance degrades performance significantly ● Potential update to Heron ○ Keeps the API, according to Twitter better in every single way ○ Open-sourced recently
  • 57. Recommendations [Spark Streaming] ● Spark Ecosystem ● Data Exploration ● Latency is not critical ● Micro-batching limitations
  • 58. Recommendations [Samza] ● Kafka is a cornerstone of your architecture ● Application requires large states ● Don’t need exactly once
  • 59. Recommendations [Kafka Streams] ● Similar functionality like Samza with nicer APIs ● Operated by Kafka itself ● Great learning curve ● At-least once delivery ● May not support more advanced functionality
  • 60. Recommendations [Apex] ● Prefer compositional approach ● Hadoop ● Great performance ● Dynamic DAG changes
  • 61. Recommendations [Flink] ● Conceptually great, fits very most use cases ● Take advantage of batch processing capabilities ● Need a functionality which is hard to implement in micro-batch ● Enough courage to use emerging project
  • 62. Dataflow and Apache Beam Dataflow Model & SDKs Apache Flink Apache Spark Direct Pipeline Google Cloud Dataflow Stream Processing Batch Processing Multiple Modes One Pipeline Many Runtimes Local or cloud Local Cloud
  • 64. MANCHESTER LONDON NEW YORK @petr_zapletal @cakesolutions 347 708 1518 enquiries@cakesolutions.net We are hiring http://www.cakesolutions.net/careers
  • 65. References ● http://storm.apache.org/ ● http://spark.apache.org/streaming/ ● http://samza.apache.org/ ● https://apex.apache.org/ ● https://flink.apache.org/ ● http://beam.incubator.apache.org/ ● http://www.cakesolutions.net/teamblogs/comparison-of-apache-stream-processing-frameworks-part-1 ● http://data-artisans.com/blog/ ● http://www.confluent.io/blog/introducing-kafka-streams-stream-processing-made-simple ● https://yahooeng.tumblr.com/post/135321837876/benchmarking-streaming-computation-engines-at ● http://www.slideshare.net/GyulaFra/largescale-stream-processing-in-the-hadoop-ecosystem ● https://www.oreilly.com/ideas/the-world-beyond-batch-streaming-101 ● http://stackoverflow.com/questions/29111549/where-do-apache-samza-and-apache-storm-differ-in-their-use-cases ● https://www.dropbox.com/s/1s8pnjwgkkvik4v/GearPump%20Edge%20Topologies%20and%20Deployment.pptx?dl=0 ● ...
  • 66. List of Figures ● https://databricks.com/blog/2015/07/30/diving-into-apache-spark-streamings-execution-model.html ● http://www.slideshare.net/ptgoetz/storm-hadoop-summit2014 ● https://databricks.com/wp-content/uploads/2015/07/image41-1024x602.png ● http://data-artisans.com/wp-content/uploads/2015/08/microbatching.png ● http://samza.apache.org/img/0.9/learn/documentation/container/stateful_job ● http://data-artisans.com/wp-content/uploads/2015/08/streambarrier.png ● https://cwiki.apache.org/confluence/display/FLINK/Stateful+Stream+Processing ● https://raw.githubusercontent.com/tweise/apex-samples/kafka-count-jdbc/exactly-once/docs/images/image00.png ● https://4.bp.blogspot.com/-RlLeDymI_mU/Vp-1cb3AxNI/AAAAAAAACSQ/5TphliHJA4w/s1600/dataflow%2BASF.pn g
  • 68. Processing Architecture Evolution Batch Pipeline Serving DBHDFS Query
  • 69. Processing Architecture Evolution Lambda Architecture Batch Layer Serving Layer Stream layer Query Allyourdata Oozie Query
  • 70. Processing Architecture Evolution Standalone Stream Processing Stream Processing
  • 71. Processing Architecture Evolution Kappa Architecture Query Stream Processing
  • 72. ETL Operations ● Transformations, joining or filtering of incoming data Streaming Applications
  • 73. Windowing ● Trends in bounded interval, like tweets or sales Streaming Applications
  • 74. Streaming Applications Machine Learning ● Clustering, Trend fitting, Classification
  • 75. Streaming Applications Pattern Recognition ● Fraud detection, Signal triggering, Anomaly detection
  • 76. Fault tolerance in streaming systems is inherently harder that in batch Fault Tolerance
  • 77. Managing State f: (input, state) => (output, state’)
  • 78. Performance Hard to design not biased test, lots of variables
  • 80. Performance ➜ Latency vs. Throughput ➜ Costs of Delivery Guarantees, Fault-tolerance & State Management
  • 81. Performance ➜ Latency vs. Throughput ➜ Costs of Delivery Guarantees, Fault-tolerance & State Management ➜ Tuning
  • 82. Performance ➜ Latency vs. Throughput ➜ Costs of Delivery Guarantees, Fault-tolerance & State Management ➜ Tuning ➜ Network operations, Data locality & Serialization
  • 83. Project Maturity When picking up the framework, you should always consider its maturity
  • 84. For a long time de-facto industrial standard Project Maturity [Storm & Trident]
  • 85. Project Maturity [Spark Streaming] The most trending Scala repository these days and one of the engines behind Scala’s popularity
  • 86. Project Maturity [Samza] Used by LinkedIn and also by tens of other companies
  • 87. Project Maturity [Kafka Streams] ???
  • 88. Project Maturity [Apex] Graduated very recently, adopted by a couple of corporate clients already
  • 89. Project Maturity [Flink] Still an emerging project, but we can see its first production deployments