SlideShare a Scribd company logo
1 of 50
Download to read offline
Large-Scale Stream Processing in
the Hadoop Ecosystem
Gyula Fóra
gyula.fora@king.com
@GyulaFora
Márton Balassi
mbalassi@apache.org
@MartonBalassi
Stream processing by
example
Real-Time Player Statistics
• Compute real-time, queryable
statistics
• Billions of events / day
• Millions of active users / day
• State quickly grows beyond
memory
• Complex event processing logic
• Strong consistency requirements
2016-04-13 Hadoop Summit 2016 (Dublin) 3
DB
DB
Real-time dashboard of telco network
2016-04-13 Hadoop Summit 2016 (Dublin) 4
• Example query: Download speed
heatmap of premium users in the
last 5 minutes
• Dependant on ~1 TB slowly changing
enrichment data
• Multi GB/s input rate
• Some of the use cases require complex
windowing logic
DB
Open source stream
processors
Apache Streaming Landscape
2016-04-13 Hadoop Summit 2016 (Dublin) 6
Apache Storm
• Pioneer of real-time analytics
• Distributed dataflow abstraction with low-level control
• Time windowing and state introduced recently
When to use Storm
• Very low latency requirements
• No need for advanced
state/windowing
• At-least-once is acceptable
2016-04-13 Hadoop Summit 2016 (Dublin) 7
Apache Samza
• Builds heavily on Kafka’s log based philosophy
• Pluggable components, but runs best with Kafka
• Scalable operator state with RocksDB
• Basic time windowing
When to use Samza
• Join streams with large states
• At-least-once is acceptable
2016-04-13 Hadoop Summit 2016 (Dublin) 8
Kafka Streams
• Streaming library on top of Apache Kafka
• Similar features to Samza but nicer API
• Big win for operational simplicity
When to use Kafka Streams
• Kafka based data infrastructure
• Join streams with large states
• At-least-once is acceptable
2016-04-13 Hadoop Summit 2016 (Dublin) 9
Apache Spark
• Unified batch and stream processing over a batch runtime
• Good integration with batch programs
• Lags behind recent streaming advancements but evolving
quickly
• Spark 2.0 comes with new streaming engine
When to use Spark
• Simpler data exploration
• Combine with (Spark) batch analytics
• Medium latency is acceptable
2016-04-13 Hadoop Summit 2016 (Dublin) 10
Apache Flink
• Unified batch and stream processing over dataflow engine
• Leader of open source streaming innovation
• Highly flexible and robust stateful and windowing computations
• Savepoints for state management
When to use Flink
• Advanced streaming analytics
• Complex windowing/state
• Need for high TP - low latency
2016-04-13 Hadoop Summit 2016 (Dublin) 11
Batch data
Kafka, RabbitMQ ...
HDFS, JDBC ...
Stream Data
Apache Apex
• Native streaming engine built natively on YARN
• Stateful operators with checkpointing to HDFS
• Advanced partitioning support with locality optimizations
When to use Apex
• Advanced streaming analytics
• Very low latency requirements
• Need extensive operator library
2016-04-13 Hadoop Summit 2016 (Dublin) 12
System comparison
2016-04-13 Hadoop Summit 2016 (Dublin) 13
Model Native Micro-batch Native Native Native
API Compositional Declarative Compositional Declarative Compositional
Fault
tolerance
Record ACKs RDD-based Log-based Checkpoints Checkpoints
Guarantee At-least-once Exactly-once At-least-once Exactly-once Exactly-once
State
Stateful
operators
State as
DStream
Stateful
operators
Stateful
operators
Stateful
operators
Windowing Time based Time based Time based Flexible Time based
Latency Very-Low High Low Low Very-Low
Throughput Medium Very-High High Very-High Very-High
Under the hood
Native Streaming
2016-04-13 Hadoop Summit 2016 (Dublin) 15
Distributed dataflow runtime
• Long standing operators
• Pipelined execution
• Usually possible to create
cyclic flows
2016-04-13 Hadoop Summit 2016 (Dublin) 16
Pros
• Full expressivity
• Low-latency execution
• Stateful operators
Cons
• Fault-tolerance is hard
• Throughput may suffer
• Load balancing is an issue
Micro-batching
2016-04-13 Hadoop Summit 2016 (Dublin) 17
Micro-batch runtime
• Computation broken down
to time intervals
• Load aware scheduling
• Easy interaction with batch
2016-04-13 Hadoop Summit 2016 (Dublin) 18
Pros
• Easy to reason about
• High-throughput
• FT comes for “free”
• Dynamic load balancing
Cons
• Latency depends on
batch size
• Limited expressivity
• Stateless by nature
Programming models
Hierarchy of Streaming APIs
2016-04-13 Hadoop Summit 2016 (Dublin) 20
DataStream
DStream
 Transformations, abstract operators
 For both engineers and data analysts
 Allows (some) automatic query
optimization
Spout, Consumer,
Bolt, Task,
Topology  Direct access to the execution graph
 Suitable for engineers
 Fine grained access but lower productivity
Declarative
Compositional
Apache Beam
• One API to rule them all: combined batch and streaming
analytics
• Open sourced by Google, based on DataFlow
• Advanced windowing
• Runners on different systems
• Google Cloud
• Flink
• Spark
• (Others to follow…)
• Useful for benchmarking?
2016-04-13 Hadoop Summit 2016 (Dublin) 21
Apache Beam
• What results are calculated?
• Where in event time are results calculated?
• When in processing time are results materialized?
• How do refinements of results relate?
2016-04-13 Hadoop Summit 2016 (Dublin) 22
Counting words…
WordCount
2016-04-13 Hadoop Summit 2016 (Dublin) 24
storm dublin flink
apache storm spark
streaming samza storm
flink apache flink
bigdata storm
flink streaming
(storm, 4)
(dublin, 1)
(flink, 4)
(apache, 2)
(spark, 1)
(streaming, 2)
(samza, 1)
(bigdata, 1)
Storm
2016-04-13 Hadoop Summit 2016 (Dublin) 25
Assembling the topology
TopologyBuilder builder = new TopologyBuilder();
builder.setSpout("spout", new SentenceSpout(), 5);
builder.setBolt("split", new Splitter(), 8).shuffleGrouping("spout");
builder.setBolt("count", new Counter(), 12)
.fieldsGrouping("split", new Fields("word"));
public class Counter extends BaseBasicBolt {
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));
}
}
Rolling word count bolt
Samza
2016-04-13 Hadoop Summit 2016 (Dublin) 26
public class WordCountTask implements StreamTask {
private KeyValueStore<String, Integer> store;
public void process( IncomingMessageEnvelope
envelope,
MessageCollector collector,
TaskCoordinator coordinator) {
String word = envelope.getMessage();
Integer count = store.get(word);
if(count == null){count = 0;}
store.put(word, count + 1);
collector.send(new OutgoingMessageEnvelope(new
SystemStream("kafka", ”wc"), Tuple2.of(word, count)));
}
}
Rolling word count task
Apex
2016-04-13 Hadoop Summit 2016 (Dublin) 27
Flink
2016-04-13 Hadoop Summit 2016 (Dublin) 28
val lines: DataStream[String] = env.socketTextStream(...)
lines.flatMap {line => line.split(" ")
.map(word => Word(word,1))}
.keyBy("word")
.sum("frequency").print()
case class Word (word: String, frequency: Int)
val lines: DataStream[String] = env.socketTextStream(...)
lines.flatMap {line => line.split(" ")
.map(word => Word(word,1))}
.keyBy("word").
.timeWindow(Time.seconds(5))
.sum("frequency").print()
Rolling word count
Window word count
Spark
2016-04-13 Hadoop Summit 2016 (Dublin) 29
Window word count
Rolling word count (new feature )
val lines = env.fromSocketStream(...)
val words = lines.flatMap(line => line.split(" "))
.map(word => (word,1))
val wordCounts = words.reduceByKey(_ + _)
wordCounts.print()
val func = (word: String, one: Option[Int], state: State[Int]) => {
val sum = one.getOrElse(0) + state.getOption.getOrElse(0)
val output = (word, sum)
state.update(sum)
output
}
val stateDstream = wordDstream.mapWithState(
StateSpec.function(func).initialState(initialRDD))
stateDstream.print()
Beam
2016-04-13 Hadoop Summit 2016 (Dublin) 30
PCollection<String> windowedLines = input
.apply(Window.<String>into(
FixedWindows.of(Duration.standardMinutes(5))));
PCollection<KV<String, Long>> wordCounts = windowedLines
.apply(ParDo.of(new DoFn<String, String>() {
@Override
public void processElement(ProcessContext c) {
for (String word : c.element().split("[^a-zA-Z']+")){
if (!word.isEmpty()) {
c.output(word);
}
}
}
}))
.apply(Count.<String>perElement());
Window word count (minimalistic version)
Fault tolerance and stateful
processing
Fault tolerance intro
• Fault-tolerance in streaming systems is inherently harder than in
batch
• Can’t just restart computation
• State is a problem
• Fast recovery is crucial
• Streaming topologies run 24/7 for a long period
• Fault-tolerance is a complex issue
• No single point of failure is allowed
• Guaranteeing input processing
• Consistent operator state
• Fast recovery
• At-least-once vs Exactly-once semantics
2016-04-13 Hadoop Summit 2016 (Dublin) 32
Storm record acknowledgements
• Track the lineage of tuples as they are
processed (anchors and acks)
• Special “acker” bolts track each lineage DAG
(efficient xor based algorithm)
• Replay the root of failed (or timed out) tuples
2016-04-13 Hadoop Summit 2016 (Dublin) 33
Samza offset tracking
• Exploits the properties of a durable, offset based
messaging layer
• Each task maintains its current offset, which moves
forward as it processes elements
• The offset is checkpointed and restored on failure
(some messages might be repeated)
2016-04-13 Hadoop Summit 2016 (Dublin) 34
Spark RDD recomputation
• Immutable data model with repeatable
computation
• Failed RDDs are recomputed using
their lineage
• Checkpoint RDDs to reduce lineage
length
• Parallel recovery of failed RDDs
• Exactly-once semantics
2016-04-13 Hadoop Summit 2016 (Dublin) 35
Flink state checkpointing
• Consistent global snapshots with exactly-once semantics
• Algorithm designed for stateful dataflows (minimal runtime
overhead)
• Pluggable state backends: Memory, FS, RocksDB, MySQL…
2016-04-13 Hadoop Summit 2016 (Dublin) 36
Apex state checkpointing
• Algorithms similar to Flink’s but
also buffers output windows
• Larger memory overhead but
faster, granular recovery
• Pluggable checkpoint backend,
HDFS by default
2016-04-13 Hadoop Summit 2016 (Dublin) 37
Performance
How much does all this matter?
The winners of last year's Twitter hack week managed to reduce the resources needed
for a specific job by 99%. [1]
There are many recent benchmarks out there
• Storm, Flink & Spark by Yahoo [2]
• Apex by DataTorrent [3,4]
• Flink by data Artisans [1,5]
[1] http://data-artisans.com/extending-the-yahoo-streaming-benchmark
[2] https://yahooeng.tumblr.com/post/135321837876/benchmarking-streaming-computation-engines-at
[3] https://www.datatorrent.com/blog/blog-implementing-linear-road-benchmark-in-apex/
[4] https://www.datatorrent.com/blog/blog-apex-performance-benchmark/
[5] http://data-artisans.com/high-throughput-low-latency-and-exactly-once-stream-processing-with-apache-flink/
2016-04-13 Hadoop Summit 2016 (Dublin) 38
Next steps for streaming
• Dynamic scaling (with state)
• Rolling upgrades
• Better state handling
• More Beam runners
• Libraries: CEP, ML
• Better batch integration
2016-04-13 Hadoop Summit 2016 (Dublin) 39
Closing
Summary
• Streaming systems are gaining popularity with many
businesses migrating some of their infrastructure
• The open source space sees a lot of innovation
• When choosing an application consider your specific use cases,
do not just follow the herd
• We have a recommended reading section :)
2016-04-13 Hadoop Summit 2016 (Dublin) 41
Thank you!
Recommended
reading
Apache Beam
• http://beam.incubator.apache.org/beam/capability/2016/03/17/c
apability-matrix.html
• https://www.oreilly.com/ideas/the-world-beyond-batch-
streaming-101
• https://www.oreilly.com/ideas/the-world-beyond-batch-
streaming-102
2016-04-13 Hadoop Summit 2016 (Dublin) 44
Apache Spark Streaming
• https://databricks.com/blog/2016/02/01/faster-stateful-stream-
processing-in-spark-streaming.html
• http://www.slideshare.net/databricks/2016-spark-summit-east-
keynote-matei-zaharia
2016-04-13 Hadoop Summit 2016 (Dublin) 45
Apache Flink
• http://flink.apache.org/news/2015/12/04/Introducing-
windows.html
• http://data-artisans.com/flink-1-0-0/
• http://data-artisans.com/how-apache-flink-enables-new-
streaming-applications/
2016-04-13 Hadoop Summit 2016 (Dublin) 46
Apache Storm
• https://community.hortonworks.com/articles/14171/windowing-
and-state-checkpointing-in-apache-storm.html
• https://yahooeng.tumblr.com/post/135321837876/benchmarking
-streaming-computation-engines-at
2016-04-13 Hadoop Summit 2016 (Dublin) 47
Samza / Kafka Streams
• http://docs.confluent.io/2.1.0-alpha1/streams/architecture.html
• http://www.confluent.io/blog/introducing-kafka-streams-stream-
processing-made-simple
• http://docs.confluent.io/2.1.0-alpha1/streams/index.html
• http://www.slideshare.net/edibice/extremely-low-latency-web-
scale-fraud-prevention-with-apache-samza-kafka-and-friends
• http://radar.oreilly.com/2014/07/why-local-state-is-a-
fundamental-primitive-in-stream-processing.html
2016-04-13 Hadoop Summit 2016 (Dublin) 48
Apache Apex
• http://docs.datatorrent.com/application_development/#apache-
apex-platform-overview
• http://docs.datatorrent.com/application_development/#fault-
tolerance
• https://github.com/apache/incubator-apex-
malhar/tree/master/demos
• https://www.datatorrent.com/introducing-apache-apex-
incubating/
2016-04-13 Hadoop Summit 2016 (Dublin) 49
List of Figures (in order of usage)
• https://upload.wikimedia.org/wikipedia/commons/thumb/2/2a/CPT-FSM-abcd.svg/326px-CPT-FSM-
abcd.svg.png
• https://storm.apache.org/images/topology.png
• https://databricks.com/wp-content/uploads/2015/07/image11-1024x655.png
• https://databricks.com/wp-content/uploads/2015/07/image21-1024x734.png
• https://people.csail.mit.edu/matei/papers/2012/hotcloud_spark_streaming.pdf, page 2.
• http://www.slideshare.net/ptgoetz/storm-hadoop-summit2014, page 69-71.
• http://samza.apache.org/img/0.9/learn/documentation/container/checkpointing.svg
• https://databricks.com/wp-content/uploads/2015/07/image41-1024x602.png
• https://storm.apache.org/documentation/images/spout-vs-state.png
• http://samza.apache.org/img/0.9/learn/documentation/container/stateful_job.png
2016-04-13 Hadoop Summit 2016 (Dublin) 50

More Related Content

What's hot

Stream Processing use cases and applications with Apache Apex by Thomas Weise
Stream Processing use cases and applications with Apache Apex by Thomas WeiseStream Processing use cases and applications with Apache Apex by Thomas Weise
Stream Processing use cases and applications with Apache Apex by Thomas WeiseBig Data Spain
 
Real-time Stream Processing with Apache Flink @ Hadoop Summit
Real-time Stream Processing with Apache Flink @ Hadoop SummitReal-time Stream Processing with Apache Flink @ Hadoop Summit
Real-time Stream Processing with Apache Flink @ Hadoop SummitGyula Fóra
 
Intro to Apache Apex - Next Gen Native Hadoop Platform - Hackac
Intro to Apache Apex - Next Gen Native Hadoop Platform - HackacIntro to Apache Apex - Next Gen Native Hadoop Platform - Hackac
Intro to Apache Apex - Next Gen Native Hadoop Platform - HackacApache Apex
 
Patterns of the Lambda Architecture -- 2015 April -- Hadoop Summit, Europe
Patterns of the Lambda Architecture -- 2015 April -- Hadoop Summit, EuropePatterns of the Lambda Architecture -- 2015 April -- Hadoop Summit, Europe
Patterns of the Lambda Architecture -- 2015 April -- Hadoop Summit, EuropeFlip Kromer
 
Apache Big Data EU 2016: Next Gen Big Data Analytics with Apache Apex
Apache Big Data EU 2016: Next Gen Big Data Analytics with Apache ApexApache Big Data EU 2016: Next Gen Big Data Analytics with Apache Apex
Apache Big Data EU 2016: Next Gen Big Data Analytics with Apache ApexApache Apex
 
Albert Bifet – Apache Samoa: Mining Big Data Streams with Apache Flink
Albert Bifet – Apache Samoa: Mining Big Data Streams with Apache FlinkAlbert Bifet – Apache Samoa: Mining Big Data Streams with Apache Flink
Albert Bifet – Apache Samoa: Mining Big Data Streams with Apache FlinkFlink Forward
 
S. Bartoli & F. Pompermaier – A Semantic Big Data Companion
S. Bartoli & F. Pompermaier – A Semantic Big Data CompanionS. Bartoli & F. Pompermaier – A Semantic Big Data Companion
S. Bartoli & F. Pompermaier – A Semantic Big Data CompanionFlink Forward
 
Mohamed Amine Abdessemed – Real-time Data Integration with Apache Flink & Kafka
Mohamed Amine Abdessemed – Real-time Data Integration with Apache Flink & KafkaMohamed Amine Abdessemed – Real-time Data Integration with Apache Flink & Kafka
Mohamed Amine Abdessemed – Real-time Data Integration with Apache Flink & KafkaFlink Forward
 
Intro to Apache Apex - Next Gen Platform for Ingest and Transform
Intro to Apache Apex - Next Gen Platform for Ingest and TransformIntro to Apache Apex - Next Gen Platform for Ingest and Transform
Intro to Apache Apex - Next Gen Platform for Ingest and TransformApache Apex
 
Kafka to Hadoop Ingest with Parsing, Dedup and other Big Data Transformations
Kafka to Hadoop Ingest with Parsing, Dedup and other Big Data TransformationsKafka to Hadoop Ingest with Parsing, Dedup and other Big Data Transformations
Kafka to Hadoop Ingest with Parsing, Dedup and other Big Data TransformationsApache Apex
 
ApacheCon: Apache Flink - Fast and Reliable Large-Scale Data Processing
ApacheCon: Apache Flink - Fast and Reliable Large-Scale Data ProcessingApacheCon: Apache Flink - Fast and Reliable Large-Scale Data Processing
ApacheCon: Apache Flink - Fast and Reliable Large-Scale Data ProcessingFabian Hueske
 
Data Stream Processing with Apache Flink
Data Stream Processing with Apache FlinkData Stream Processing with Apache Flink
Data Stream Processing with Apache FlinkFabian Hueske
 
Apache Flink Meetup Munich (November 2015): Flink Overview, Architecture, Int...
Apache Flink Meetup Munich (November 2015): Flink Overview, Architecture, Int...Apache Flink Meetup Munich (November 2015): Flink Overview, Architecture, Int...
Apache Flink Meetup Munich (November 2015): Flink Overview, Architecture, Int...Robert Metzger
 
A Data Streaming Architecture with Apache Flink (berlin Buzzwords 2016)
A Data Streaming Architecture with Apache Flink (berlin Buzzwords 2016)A Data Streaming Architecture with Apache Flink (berlin Buzzwords 2016)
A Data Streaming Architecture with Apache Flink (berlin Buzzwords 2016)Robert Metzger
 
Apache Beam (incubating)
Apache Beam (incubating)Apache Beam (incubating)
Apache Beam (incubating)Apache Apex
 
January 2016 Flink Community Update & Roadmap 2016
January 2016 Flink Community Update & Roadmap 2016January 2016 Flink Community Update & Roadmap 2016
January 2016 Flink Community Update & Roadmap 2016Robert Metzger
 
Flink Apachecon Presentation
Flink Apachecon PresentationFlink Apachecon Presentation
Flink Apachecon PresentationGyula Fóra
 
Towards sql for streams
Towards sql for streamsTowards sql for streams
Towards sql for streamsRadu Tudoran
 

What's hot (20)

Stream Processing use cases and applications with Apache Apex by Thomas Weise
Stream Processing use cases and applications with Apache Apex by Thomas WeiseStream Processing use cases and applications with Apache Apex by Thomas Weise
Stream Processing use cases and applications with Apache Apex by Thomas Weise
 
Real-time Stream Processing with Apache Flink @ Hadoop Summit
Real-time Stream Processing with Apache Flink @ Hadoop SummitReal-time Stream Processing with Apache Flink @ Hadoop Summit
Real-time Stream Processing with Apache Flink @ Hadoop Summit
 
Intro to Apache Apex - Next Gen Native Hadoop Platform - Hackac
Intro to Apache Apex - Next Gen Native Hadoop Platform - HackacIntro to Apache Apex - Next Gen Native Hadoop Platform - Hackac
Intro to Apache Apex - Next Gen Native Hadoop Platform - Hackac
 
Patterns of the Lambda Architecture -- 2015 April -- Hadoop Summit, Europe
Patterns of the Lambda Architecture -- 2015 April -- Hadoop Summit, EuropePatterns of the Lambda Architecture -- 2015 April -- Hadoop Summit, Europe
Patterns of the Lambda Architecture -- 2015 April -- Hadoop Summit, Europe
 
Apache Big Data EU 2016: Next Gen Big Data Analytics with Apache Apex
Apache Big Data EU 2016: Next Gen Big Data Analytics with Apache ApexApache Big Data EU 2016: Next Gen Big Data Analytics with Apache Apex
Apache Big Data EU 2016: Next Gen Big Data Analytics with Apache Apex
 
The Future of Apache Storm
The Future of Apache StormThe Future of Apache Storm
The Future of Apache Storm
 
Albert Bifet – Apache Samoa: Mining Big Data Streams with Apache Flink
Albert Bifet – Apache Samoa: Mining Big Data Streams with Apache FlinkAlbert Bifet – Apache Samoa: Mining Big Data Streams with Apache Flink
Albert Bifet – Apache Samoa: Mining Big Data Streams with Apache Flink
 
S. Bartoli & F. Pompermaier – A Semantic Big Data Companion
S. Bartoli & F. Pompermaier – A Semantic Big Data CompanionS. Bartoli & F. Pompermaier – A Semantic Big Data Companion
S. Bartoli & F. Pompermaier – A Semantic Big Data Companion
 
Mohamed Amine Abdessemed – Real-time Data Integration with Apache Flink & Kafka
Mohamed Amine Abdessemed – Real-time Data Integration with Apache Flink & KafkaMohamed Amine Abdessemed – Real-time Data Integration with Apache Flink & Kafka
Mohamed Amine Abdessemed – Real-time Data Integration with Apache Flink & Kafka
 
Intro to Apache Apex - Next Gen Platform for Ingest and Transform
Intro to Apache Apex - Next Gen Platform for Ingest and TransformIntro to Apache Apex - Next Gen Platform for Ingest and Transform
Intro to Apache Apex - Next Gen Platform for Ingest and Transform
 
Kafka to Hadoop Ingest with Parsing, Dedup and other Big Data Transformations
Kafka to Hadoop Ingest with Parsing, Dedup and other Big Data TransformationsKafka to Hadoop Ingest with Parsing, Dedup and other Big Data Transformations
Kafka to Hadoop Ingest with Parsing, Dedup and other Big Data Transformations
 
Next Gen Big Data Analytics with Apache Apex
Next Gen Big Data Analytics with Apache Apex Next Gen Big Data Analytics with Apache Apex
Next Gen Big Data Analytics with Apache Apex
 
ApacheCon: Apache Flink - Fast and Reliable Large-Scale Data Processing
ApacheCon: Apache Flink - Fast and Reliable Large-Scale Data ProcessingApacheCon: Apache Flink - Fast and Reliable Large-Scale Data Processing
ApacheCon: Apache Flink - Fast and Reliable Large-Scale Data Processing
 
Data Stream Processing with Apache Flink
Data Stream Processing with Apache FlinkData Stream Processing with Apache Flink
Data Stream Processing with Apache Flink
 
Apache Flink Meetup Munich (November 2015): Flink Overview, Architecture, Int...
Apache Flink Meetup Munich (November 2015): Flink Overview, Architecture, Int...Apache Flink Meetup Munich (November 2015): Flink Overview, Architecture, Int...
Apache Flink Meetup Munich (November 2015): Flink Overview, Architecture, Int...
 
A Data Streaming Architecture with Apache Flink (berlin Buzzwords 2016)
A Data Streaming Architecture with Apache Flink (berlin Buzzwords 2016)A Data Streaming Architecture with Apache Flink (berlin Buzzwords 2016)
A Data Streaming Architecture with Apache Flink (berlin Buzzwords 2016)
 
Apache Beam (incubating)
Apache Beam (incubating)Apache Beam (incubating)
Apache Beam (incubating)
 
January 2016 Flink Community Update & Roadmap 2016
January 2016 Flink Community Update & Roadmap 2016January 2016 Flink Community Update & Roadmap 2016
January 2016 Flink Community Update & Roadmap 2016
 
Flink Apachecon Presentation
Flink Apachecon PresentationFlink Apachecon Presentation
Flink Apachecon Presentation
 
Towards sql for streams
Towards sql for streamsTowards sql for streams
Towards sql for streams
 

Viewers also liked

Introduction to Real-time data processing
Introduction to Real-time data processingIntroduction to Real-time data processing
Introduction to Real-time data processingYogi Devendra Vyavahare
 
GE IOT Predix Time Series & Data Ingestion Service using Apache Apex (Hadoop)
GE IOT Predix Time Series & Data Ingestion Service using Apache Apex (Hadoop)GE IOT Predix Time Series & Data Ingestion Service using Apache Apex (Hadoop)
GE IOT Predix Time Series & Data Ingestion Service using Apache Apex (Hadoop)Apache Apex
 
Real Time Analytics with Apache Cassandra - Cassandra Day Munich
Real Time Analytics with Apache Cassandra - Cassandra Day MunichReal Time Analytics with Apache Cassandra - Cassandra Day Munich
Real Time Analytics with Apache Cassandra - Cassandra Day MunichGuido Schmutz
 
Building Big Data Streaming Architectures
Building Big Data Streaming ArchitecturesBuilding Big Data Streaming Architectures
Building Big Data Streaming ArchitecturesDavid Martínez Rego
 
KDD 2016 Streaming Analytics Tutorial
KDD 2016 Streaming Analytics TutorialKDD 2016 Streaming Analytics Tutorial
KDD 2016 Streaming Analytics TutorialNeera Agarwal
 
RBea: Scalable Real-Time Analytics at King
RBea: Scalable Real-Time Analytics at KingRBea: Scalable Real-Time Analytics at King
RBea: Scalable Real-Time Analytics at KingGyula Fóra
 
Real Time Analytics with Apache Cassandra - Cassandra Day Berlin
Real Time Analytics with Apache Cassandra - Cassandra Day BerlinReal Time Analytics with Apache Cassandra - Cassandra Day Berlin
Real Time Analytics with Apache Cassandra - Cassandra Day BerlinGuido Schmutz
 
Real-time analytics as a service at King
Real-time analytics as a service at King Real-time analytics as a service at King
Real-time analytics as a service at King Gyula Fóra
 
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
 
Big Data Streams Architectures. Why? What? How?
Big Data Streams Architectures. Why? What? How?Big Data Streams Architectures. Why? What? How?
Big Data Streams Architectures. Why? What? How?Anton Nazaruk
 
Stream Analytics in the Enterprise
Stream Analytics in the EnterpriseStream Analytics in the Enterprise
Stream Analytics in the EnterpriseJesus Rodriguez
 
Apache Apex as a YARN Apllication
Apache Apex as a YARN ApllicationApache Apex as a YARN Apllication
Apache Apex as a YARN ApllicationApache Apex
 
Reliable Data Intestion in BigData / IoT
Reliable Data Intestion in BigData / IoTReliable Data Intestion in BigData / IoT
Reliable Data Intestion in BigData / IoTGuido Schmutz
 
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
 
Scalable Distributed Real-Time Clustering for Big Data Streams
Scalable Distributed Real-Time Clustering for Big Data StreamsScalable Distributed Real-Time Clustering for Big Data Streams
Scalable Distributed Real-Time Clustering for Big Data StreamsAntonio Severien
 
Innovation in the Data Warehouse - StampedeCon 2016
Innovation in the Data Warehouse - StampedeCon 2016Innovation in the Data Warehouse - StampedeCon 2016
Innovation in the Data Warehouse - StampedeCon 2016StampedeCon
 
Node Architecture Implications for In-Memory Data Analytics on Scale-in Clusters
Node Architecture Implications for In-Memory Data Analytics on Scale-in ClustersNode Architecture Implications for In-Memory Data Analytics on Scale-in Clusters
Node Architecture Implications for In-Memory Data Analytics on Scale-in ClustersAhsan Javed Awan
 
The end of polling : why and how to transform a REST API into a Data Streamin...
The end of polling : why and how to transform a REST API into a Data Streamin...The end of polling : why and how to transform a REST API into a Data Streamin...
The end of polling : why and how to transform a REST API into a Data Streamin...Audrey Neveu
 
Oracle Stream Analytics - Simplifying Stream Processing
Oracle Stream Analytics - Simplifying Stream ProcessingOracle Stream Analytics - Simplifying Stream Processing
Oracle Stream Analytics - Simplifying Stream ProcessingGuido Schmutz
 

Viewers also liked (20)

Introduction to Real-time data processing
Introduction to Real-time data processingIntroduction to Real-time data processing
Introduction to Real-time data processing
 
GE IOT Predix Time Series & Data Ingestion Service using Apache Apex (Hadoop)
GE IOT Predix Time Series & Data Ingestion Service using Apache Apex (Hadoop)GE IOT Predix Time Series & Data Ingestion Service using Apache Apex (Hadoop)
GE IOT Predix Time Series & Data Ingestion Service using Apache Apex (Hadoop)
 
Real Time Analytics with Apache Cassandra - Cassandra Day Munich
Real Time Analytics with Apache Cassandra - Cassandra Day MunichReal Time Analytics with Apache Cassandra - Cassandra Day Munich
Real Time Analytics with Apache Cassandra - Cassandra Day Munich
 
Building Big Data Streaming Architectures
Building Big Data Streaming ArchitecturesBuilding Big Data Streaming Architectures
Building Big Data Streaming Architectures
 
KDD 2016 Streaming Analytics Tutorial
KDD 2016 Streaming Analytics TutorialKDD 2016 Streaming Analytics Tutorial
KDD 2016 Streaming Analytics Tutorial
 
RBea: Scalable Real-Time Analytics at King
RBea: Scalable Real-Time Analytics at KingRBea: Scalable Real-Time Analytics at King
RBea: Scalable Real-Time Analytics at King
 
Real Time Analytics with Apache Cassandra - Cassandra Day Berlin
Real Time Analytics with Apache Cassandra - Cassandra Day BerlinReal Time Analytics with Apache Cassandra - Cassandra Day Berlin
Real Time Analytics with Apache Cassandra - Cassandra Day Berlin
 
Real-time analytics as a service at King
Real-time analytics as a service at King Real-time analytics as a service at King
Real-time analytics as a service at King
 
Streaming Analytics
Streaming AnalyticsStreaming Analytics
Streaming Analytics
 
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
 
Big Data Streams Architectures. Why? What? How?
Big Data Streams Architectures. Why? What? How?Big Data Streams Architectures. Why? What? How?
Big Data Streams Architectures. Why? What? How?
 
Stream Analytics in the Enterprise
Stream Analytics in the EnterpriseStream Analytics in the Enterprise
Stream Analytics in the Enterprise
 
Apache Apex as a YARN Apllication
Apache Apex as a YARN ApllicationApache Apex as a YARN Apllication
Apache Apex as a YARN Apllication
 
Reliable Data Intestion in BigData / IoT
Reliable Data Intestion in BigData / IoTReliable Data Intestion in BigData / IoT
Reliable Data Intestion in BigData / IoT
 
Stream Processing Everywhere - What to use?
Stream Processing Everywhere - What to use?Stream Processing Everywhere - What to use?
Stream Processing Everywhere - What to use?
 
Scalable Distributed Real-Time Clustering for Big Data Streams
Scalable Distributed Real-Time Clustering for Big Data StreamsScalable Distributed Real-Time Clustering for Big Data Streams
Scalable Distributed Real-Time Clustering for Big Data Streams
 
Innovation in the Data Warehouse - StampedeCon 2016
Innovation in the Data Warehouse - StampedeCon 2016Innovation in the Data Warehouse - StampedeCon 2016
Innovation in the Data Warehouse - StampedeCon 2016
 
Node Architecture Implications for In-Memory Data Analytics on Scale-in Clusters
Node Architecture Implications for In-Memory Data Analytics on Scale-in ClustersNode Architecture Implications for In-Memory Data Analytics on Scale-in Clusters
Node Architecture Implications for In-Memory Data Analytics on Scale-in Clusters
 
The end of polling : why and how to transform a REST API into a Data Streamin...
The end of polling : why and how to transform a REST API into a Data Streamin...The end of polling : why and how to transform a REST API into a Data Streamin...
The end of polling : why and how to transform a REST API into a Data Streamin...
 
Oracle Stream Analytics - Simplifying Stream Processing
Oracle Stream Analytics - Simplifying Stream ProcessingOracle Stream Analytics - Simplifying Stream Processing
Oracle Stream Analytics - Simplifying Stream Processing
 

Similar to Large-Scale Stream Processing in the Hadoop Ecosystem - Hadoop Summit 2016

Apache Flink: Past, Present and Future
Apache Flink: Past, Present and FutureApache Flink: Past, Present and Future
Apache Flink: Past, Present and FutureGyula Fóra
 
Cloud Lambda Architecture Patterns
Cloud Lambda Architecture PatternsCloud Lambda Architecture Patterns
Cloud Lambda Architecture PatternsAsis Mohanty
 
Introduction to Apache Apex and writing a big data streaming application
Introduction to Apache Apex and writing a big data streaming application  Introduction to Apache Apex and writing a big data streaming application
Introduction to Apache Apex and writing a big data streaming application Apache Apex
 
Cloudera Impala - Las Vegas Big Data Meetup Nov 5th 2014
Cloudera Impala - Las Vegas Big Data Meetup Nov 5th 2014Cloudera Impala - Las Vegas Big Data Meetup Nov 5th 2014
Cloudera Impala - Las Vegas Big Data Meetup Nov 5th 2014cdmaxime
 
SQL Engines for Hadoop - The case for Impala
SQL Engines for Hadoop - The case for ImpalaSQL Engines for Hadoop - The case for Impala
SQL Engines for Hadoop - The case for Impalamarkgrover
 
Real time fraud detection at 1+M scale on hadoop stack
Real time fraud detection at 1+M scale on hadoop stackReal time fraud detection at 1+M scale on hadoop stack
Real time fraud detection at 1+M scale on hadoop stackDataWorks Summit/Hadoop Summit
 
Stream Data Processing at Big Data Landscape by Oleksandr Fedirko
Stream Data Processing at Big Data Landscape by Oleksandr Fedirko Stream Data Processing at Big Data Landscape by Oleksandr Fedirko
Stream Data Processing at Big Data Landscape by Oleksandr Fedirko GlobalLogic Ukraine
 
Replicate from Oracle to data warehouses and analytics
Replicate from Oracle to data warehouses and analyticsReplicate from Oracle to data warehouses and analytics
Replicate from Oracle to data warehouses and analyticsContinuent
 
Spark and Couchbase: Augmenting the Operational Database with Spark
Spark and Couchbase: Augmenting the Operational Database with SparkSpark and Couchbase: Augmenting the Operational Database with Spark
Spark and Couchbase: Augmenting the Operational Database with SparkSpark Summit
 
Apache Spark Fundamentals
Apache Spark FundamentalsApache Spark Fundamentals
Apache Spark FundamentalsZahra Eskandari
 
Accelerating apache spark with rdma
Accelerating apache spark with rdmaAccelerating apache spark with rdma
Accelerating apache spark with rdmainside-BigData.com
 
Spark introduction and architecture
Spark introduction and architectureSpark introduction and architecture
Spark introduction and architectureSohil Jain
 
Spark introduction and architecture
Spark introduction and architectureSpark introduction and architecture
Spark introduction and architectureSohil Jain
 
QCon London - Stream Processing with Apache Flink
QCon London - Stream Processing with Apache FlinkQCon London - Stream Processing with Apache Flink
QCon London - Stream Processing with Apache FlinkRobert Metzger
 
October 2016 HUG: Architecture of an Open Source RDBMS powered by HBase and ...
October 2016 HUG: Architecture of an Open Source RDBMS powered by HBase and ...October 2016 HUG: Architecture of an Open Source RDBMS powered by HBase and ...
October 2016 HUG: Architecture of an Open Source RDBMS powered by HBase and ...Yahoo Developer Network
 
GOTO Night Amsterdam - Stream processing with Apache Flink
GOTO Night Amsterdam - Stream processing with Apache FlinkGOTO Night Amsterdam - Stream processing with Apache Flink
GOTO Night Amsterdam - Stream processing with Apache FlinkRobert Metzger
 
Fabian Hueske – Cascading on Flink
Fabian Hueske – Cascading on FlinkFabian Hueske – Cascading on Flink
Fabian Hueske – Cascading on FlinkFlink Forward
 
Stream Processing with Apache Flink (Flink.tw Meetup 2016/07/19)
Stream Processing with Apache Flink (Flink.tw Meetup 2016/07/19)Stream Processing with Apache Flink (Flink.tw Meetup 2016/07/19)
Stream Processing with Apache Flink (Flink.tw Meetup 2016/07/19)Apache Flink Taiwan User Group
 

Similar to Large-Scale Stream Processing in the Hadoop Ecosystem - Hadoop Summit 2016 (20)

Apache Flink: Past, Present and Future
Apache Flink: Past, Present and FutureApache Flink: Past, Present and Future
Apache Flink: Past, Present and Future
 
Introduction to Hadoop Administration
Introduction to Hadoop AdministrationIntroduction to Hadoop Administration
Introduction to Hadoop Administration
 
Introduction to Hadoop Administration
Introduction to Hadoop AdministrationIntroduction to Hadoop Administration
Introduction to Hadoop Administration
 
Cloud Lambda Architecture Patterns
Cloud Lambda Architecture PatternsCloud Lambda Architecture Patterns
Cloud Lambda Architecture Patterns
 
Introduction to Apache Apex and writing a big data streaming application
Introduction to Apache Apex and writing a big data streaming application  Introduction to Apache Apex and writing a big data streaming application
Introduction to Apache Apex and writing a big data streaming application
 
Cloudera Impala - Las Vegas Big Data Meetup Nov 5th 2014
Cloudera Impala - Las Vegas Big Data Meetup Nov 5th 2014Cloudera Impala - Las Vegas Big Data Meetup Nov 5th 2014
Cloudera Impala - Las Vegas Big Data Meetup Nov 5th 2014
 
SQL Engines for Hadoop - The case for Impala
SQL Engines for Hadoop - The case for ImpalaSQL Engines for Hadoop - The case for Impala
SQL Engines for Hadoop - The case for Impala
 
Real time fraud detection at 1+M scale on hadoop stack
Real time fraud detection at 1+M scale on hadoop stackReal time fraud detection at 1+M scale on hadoop stack
Real time fraud detection at 1+M scale on hadoop stack
 
Stream Data Processing at Big Data Landscape by Oleksandr Fedirko
Stream Data Processing at Big Data Landscape by Oleksandr Fedirko Stream Data Processing at Big Data Landscape by Oleksandr Fedirko
Stream Data Processing at Big Data Landscape by Oleksandr Fedirko
 
Replicate from Oracle to data warehouses and analytics
Replicate from Oracle to data warehouses and analyticsReplicate from Oracle to data warehouses and analytics
Replicate from Oracle to data warehouses and analytics
 
Spark and Couchbase: Augmenting the Operational Database with Spark
Spark and Couchbase: Augmenting the Operational Database with SparkSpark and Couchbase: Augmenting the Operational Database with Spark
Spark and Couchbase: Augmenting the Operational Database with Spark
 
Apache Spark Fundamentals
Apache Spark FundamentalsApache Spark Fundamentals
Apache Spark Fundamentals
 
Accelerating apache spark with rdma
Accelerating apache spark with rdmaAccelerating apache spark with rdma
Accelerating apache spark with rdma
 
Spark introduction and architecture
Spark introduction and architectureSpark introduction and architecture
Spark introduction and architecture
 
Spark introduction and architecture
Spark introduction and architectureSpark introduction and architecture
Spark introduction and architecture
 
QCon London - Stream Processing with Apache Flink
QCon London - Stream Processing with Apache FlinkQCon London - Stream Processing with Apache Flink
QCon London - Stream Processing with Apache Flink
 
October 2016 HUG: Architecture of an Open Source RDBMS powered by HBase and ...
October 2016 HUG: Architecture of an Open Source RDBMS powered by HBase and ...October 2016 HUG: Architecture of an Open Source RDBMS powered by HBase and ...
October 2016 HUG: Architecture of an Open Source RDBMS powered by HBase and ...
 
GOTO Night Amsterdam - Stream processing with Apache Flink
GOTO Night Amsterdam - Stream processing with Apache FlinkGOTO Night Amsterdam - Stream processing with Apache Flink
GOTO Night Amsterdam - Stream processing with Apache Flink
 
Fabian Hueske – Cascading on Flink
Fabian Hueske – Cascading on FlinkFabian Hueske – Cascading on Flink
Fabian Hueske – Cascading on Flink
 
Stream Processing with Apache Flink (Flink.tw Meetup 2016/07/19)
Stream Processing with Apache Flink (Flink.tw Meetup 2016/07/19)Stream Processing with Apache Flink (Flink.tw Meetup 2016/07/19)
Stream Processing with Apache Flink (Flink.tw Meetup 2016/07/19)
 

Recently uploaded

9711147426✨Call In girls Gurgaon Sector 31. SCO 25 escort service
9711147426✨Call In girls Gurgaon Sector 31. SCO 25 escort service9711147426✨Call In girls Gurgaon Sector 31. SCO 25 escort service
9711147426✨Call In girls Gurgaon Sector 31. SCO 25 escort servicejennyeacort
 
From idea to production in a day – Leveraging Azure ML and Streamlit to build...
From idea to production in a day – Leveraging Azure ML and Streamlit to build...From idea to production in a day – Leveraging Azure ML and Streamlit to build...
From idea to production in a day – Leveraging Azure ML and Streamlit to build...Florian Roscheck
 
Consent & Privacy Signals on Google *Pixels* - MeasureCamp Amsterdam 2024
Consent & Privacy Signals on Google *Pixels* - MeasureCamp Amsterdam 2024Consent & Privacy Signals on Google *Pixels* - MeasureCamp Amsterdam 2024
Consent & Privacy Signals on Google *Pixels* - MeasureCamp Amsterdam 2024thyngster
 
Predictive Analysis - Using Insight-informed Data to Determine Factors Drivin...
Predictive Analysis - Using Insight-informed Data to Determine Factors Drivin...Predictive Analysis - Using Insight-informed Data to Determine Factors Drivin...
Predictive Analysis - Using Insight-informed Data to Determine Factors Drivin...ThinkInnovation
 
PKS-TGC-1084-630 - Stage 1 Proposal.pptx
PKS-TGC-1084-630 - Stage 1 Proposal.pptxPKS-TGC-1084-630 - Stage 1 Proposal.pptx
PKS-TGC-1084-630 - Stage 1 Proposal.pptxPramod Kumar Srivastava
 
Indian Call Girls in Abu Dhabi O5286O24O8 Call Girls in Abu Dhabi By Independ...
Indian Call Girls in Abu Dhabi O5286O24O8 Call Girls in Abu Dhabi By Independ...Indian Call Girls in Abu Dhabi O5286O24O8 Call Girls in Abu Dhabi By Independ...
Indian Call Girls in Abu Dhabi O5286O24O8 Call Girls in Abu Dhabi By Independ...dajasot375
 
Brighton SEO | April 2024 | Data Storytelling
Brighton SEO | April 2024 | Data StorytellingBrighton SEO | April 2024 | Data Storytelling
Brighton SEO | April 2024 | Data StorytellingNeil Barnes
 
RA-11058_IRR-COMPRESS Do 198 series of 1998
RA-11058_IRR-COMPRESS Do 198 series of 1998RA-11058_IRR-COMPRESS Do 198 series of 1998
RA-11058_IRR-COMPRESS Do 198 series of 1998YohFuh
 
专业一比一美国俄亥俄大学毕业证成绩单pdf电子版制作修改
专业一比一美国俄亥俄大学毕业证成绩单pdf电子版制作修改专业一比一美国俄亥俄大学毕业证成绩单pdf电子版制作修改
专业一比一美国俄亥俄大学毕业证成绩单pdf电子版制作修改yuu sss
 
Amazon TQM (2) Amazon TQM (2)Amazon TQM (2).pptx
Amazon TQM (2) Amazon TQM (2)Amazon TQM (2).pptxAmazon TQM (2) Amazon TQM (2)Amazon TQM (2).pptx
Amazon TQM (2) Amazon TQM (2)Amazon TQM (2).pptxAbdelrhman abooda
 
dokumen.tips_chapter-4-transient-heat-conduction-mehmet-kanoglu.ppt
dokumen.tips_chapter-4-transient-heat-conduction-mehmet-kanoglu.pptdokumen.tips_chapter-4-transient-heat-conduction-mehmet-kanoglu.ppt
dokumen.tips_chapter-4-transient-heat-conduction-mehmet-kanoglu.pptSonatrach
 
RS 9000 Call In girls Dwarka Mor (DELHI)⇛9711147426🔝Delhi
RS 9000 Call In girls Dwarka Mor (DELHI)⇛9711147426🔝DelhiRS 9000 Call In girls Dwarka Mor (DELHI)⇛9711147426🔝Delhi
RS 9000 Call In girls Dwarka Mor (DELHI)⇛9711147426🔝Delhijennyeacort
 
Kantar AI Summit- Under Embargo till Wednesday, 24th April 2024, 4 PM, IST.pdf
Kantar AI Summit- Under Embargo till Wednesday, 24th April 2024, 4 PM, IST.pdfKantar AI Summit- Under Embargo till Wednesday, 24th April 2024, 4 PM, IST.pdf
Kantar AI Summit- Under Embargo till Wednesday, 24th April 2024, 4 PM, IST.pdfSocial Samosa
 
Data Science Jobs and Salaries Analysis.pptx
Data Science Jobs and Salaries Analysis.pptxData Science Jobs and Salaries Analysis.pptx
Data Science Jobs and Salaries Analysis.pptxFurkanTasci3
 
Effects of Smartphone Addiction on the Academic Performances of Grades 9 to 1...
Effects of Smartphone Addiction on the Academic Performances of Grades 9 to 1...Effects of Smartphone Addiction on the Academic Performances of Grades 9 to 1...
Effects of Smartphone Addiction on the Academic Performances of Grades 9 to 1...limedy534
 
VIP High Class Call Girls Jamshedpur Anushka 8250192130 Independent Escort Se...
VIP High Class Call Girls Jamshedpur Anushka 8250192130 Independent Escort Se...VIP High Class Call Girls Jamshedpur Anushka 8250192130 Independent Escort Se...
VIP High Class Call Girls Jamshedpur Anushka 8250192130 Independent Escort Se...Suhani Kapoor
 
Call Girls in Defence Colony Delhi 💯Call Us 🔝8264348440🔝
Call Girls in Defence Colony Delhi 💯Call Us 🔝8264348440🔝Call Girls in Defence Colony Delhi 💯Call Us 🔝8264348440🔝
Call Girls in Defence Colony Delhi 💯Call Us 🔝8264348440🔝soniya singh
 
20240419 - Measurecamp Amsterdam - SAM.pdf
20240419 - Measurecamp Amsterdam - SAM.pdf20240419 - Measurecamp Amsterdam - SAM.pdf
20240419 - Measurecamp Amsterdam - SAM.pdfHuman37
 
办理学位证中佛罗里达大学毕业证,UCF成绩单原版一比一
办理学位证中佛罗里达大学毕业证,UCF成绩单原版一比一办理学位证中佛罗里达大学毕业证,UCF成绩单原版一比一
办理学位证中佛罗里达大学毕业证,UCF成绩单原版一比一F sss
 
RadioAdProWritingCinderellabyButleri.pdf
RadioAdProWritingCinderellabyButleri.pdfRadioAdProWritingCinderellabyButleri.pdf
RadioAdProWritingCinderellabyButleri.pdfgstagge
 

Recently uploaded (20)

9711147426✨Call In girls Gurgaon Sector 31. SCO 25 escort service
9711147426✨Call In girls Gurgaon Sector 31. SCO 25 escort service9711147426✨Call In girls Gurgaon Sector 31. SCO 25 escort service
9711147426✨Call In girls Gurgaon Sector 31. SCO 25 escort service
 
From idea to production in a day – Leveraging Azure ML and Streamlit to build...
From idea to production in a day – Leveraging Azure ML and Streamlit to build...From idea to production in a day – Leveraging Azure ML and Streamlit to build...
From idea to production in a day – Leveraging Azure ML and Streamlit to build...
 
Consent & Privacy Signals on Google *Pixels* - MeasureCamp Amsterdam 2024
Consent & Privacy Signals on Google *Pixels* - MeasureCamp Amsterdam 2024Consent & Privacy Signals on Google *Pixels* - MeasureCamp Amsterdam 2024
Consent & Privacy Signals on Google *Pixels* - MeasureCamp Amsterdam 2024
 
Predictive Analysis - Using Insight-informed Data to Determine Factors Drivin...
Predictive Analysis - Using Insight-informed Data to Determine Factors Drivin...Predictive Analysis - Using Insight-informed Data to Determine Factors Drivin...
Predictive Analysis - Using Insight-informed Data to Determine Factors Drivin...
 
PKS-TGC-1084-630 - Stage 1 Proposal.pptx
PKS-TGC-1084-630 - Stage 1 Proposal.pptxPKS-TGC-1084-630 - Stage 1 Proposal.pptx
PKS-TGC-1084-630 - Stage 1 Proposal.pptx
 
Indian Call Girls in Abu Dhabi O5286O24O8 Call Girls in Abu Dhabi By Independ...
Indian Call Girls in Abu Dhabi O5286O24O8 Call Girls in Abu Dhabi By Independ...Indian Call Girls in Abu Dhabi O5286O24O8 Call Girls in Abu Dhabi By Independ...
Indian Call Girls in Abu Dhabi O5286O24O8 Call Girls in Abu Dhabi By Independ...
 
Brighton SEO | April 2024 | Data Storytelling
Brighton SEO | April 2024 | Data StorytellingBrighton SEO | April 2024 | Data Storytelling
Brighton SEO | April 2024 | Data Storytelling
 
RA-11058_IRR-COMPRESS Do 198 series of 1998
RA-11058_IRR-COMPRESS Do 198 series of 1998RA-11058_IRR-COMPRESS Do 198 series of 1998
RA-11058_IRR-COMPRESS Do 198 series of 1998
 
专业一比一美国俄亥俄大学毕业证成绩单pdf电子版制作修改
专业一比一美国俄亥俄大学毕业证成绩单pdf电子版制作修改专业一比一美国俄亥俄大学毕业证成绩单pdf电子版制作修改
专业一比一美国俄亥俄大学毕业证成绩单pdf电子版制作修改
 
Amazon TQM (2) Amazon TQM (2)Amazon TQM (2).pptx
Amazon TQM (2) Amazon TQM (2)Amazon TQM (2).pptxAmazon TQM (2) Amazon TQM (2)Amazon TQM (2).pptx
Amazon TQM (2) Amazon TQM (2)Amazon TQM (2).pptx
 
dokumen.tips_chapter-4-transient-heat-conduction-mehmet-kanoglu.ppt
dokumen.tips_chapter-4-transient-heat-conduction-mehmet-kanoglu.pptdokumen.tips_chapter-4-transient-heat-conduction-mehmet-kanoglu.ppt
dokumen.tips_chapter-4-transient-heat-conduction-mehmet-kanoglu.ppt
 
RS 9000 Call In girls Dwarka Mor (DELHI)⇛9711147426🔝Delhi
RS 9000 Call In girls Dwarka Mor (DELHI)⇛9711147426🔝DelhiRS 9000 Call In girls Dwarka Mor (DELHI)⇛9711147426🔝Delhi
RS 9000 Call In girls Dwarka Mor (DELHI)⇛9711147426🔝Delhi
 
Kantar AI Summit- Under Embargo till Wednesday, 24th April 2024, 4 PM, IST.pdf
Kantar AI Summit- Under Embargo till Wednesday, 24th April 2024, 4 PM, IST.pdfKantar AI Summit- Under Embargo till Wednesday, 24th April 2024, 4 PM, IST.pdf
Kantar AI Summit- Under Embargo till Wednesday, 24th April 2024, 4 PM, IST.pdf
 
Data Science Jobs and Salaries Analysis.pptx
Data Science Jobs and Salaries Analysis.pptxData Science Jobs and Salaries Analysis.pptx
Data Science Jobs and Salaries Analysis.pptx
 
Effects of Smartphone Addiction on the Academic Performances of Grades 9 to 1...
Effects of Smartphone Addiction on the Academic Performances of Grades 9 to 1...Effects of Smartphone Addiction on the Academic Performances of Grades 9 to 1...
Effects of Smartphone Addiction on the Academic Performances of Grades 9 to 1...
 
VIP High Class Call Girls Jamshedpur Anushka 8250192130 Independent Escort Se...
VIP High Class Call Girls Jamshedpur Anushka 8250192130 Independent Escort Se...VIP High Class Call Girls Jamshedpur Anushka 8250192130 Independent Escort Se...
VIP High Class Call Girls Jamshedpur Anushka 8250192130 Independent Escort Se...
 
Call Girls in Defence Colony Delhi 💯Call Us 🔝8264348440🔝
Call Girls in Defence Colony Delhi 💯Call Us 🔝8264348440🔝Call Girls in Defence Colony Delhi 💯Call Us 🔝8264348440🔝
Call Girls in Defence Colony Delhi 💯Call Us 🔝8264348440🔝
 
20240419 - Measurecamp Amsterdam - SAM.pdf
20240419 - Measurecamp Amsterdam - SAM.pdf20240419 - Measurecamp Amsterdam - SAM.pdf
20240419 - Measurecamp Amsterdam - SAM.pdf
 
办理学位证中佛罗里达大学毕业证,UCF成绩单原版一比一
办理学位证中佛罗里达大学毕业证,UCF成绩单原版一比一办理学位证中佛罗里达大学毕业证,UCF成绩单原版一比一
办理学位证中佛罗里达大学毕业证,UCF成绩单原版一比一
 
RadioAdProWritingCinderellabyButleri.pdf
RadioAdProWritingCinderellabyButleri.pdfRadioAdProWritingCinderellabyButleri.pdf
RadioAdProWritingCinderellabyButleri.pdf
 

Large-Scale Stream Processing in the Hadoop Ecosystem - Hadoop Summit 2016

  • 1. Large-Scale Stream Processing in the Hadoop Ecosystem Gyula Fóra gyula.fora@king.com @GyulaFora Márton Balassi mbalassi@apache.org @MartonBalassi
  • 3. Real-Time Player Statistics • Compute real-time, queryable statistics • Billions of events / day • Millions of active users / day • State quickly grows beyond memory • Complex event processing logic • Strong consistency requirements 2016-04-13 Hadoop Summit 2016 (Dublin) 3 DB DB
  • 4. Real-time dashboard of telco network 2016-04-13 Hadoop Summit 2016 (Dublin) 4 • Example query: Download speed heatmap of premium users in the last 5 minutes • Dependant on ~1 TB slowly changing enrichment data • Multi GB/s input rate • Some of the use cases require complex windowing logic DB
  • 6. Apache Streaming Landscape 2016-04-13 Hadoop Summit 2016 (Dublin) 6
  • 7. Apache Storm • Pioneer of real-time analytics • Distributed dataflow abstraction with low-level control • Time windowing and state introduced recently When to use Storm • Very low latency requirements • No need for advanced state/windowing • At-least-once is acceptable 2016-04-13 Hadoop Summit 2016 (Dublin) 7
  • 8. Apache Samza • Builds heavily on Kafka’s log based philosophy • Pluggable components, but runs best with Kafka • Scalable operator state with RocksDB • Basic time windowing When to use Samza • Join streams with large states • At-least-once is acceptable 2016-04-13 Hadoop Summit 2016 (Dublin) 8
  • 9. Kafka Streams • Streaming library on top of Apache Kafka • Similar features to Samza but nicer API • Big win for operational simplicity When to use Kafka Streams • Kafka based data infrastructure • Join streams with large states • At-least-once is acceptable 2016-04-13 Hadoop Summit 2016 (Dublin) 9
  • 10. Apache Spark • Unified batch and stream processing over a batch runtime • Good integration with batch programs • Lags behind recent streaming advancements but evolving quickly • Spark 2.0 comes with new streaming engine When to use Spark • Simpler data exploration • Combine with (Spark) batch analytics • Medium latency is acceptable 2016-04-13 Hadoop Summit 2016 (Dublin) 10
  • 11. Apache Flink • Unified batch and stream processing over dataflow engine • Leader of open source streaming innovation • Highly flexible and robust stateful and windowing computations • Savepoints for state management When to use Flink • Advanced streaming analytics • Complex windowing/state • Need for high TP - low latency 2016-04-13 Hadoop Summit 2016 (Dublin) 11 Batch data Kafka, RabbitMQ ... HDFS, JDBC ... Stream Data
  • 12. Apache Apex • Native streaming engine built natively on YARN • Stateful operators with checkpointing to HDFS • Advanced partitioning support with locality optimizations When to use Apex • Advanced streaming analytics • Very low latency requirements • Need extensive operator library 2016-04-13 Hadoop Summit 2016 (Dublin) 12
  • 13. System comparison 2016-04-13 Hadoop Summit 2016 (Dublin) 13 Model Native Micro-batch Native Native Native API Compositional Declarative Compositional Declarative Compositional Fault tolerance Record ACKs RDD-based Log-based Checkpoints Checkpoints Guarantee At-least-once Exactly-once At-least-once Exactly-once Exactly-once State Stateful operators State as DStream Stateful operators Stateful operators Stateful operators Windowing Time based Time based Time based Flexible Time based Latency Very-Low High Low Low Very-Low Throughput Medium Very-High High Very-High Very-High
  • 15. Native Streaming 2016-04-13 Hadoop Summit 2016 (Dublin) 15
  • 16. Distributed dataflow runtime • Long standing operators • Pipelined execution • Usually possible to create cyclic flows 2016-04-13 Hadoop Summit 2016 (Dublin) 16 Pros • Full expressivity • Low-latency execution • Stateful operators Cons • Fault-tolerance is hard • Throughput may suffer • Load balancing is an issue
  • 18. Micro-batch runtime • Computation broken down to time intervals • Load aware scheduling • Easy interaction with batch 2016-04-13 Hadoop Summit 2016 (Dublin) 18 Pros • Easy to reason about • High-throughput • FT comes for “free” • Dynamic load balancing Cons • Latency depends on batch size • Limited expressivity • Stateless by nature
  • 20. Hierarchy of Streaming APIs 2016-04-13 Hadoop Summit 2016 (Dublin) 20 DataStream DStream  Transformations, abstract operators  For both engineers and data analysts  Allows (some) automatic query optimization Spout, Consumer, Bolt, Task, Topology  Direct access to the execution graph  Suitable for engineers  Fine grained access but lower productivity Declarative Compositional
  • 21. Apache Beam • One API to rule them all: combined batch and streaming analytics • Open sourced by Google, based on DataFlow • Advanced windowing • Runners on different systems • Google Cloud • Flink • Spark • (Others to follow…) • Useful for benchmarking? 2016-04-13 Hadoop Summit 2016 (Dublin) 21
  • 22. Apache Beam • What results are calculated? • Where in event time are results calculated? • When in processing time are results materialized? • How do refinements of results relate? 2016-04-13 Hadoop Summit 2016 (Dublin) 22
  • 24. WordCount 2016-04-13 Hadoop Summit 2016 (Dublin) 24 storm dublin flink apache storm spark streaming samza storm flink apache flink bigdata storm flink streaming (storm, 4) (dublin, 1) (flink, 4) (apache, 2) (spark, 1) (streaming, 2) (samza, 1) (bigdata, 1)
  • 25. Storm 2016-04-13 Hadoop Summit 2016 (Dublin) 25 Assembling the topology TopologyBuilder builder = new TopologyBuilder(); builder.setSpout("spout", new SentenceSpout(), 5); builder.setBolt("split", new Splitter(), 8).shuffleGrouping("spout"); builder.setBolt("count", new Counter(), 12) .fieldsGrouping("split", new Fields("word")); public class Counter extends BaseBasicBolt { 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)); } } Rolling word count bolt
  • 26. Samza 2016-04-13 Hadoop Summit 2016 (Dublin) 26 public class WordCountTask implements StreamTask { private KeyValueStore<String, Integer> store; public void process( IncomingMessageEnvelope envelope, MessageCollector collector, TaskCoordinator coordinator) { String word = envelope.getMessage(); Integer count = store.get(word); if(count == null){count = 0;} store.put(word, count + 1); collector.send(new OutgoingMessageEnvelope(new SystemStream("kafka", ”wc"), Tuple2.of(word, count))); } } Rolling word count task
  • 27. Apex 2016-04-13 Hadoop Summit 2016 (Dublin) 27
  • 28. Flink 2016-04-13 Hadoop Summit 2016 (Dublin) 28 val lines: DataStream[String] = env.socketTextStream(...) lines.flatMap {line => line.split(" ") .map(word => Word(word,1))} .keyBy("word") .sum("frequency").print() case class Word (word: String, frequency: Int) val lines: DataStream[String] = env.socketTextStream(...) lines.flatMap {line => line.split(" ") .map(word => Word(word,1))} .keyBy("word"). .timeWindow(Time.seconds(5)) .sum("frequency").print() Rolling word count Window word count
  • 29. Spark 2016-04-13 Hadoop Summit 2016 (Dublin) 29 Window word count Rolling word count (new feature ) val lines = env.fromSocketStream(...) val words = lines.flatMap(line => line.split(" ")) .map(word => (word,1)) val wordCounts = words.reduceByKey(_ + _) wordCounts.print() val func = (word: String, one: Option[Int], state: State[Int]) => { val sum = one.getOrElse(0) + state.getOption.getOrElse(0) val output = (word, sum) state.update(sum) output } val stateDstream = wordDstream.mapWithState( StateSpec.function(func).initialState(initialRDD)) stateDstream.print()
  • 30. Beam 2016-04-13 Hadoop Summit 2016 (Dublin) 30 PCollection<String> windowedLines = input .apply(Window.<String>into( FixedWindows.of(Duration.standardMinutes(5)))); PCollection<KV<String, Long>> wordCounts = windowedLines .apply(ParDo.of(new DoFn<String, String>() { @Override public void processElement(ProcessContext c) { for (String word : c.element().split("[^a-zA-Z']+")){ if (!word.isEmpty()) { c.output(word); } } } })) .apply(Count.<String>perElement()); Window word count (minimalistic version)
  • 31. Fault tolerance and stateful processing
  • 32. Fault tolerance intro • Fault-tolerance in streaming systems is inherently harder than in batch • Can’t just restart computation • State is a problem • Fast recovery is crucial • Streaming topologies run 24/7 for a long period • Fault-tolerance is a complex issue • No single point of failure is allowed • Guaranteeing input processing • Consistent operator state • Fast recovery • At-least-once vs Exactly-once semantics 2016-04-13 Hadoop Summit 2016 (Dublin) 32
  • 33. Storm record acknowledgements • Track the lineage of tuples as they are processed (anchors and acks) • Special “acker” bolts track each lineage DAG (efficient xor based algorithm) • Replay the root of failed (or timed out) tuples 2016-04-13 Hadoop Summit 2016 (Dublin) 33
  • 34. Samza offset tracking • Exploits the properties of a durable, offset based messaging layer • Each task maintains its current offset, which moves forward as it processes elements • The offset is checkpointed and restored on failure (some messages might be repeated) 2016-04-13 Hadoop Summit 2016 (Dublin) 34
  • 35. Spark RDD recomputation • Immutable data model with repeatable computation • Failed RDDs are recomputed using their lineage • Checkpoint RDDs to reduce lineage length • Parallel recovery of failed RDDs • Exactly-once semantics 2016-04-13 Hadoop Summit 2016 (Dublin) 35
  • 36. Flink state checkpointing • Consistent global snapshots with exactly-once semantics • Algorithm designed for stateful dataflows (minimal runtime overhead) • Pluggable state backends: Memory, FS, RocksDB, MySQL… 2016-04-13 Hadoop Summit 2016 (Dublin) 36
  • 37. Apex state checkpointing • Algorithms similar to Flink’s but also buffers output windows • Larger memory overhead but faster, granular recovery • Pluggable checkpoint backend, HDFS by default 2016-04-13 Hadoop Summit 2016 (Dublin) 37
  • 38. Performance How much does all this matter? The winners of last year's Twitter hack week managed to reduce the resources needed for a specific job by 99%. [1] There are many recent benchmarks out there • Storm, Flink & Spark by Yahoo [2] • Apex by DataTorrent [3,4] • Flink by data Artisans [1,5] [1] http://data-artisans.com/extending-the-yahoo-streaming-benchmark [2] https://yahooeng.tumblr.com/post/135321837876/benchmarking-streaming-computation-engines-at [3] https://www.datatorrent.com/blog/blog-implementing-linear-road-benchmark-in-apex/ [4] https://www.datatorrent.com/blog/blog-apex-performance-benchmark/ [5] http://data-artisans.com/high-throughput-low-latency-and-exactly-once-stream-processing-with-apache-flink/ 2016-04-13 Hadoop Summit 2016 (Dublin) 38
  • 39. Next steps for streaming • Dynamic scaling (with state) • Rolling upgrades • Better state handling • More Beam runners • Libraries: CEP, ML • Better batch integration 2016-04-13 Hadoop Summit 2016 (Dublin) 39
  • 41. Summary • Streaming systems are gaining popularity with many businesses migrating some of their infrastructure • The open source space sees a lot of innovation • When choosing an application consider your specific use cases, do not just follow the herd • We have a recommended reading section :) 2016-04-13 Hadoop Summit 2016 (Dublin) 41
  • 44. Apache Beam • http://beam.incubator.apache.org/beam/capability/2016/03/17/c apability-matrix.html • https://www.oreilly.com/ideas/the-world-beyond-batch- streaming-101 • https://www.oreilly.com/ideas/the-world-beyond-batch- streaming-102 2016-04-13 Hadoop Summit 2016 (Dublin) 44
  • 45. Apache Spark Streaming • https://databricks.com/blog/2016/02/01/faster-stateful-stream- processing-in-spark-streaming.html • http://www.slideshare.net/databricks/2016-spark-summit-east- keynote-matei-zaharia 2016-04-13 Hadoop Summit 2016 (Dublin) 45
  • 46. Apache Flink • http://flink.apache.org/news/2015/12/04/Introducing- windows.html • http://data-artisans.com/flink-1-0-0/ • http://data-artisans.com/how-apache-flink-enables-new- streaming-applications/ 2016-04-13 Hadoop Summit 2016 (Dublin) 46
  • 47. Apache Storm • https://community.hortonworks.com/articles/14171/windowing- and-state-checkpointing-in-apache-storm.html • https://yahooeng.tumblr.com/post/135321837876/benchmarking -streaming-computation-engines-at 2016-04-13 Hadoop Summit 2016 (Dublin) 47
  • 48. Samza / Kafka Streams • http://docs.confluent.io/2.1.0-alpha1/streams/architecture.html • http://www.confluent.io/blog/introducing-kafka-streams-stream- processing-made-simple • http://docs.confluent.io/2.1.0-alpha1/streams/index.html • http://www.slideshare.net/edibice/extremely-low-latency-web- scale-fraud-prevention-with-apache-samza-kafka-and-friends • http://radar.oreilly.com/2014/07/why-local-state-is-a- fundamental-primitive-in-stream-processing.html 2016-04-13 Hadoop Summit 2016 (Dublin) 48
  • 49. Apache Apex • http://docs.datatorrent.com/application_development/#apache- apex-platform-overview • http://docs.datatorrent.com/application_development/#fault- tolerance • https://github.com/apache/incubator-apex- malhar/tree/master/demos • https://www.datatorrent.com/introducing-apache-apex- incubating/ 2016-04-13 Hadoop Summit 2016 (Dublin) 49
  • 50. List of Figures (in order of usage) • https://upload.wikimedia.org/wikipedia/commons/thumb/2/2a/CPT-FSM-abcd.svg/326px-CPT-FSM- abcd.svg.png • https://storm.apache.org/images/topology.png • https://databricks.com/wp-content/uploads/2015/07/image11-1024x655.png • https://databricks.com/wp-content/uploads/2015/07/image21-1024x734.png • https://people.csail.mit.edu/matei/papers/2012/hotcloud_spark_streaming.pdf, page 2. • http://www.slideshare.net/ptgoetz/storm-hadoop-summit2014, page 69-71. • http://samza.apache.org/img/0.9/learn/documentation/container/checkpointing.svg • https://databricks.com/wp-content/uploads/2015/07/image41-1024x602.png • https://storm.apache.org/documentation/images/spout-vs-state.png • http://samza.apache.org/img/0.9/learn/documentation/container/stateful_job.png 2016-04-13 Hadoop Summit 2016 (Dublin) 50