SlideShare uma empresa Scribd logo
1 de 38
© 2015 IBM Corporation0
Power of data. Simplicity of design. Speed of innovation.
IBM Spark
Comparison of various streaming analytics
technologies
Mario Briggs
Sachin Aggarwal
March 12, 2016
© 2015 IBM Corporation1
Agenda
 Streaming Analytics System Architecture
 Features needed in Streaming Analytic Applications (What and Why?)
 Apache Storm, Trident and Flink vis-à-vis Streaming features
 Spark Streaming
 Summing it all up
 How to benchmark Spark Streaming?
© 2015 IBM Corporation2
What is a Streaming Analytics Application?
Event
Stream
Collecting
/Processing
Event Result
© 2015 IBM Corporation3
Streaming System Architecture
 Continuous Operator Architecture – Static Scheduling
Source Sink
Filter, Transform,
Aggregate processing
Filter, Transform,
Aggregate processing
© 2015 IBM Corporation4
Task Scheduler Architecture
Task Set
Task Scheduler
Thread
+ Data
Task Executors
Thread
+ Data
Thread
+ Data
© 2015 IBM Corporation5
Streaming System Architecture’s – Pros & Cons
 Continuous Operator Architecture
 No Task scheduling overheads
 Task Scheduler Architecture
 Dynamic Data partitions can add more parallelism
© 2015 IBM Corporation6
Features Needed By Streaming Analytics Applications
 Fault Tolerance
 Message Processing Guarantees
 Back Pressure
 Stateful vs Stateless
 Built-in Primitives vs Roll your own
 Lambda Architecture
 Better Resource Utilization
 Tuple Level processing vs Micro-batch
© 2015 IBM Corporation7
Fault Tolerance
Typical Architecture
Master
Node
Cluster
Co-ordination
Worker
Worker
Worker
Worker
What happens when a worker
or node dies ? What if the
worker is a receiver ?
What happens master dies ?
© 2015 IBM Corporation8
Message Processing Guarantees
 At most once processing
 There could be messages lost / not processed. No message will be processed in duplicate.
 At least once processing
 Messages can be processed in duplicate.
 Exactly once processing
 No Messages lost. No duplicate processing of messages
 End-to-end message processing guarantees depends on processing guarantees of the 3 main elements
 Source: requires reliable sources.
 Processing: processing gaurantee of the streaming system.
 Sink: requires Sink with atomic-write capability
© 2015 IBM Corporation9
Back Pressure
 Processing conditions in a streaming application can change over time
 Unexpected slowdowns in the processing components
• For e.g. Data stores being written to or looked-up became slow
 Unexpected surge in rate/load of input data.
 Back pressure determines what happens when the above occurs
 Nothing special. Lose data or system becomes unstable and components start crashing.
 Make upstream components keep repeating same work?
 Or all components upstream of the slowed-down component, throttle themselves by some mechanism
© 2015 IBM Corporation10
Stateful vs Stateless, Built-in primitives vs. Roll your own
 Some of the most common logic you implement in your streaming application
Calculate and maintain aggregates over time, by a Key.
Join multiple input streams by a key. Lookup master data ‘table’
Rolling counts over time windows (by time or count). Trigger on thresholds breached condition.
 Stateful vs Stateless is linked to fault tolerance
 What happens when you maintain aggregates and the node holding that does down.
Do you have to replay from start or can continue from last saved / checkpointed.
© 2015 IBM Corporation11
Lambda Architecture
 Many systems today need to do both real time as well as historical/batch processing on the same data.
 Can you share ‘same’ implementation and logic across these two?
 If you can’t, then chances are you have 2 separate implementations of similar logic that is not in sync
 Will have different answers from realTime vs historical system
© 2015 IBM Corporation12
Better Resource Utilization & Tuple Level vs. Micro-batch
 Can your Streaming application share same clusters with other jobs via a common Resource Manager?
 Can you reserve resource at the required granular level.
© 2015 IBM Corporation13
Apache Storm – Lets understand the programming model
© 2015 IBM Corporation14
Apache Storm – Process Model
Topology : Network of spouts and Bolts
© 2015 IBM Corporation15
 True streaming i.e. message level.
 Stateless
 Cannot maintain state across message’s
 Executor failures, require replay from start to build state (fault tolerance of executor)
 No built-in primitives
 All user code (no built-in aggregates/joins/grouping)
 Message processing guarantee's
 at most once. Yes (no tracking & replay)
 at least once. Yes ( tracks every tuple and children and replays if necessary)
 exactly once. No
 Run on Shared clusters
 No. Scheduler component (Nimbus) doesn’t work with YARN/Mesos
 Hortonworks distro allows to run as Yarn application
Apache Storm - Features
© 2015 IBM Corporation16
 Fault Tolerance (Master)
 When Nimbus node dies, if a worker node dies then no reassignment of worker to other nodes happen.
 Existing workers continue to run and if failed, ‘Supervisor’ will restarted them
 No new jobs can be submitted
 Back Pressure
 No
 Unified Programming Model
 No
Apache Storm - Features
© 2015 IBM Corporation17
 Batching support for Apache Storm
 Stateful
 Can maintain state across message’s/batches
 Failure of executor, state recoverable from external stores rather than replay from start
Apache Trident
© 2015 IBM Corporation18
Apache Trident – What’s new in the program model?
© 2015 IBM Corporation19
 Built-in primitives – Yes
Aggregates/grouping/function/filters/joins
 Message processing guarantee's
 All three levels
 Exactly once (atomic txn’s to external store, using a unique batchId & gauranteed ordering of
updates among batches)
 Run on Shared clusters, Back Pressure
 No (Same as Storm)
 Fault Tolerance (Master)
 Same as Storm
Apache Trident
© 2015 IBM Corporation20
Apache Storm and Trident drawbacks
Source of Storm’s problems – Multiple topology’s tasks, run in a Single JVM process
© 2015 IBM Corporation21
Apache Storm and Trident drawbacks
 One Storm ‘Worker’ (a Single JVM process) runs too many different components and multiple
different tasks, each requiring different resources.
 Each tuple needs to pass through 4 threads in a worker
 Single global queues and log files across all tasks in a worker.
 hard to debug and size correctly. Thus have to oversize a lot, not efficient.
• E.g. oversizing memory means when doing a stack dump will cause missed heartbeats.
 Nimbus scheduler does not support resource reservation and isolation at a worker level.
 Because of above, twitter runs each topology (stream application) on dedicated cluster.
http://dl.acm.org/citation.cfm?id=2742788
© 2015 IBM Corporation22
Heron Topology
© 2015 IBM Corporation23
 True streaming
 Data Exchange buffer control
 Stateful - Yes
 Light-weight async checkpointing mechanism (Barriers)
 Built-in primitives. Yes
 Transformation, aggregations, windows, joins, connect, split
 Checkpoint local variables.
 Tumbling windows, sliding windows.
 Window triggers.
 Time windows (Event time, ingestion time, processing time support)
 Train models, update & predict.
 Message processing guarantee's
 Exactly once
 Sink exactly-once -> HDFS
Apache Flink
© 2015 IBM Corporation24
Apache Flink Process Model
© 2015 IBM Corporation25
 Fault Tolerance of Master
 Yes. (since 0.10.0. Only for YARN & standalone)
 Back Pressure
 Yes. (simple watermark for transfer buffers)
 Run on Shared clusters – Yes
 Programming Model
 Overlapping programming model (Batch does not have SQL support)
Apache Flink
© 2015 IBM Corporation26
Spark Streaming
 Micro batching
 Stateful
 Yes. (UpdateStateByKey , mapWithState functions)
 Built-in primitives – Yes
Aggregates/grouping/function/filters/joins
Sliding / Tumbling Windows
Train models & predict in a streaming app.
Event Time (slated for 2.0)
 Message processing guarantee's
 Exactly once
 Run on Shared clusters
 Yes
© 2015 IBM Corporation27
Spark Streaming
 Back Pressure
 Yes
 Fault Tolerance of Master
 Yes. (Dstream state saved to checkpoint file, used to restart master)
Programming Model
 Integrate wholly with Spark (MLlib, SparkSQL/DataFrame, RDD)
 Overlapping model for Batch and Streaming version of application
© 2015 IBM Corporation28
Feature Storm Trident Flink Spark
Streaming
Stateful No Yes (with
External
Stores)
YES YES
Message Processing
Guarantees
No Exactly
Once
Exactly Once
(with external
Stores)
Exactly Once Exactly Once
Back Pressure No No YES YES
Built-in Primitives No Yes YES YES
Overlapping
Programming Model
No No YES YES
Work with Resource
Schedulers
No No YES YES
True Streaming YES Micro-batch YES Micro-batch
Summing it all up
© 2015 IBM Corporation29
How to benchmark Spark Streaming?
© 2015 IBM Corporation30
Setup Information
load-data
Pulls data from twitter and stores as fixed dataset
push-to-kafka
Reads fixed dataset and push to Kafka at specific rate
spark-benchmarks
Reads data from Kafka and execute benchmark code
flink-benchmarks –(WIP)
Reads data from Kafka and execute benchmark code
© 2015 IBM Corporation31
Listener Interface in Spark
• StreamingListener-A listener interface for receiving information about an ongoing
streaming computation.
• Functions we need to override:-
– onReceiverStarted: Called when processing of a job of a batch has completed
– onReceiverError: Called when a receiver has reported an error
– onReceiverStopped: Called when a receiver has been stopped
– onBatchSubmitted: Called when a batch of jobs has been submitted for processing
– onBatchStarted: Called when processing of a batch of jobs has started.
– onBatchCompleted: Called when processing of a batch of jobs has completed
– onOutputOperationStarted: Called when processing of a job of a batch has started
– onOutputOperationCompleted: Called when processing of a job of a batch has completed
© 2015 IBM Corporation32
Implementation Details
Flag to check first batch
 Record startTime
For each batch
 Update totalRecords + batchCompleted.batchInfo.numRecords
 Increment batchCount
If totalRecords >= recordLimit
 Record Endtime
 avgLatency = totalDelay / totalRecords
 recordThroughput = totalRecords / totalTime
© 2015 IBM Corporation33
Observation: Execution with various batch intervals
Batch interval (ms) Batch Count Total Records Total consumed time (s)
Average Latency/batch
interval
avg delay per batch Average records/second
1000 1,056 100,000,020 1,056 1,429 429 94688.7392386
900 1,175 100,000,020 1,058 1,362 462 94541.2998858
800 1,322 100,000,020 1,058 1,291 491 94541.2998858
700 1,487 100,000,020 1,041 1,184 484 96034.5609108
600 1,770 100,000,020 1,062 1,102 502 94159.4948532
500 1,681 100,000,020 1,042 1,104 604 95943.1594458
400 1,355 100,000,020 1,033 1,202 802 96774.6187322
300 979 100,000,020 1,062 1,712 1,412 94190.3586768
200 755 100,000,020 1,092 2,212 2,012 91577.2064378
100 718 100,000,020 1,076 2,397 2,297 92963.5183339
© 2015 IBM Corporation34
Observation: Execution with various input rate
500 60 1144 1457733645809 1457733645982 1457734424894 1457734424894 100017510 779.085 1165.648763 128378.1744
500 54 1056 1457734624836 1457734624971 1457735504549 1457735504549 108014596 879.713 1378.846423 122783.9034
500 45 1552 1457735724327 1457735724487 1457736730486 1457736730486 112535122 1006.159 1130.425913 111846.2609
500 36 2290 1457736944320 1457736944450 1457738209266 1457738209266 120010057 1264.946 1007.914431 94873.66022
500 24 3600 1457738405811 1457738405990 1457740206047 1457740206047 120005209 1800.236 962.950761 66660.82058
500 12 6517 1457740349824 1457740349984 1457743608456 1457743608456 120000935 3258.632 875.2863643 36825.55594
500 6 12956 1457743741277 1457743741433 1457750219374 1457750219374 120000857 6478.097 766.6043024 18524.09079
© 2015 IBM Corporation35
© 2015 IBM Corporation36
Back up
© 2015 IBM Corporation37

Mais conteúdo relacionado

Mais procurados

Evolving from Messaging to Event Streaming
Evolving from Messaging to Event StreamingEvolving from Messaging to Event Streaming
Evolving from Messaging to Event Streamingconfluent
 
Apache Deep Learning 201 - Barcelona DWS March 2019
Apache Deep Learning 201 - Barcelona DWS March 2019Apache Deep Learning 201 - Barcelona DWS March 2019
Apache Deep Learning 201 - Barcelona DWS March 2019Timothy Spann
 
Designing For Multicloud, CF Summit Frankfurt 2016
Designing For Multicloud, CF Summit Frankfurt 2016Designing For Multicloud, CF Summit Frankfurt 2016
Designing For Multicloud, CF Summit Frankfurt 2016Mark D'Cunha
 
Real-World Pulsar Architectural Patterns
Real-World Pulsar Architectural PatternsReal-World Pulsar Architectural Patterns
Real-World Pulsar Architectural PatternsDevin Bost
 
Serverless machine learning architectures at Helixa
Serverless machine learning architectures at HelixaServerless machine learning architectures at Helixa
Serverless machine learning architectures at HelixaData Science Milan
 
Best Practices for ETL with Apache NiFi on Kubernetes - Albert Lewandowski, G...
Best Practices for ETL with Apache NiFi on Kubernetes - Albert Lewandowski, G...Best Practices for ETL with Apache NiFi on Kubernetes - Albert Lewandowski, G...
Best Practices for ETL with Apache NiFi on Kubernetes - Albert Lewandowski, G...GetInData
 
Cloud Operations with Streaming Analytics using Apache NiFi and Apache Flink
Cloud Operations with Streaming Analytics using Apache NiFi and Apache FlinkCloud Operations with Streaming Analytics using Apache NiFi and Apache Flink
Cloud Operations with Streaming Analytics using Apache NiFi and Apache FlinkDataWorks Summit
 
Cloudstate - Towards Stateful Serverless
Cloudstate - Towards Stateful ServerlessCloudstate - Towards Stateful Serverless
Cloudstate - Towards Stateful ServerlessLightbend
 
A Journey to Reactive Function Programming
A Journey to Reactive Function ProgrammingA Journey to Reactive Function Programming
A Journey to Reactive Function ProgrammingAhmed Soliman
 
The Event Mesh: real-time, event-driven, responsive APIs and beyond
The Event Mesh: real-time, event-driven, responsive APIs and beyondThe Event Mesh: real-time, event-driven, responsive APIs and beyond
The Event Mesh: real-time, event-driven, responsive APIs and beyondSolace
 
AWS re:Invent 2016: Learn how IFTTT uses ElastiCache for Redis to predict eve...
AWS re:Invent 2016: Learn how IFTTT uses ElastiCache for Redis to predict eve...AWS re:Invent 2016: Learn how IFTTT uses ElastiCache for Redis to predict eve...
AWS re:Invent 2016: Learn how IFTTT uses ElastiCache for Redis to predict eve...Amazon Web Services
 
Why Cloud-Native Kafka Matters: 4 Reasons to Stop Managing it Yourself
Why Cloud-Native Kafka Matters: 4 Reasons to Stop Managing it YourselfWhy Cloud-Native Kafka Matters: 4 Reasons to Stop Managing it Yourself
Why Cloud-Native Kafka Matters: 4 Reasons to Stop Managing it YourselfDATAVERSITY
 
Ten reasons to choose Apache Pulsar over Apache Kafka for Event Sourcing_Robe...
Ten reasons to choose Apache Pulsar over Apache Kafka for Event Sourcing_Robe...Ten reasons to choose Apache Pulsar over Apache Kafka for Event Sourcing_Robe...
Ten reasons to choose Apache Pulsar over Apache Kafka for Event Sourcing_Robe...StreamNative
 
AWS User Group Meetup Berlin - Kay Lerch on Apache NiFi (2016-04-19)
AWS User Group Meetup Berlin - Kay Lerch on Apache NiFi (2016-04-19)AWS User Group Meetup Berlin - Kay Lerch on Apache NiFi (2016-04-19)
AWS User Group Meetup Berlin - Kay Lerch on Apache NiFi (2016-04-19)Kay Lerch
 
Maximilian Michels - Flink and Beam
Maximilian Michels - Flink and BeamMaximilian Michels - Flink and Beam
Maximilian Michels - Flink and BeamFlink Forward
 
Streaming Sensor Data Slides_Virender
Streaming Sensor Data Slides_VirenderStreaming Sensor Data Slides_Virender
Streaming Sensor Data Slides_Virendervithakur
 
Event streaming: A paradigm shift in enterprise software architecture
Event streaming: A paradigm shift in enterprise software architectureEvent streaming: A paradigm shift in enterprise software architecture
Event streaming: A paradigm shift in enterprise software architectureSina Sojoodi
 
IoT Sensor Analytics with Kafka, ksqlDB and TensorFlow
IoT Sensor Analytics with Kafka, ksqlDB and TensorFlowIoT Sensor Analytics with Kafka, ksqlDB and TensorFlow
IoT Sensor Analytics with Kafka, ksqlDB and TensorFlowKai Wähner
 
REST - Why, When and How? at AMIS25
REST - Why, When and How? at AMIS25REST - Why, When and How? at AMIS25
REST - Why, When and How? at AMIS25Jon Petter Hjulstad
 
Lessons from Building Large-Scale, Multi-Cloud, SaaS Software at Databricks
Lessons from Building Large-Scale, Multi-Cloud, SaaS Software at DatabricksLessons from Building Large-Scale, Multi-Cloud, SaaS Software at Databricks
Lessons from Building Large-Scale, Multi-Cloud, SaaS Software at DatabricksDatabricks
 

Mais procurados (20)

Evolving from Messaging to Event Streaming
Evolving from Messaging to Event StreamingEvolving from Messaging to Event Streaming
Evolving from Messaging to Event Streaming
 
Apache Deep Learning 201 - Barcelona DWS March 2019
Apache Deep Learning 201 - Barcelona DWS March 2019Apache Deep Learning 201 - Barcelona DWS March 2019
Apache Deep Learning 201 - Barcelona DWS March 2019
 
Designing For Multicloud, CF Summit Frankfurt 2016
Designing For Multicloud, CF Summit Frankfurt 2016Designing For Multicloud, CF Summit Frankfurt 2016
Designing For Multicloud, CF Summit Frankfurt 2016
 
Real-World Pulsar Architectural Patterns
Real-World Pulsar Architectural PatternsReal-World Pulsar Architectural Patterns
Real-World Pulsar Architectural Patterns
 
Serverless machine learning architectures at Helixa
Serverless machine learning architectures at HelixaServerless machine learning architectures at Helixa
Serverless machine learning architectures at Helixa
 
Best Practices for ETL with Apache NiFi on Kubernetes - Albert Lewandowski, G...
Best Practices for ETL with Apache NiFi on Kubernetes - Albert Lewandowski, G...Best Practices for ETL with Apache NiFi on Kubernetes - Albert Lewandowski, G...
Best Practices for ETL with Apache NiFi on Kubernetes - Albert Lewandowski, G...
 
Cloud Operations with Streaming Analytics using Apache NiFi and Apache Flink
Cloud Operations with Streaming Analytics using Apache NiFi and Apache FlinkCloud Operations with Streaming Analytics using Apache NiFi and Apache Flink
Cloud Operations with Streaming Analytics using Apache NiFi and Apache Flink
 
Cloudstate - Towards Stateful Serverless
Cloudstate - Towards Stateful ServerlessCloudstate - Towards Stateful Serverless
Cloudstate - Towards Stateful Serverless
 
A Journey to Reactive Function Programming
A Journey to Reactive Function ProgrammingA Journey to Reactive Function Programming
A Journey to Reactive Function Programming
 
The Event Mesh: real-time, event-driven, responsive APIs and beyond
The Event Mesh: real-time, event-driven, responsive APIs and beyondThe Event Mesh: real-time, event-driven, responsive APIs and beyond
The Event Mesh: real-time, event-driven, responsive APIs and beyond
 
AWS re:Invent 2016: Learn how IFTTT uses ElastiCache for Redis to predict eve...
AWS re:Invent 2016: Learn how IFTTT uses ElastiCache for Redis to predict eve...AWS re:Invent 2016: Learn how IFTTT uses ElastiCache for Redis to predict eve...
AWS re:Invent 2016: Learn how IFTTT uses ElastiCache for Redis to predict eve...
 
Why Cloud-Native Kafka Matters: 4 Reasons to Stop Managing it Yourself
Why Cloud-Native Kafka Matters: 4 Reasons to Stop Managing it YourselfWhy Cloud-Native Kafka Matters: 4 Reasons to Stop Managing it Yourself
Why Cloud-Native Kafka Matters: 4 Reasons to Stop Managing it Yourself
 
Ten reasons to choose Apache Pulsar over Apache Kafka for Event Sourcing_Robe...
Ten reasons to choose Apache Pulsar over Apache Kafka for Event Sourcing_Robe...Ten reasons to choose Apache Pulsar over Apache Kafka for Event Sourcing_Robe...
Ten reasons to choose Apache Pulsar over Apache Kafka for Event Sourcing_Robe...
 
AWS User Group Meetup Berlin - Kay Lerch on Apache NiFi (2016-04-19)
AWS User Group Meetup Berlin - Kay Lerch on Apache NiFi (2016-04-19)AWS User Group Meetup Berlin - Kay Lerch on Apache NiFi (2016-04-19)
AWS User Group Meetup Berlin - Kay Lerch on Apache NiFi (2016-04-19)
 
Maximilian Michels - Flink and Beam
Maximilian Michels - Flink and BeamMaximilian Michels - Flink and Beam
Maximilian Michels - Flink and Beam
 
Streaming Sensor Data Slides_Virender
Streaming Sensor Data Slides_VirenderStreaming Sensor Data Slides_Virender
Streaming Sensor Data Slides_Virender
 
Event streaming: A paradigm shift in enterprise software architecture
Event streaming: A paradigm shift in enterprise software architectureEvent streaming: A paradigm shift in enterprise software architecture
Event streaming: A paradigm shift in enterprise software architecture
 
IoT Sensor Analytics with Kafka, ksqlDB and TensorFlow
IoT Sensor Analytics with Kafka, ksqlDB and TensorFlowIoT Sensor Analytics with Kafka, ksqlDB and TensorFlow
IoT Sensor Analytics with Kafka, ksqlDB and TensorFlow
 
REST - Why, When and How? at AMIS25
REST - Why, When and How? at AMIS25REST - Why, When and How? at AMIS25
REST - Why, When and How? at AMIS25
 
Lessons from Building Large-Scale, Multi-Cloud, SaaS Software at Databricks
Lessons from Building Large-Scale, Multi-Cloud, SaaS Software at DatabricksLessons from Building Large-Scale, Multi-Cloud, SaaS Software at Databricks
Lessons from Building Large-Scale, Multi-Cloud, SaaS Software at Databricks
 

Destaque

Real-Time Event & Stream Processing on MS Azure
Real-Time Event & Stream Processing on MS AzureReal-Time Event & Stream Processing on MS Azure
Real-Time Event & Stream Processing on MS AzureKhalid Salama
 
IoT Innovation Lab Berlin @relayr - Kay Lerch on Getting basics right for you...
IoT Innovation Lab Berlin @relayr - Kay Lerch on Getting basics right for you...IoT Innovation Lab Berlin @relayr - Kay Lerch on Getting basics right for you...
IoT Innovation Lab Berlin @relayr - Kay Lerch on Getting basics right for you...Kay Lerch
 
Dataflow with Apache NiFi - Apache NiFi Meetup - 2016 Hadoop Summit - San Jose
Dataflow with Apache NiFi - Apache NiFi Meetup - 2016 Hadoop Summit - San JoseDataflow with Apache NiFi - Apache NiFi Meetup - 2016 Hadoop Summit - San Jose
Dataflow with Apache NiFi - Apache NiFi Meetup - 2016 Hadoop Summit - San JoseAldrin Piri
 
Big Data Day LA 2015 - Always-on Ingestion for Data at Scale by Arvind Prabha...
Big Data Day LA 2015 - Always-on Ingestion for Data at Scale by Arvind Prabha...Big Data Day LA 2015 - Always-on Ingestion for Data at Scale by Arvind Prabha...
Big Data Day LA 2015 - Always-on Ingestion for Data at Scale by Arvind Prabha...Data Con LA
 
Developing Connected Applications with AWS IoT - Technical 301
Developing Connected Applications with AWS IoT - Technical 301Developing Connected Applications with AWS IoT - Technical 301
Developing Connected Applications with AWS IoT - Technical 301Amazon Web Services
 
Lightbend Fast Data Platform
Lightbend Fast Data PlatformLightbend Fast Data Platform
Lightbend Fast Data PlatformLightbend
 
How to Build Continuous Ingestion for the Internet of Things
How to Build Continuous Ingestion for the Internet of ThingsHow to Build Continuous Ingestion for the Internet of Things
How to Build Continuous Ingestion for the Internet of ThingsCloudera, Inc.
 
Study: The Future of VR, AR and Self-Driving Cars
Study: The Future of VR, AR and Self-Driving CarsStudy: The Future of VR, AR and Self-Driving Cars
Study: The Future of VR, AR and Self-Driving CarsLinkedIn
 
Getting started with Azure Event Hubs and Stream Analytics services
Getting started with Azure Event Hubs and Stream Analytics servicesGetting started with Azure Event Hubs and Stream Analytics services
Getting started with Azure Event Hubs and Stream Analytics servicesVladimir Bychkov
 
London Apache Kafka Meetup (Jan 2017)
London Apache Kafka Meetup (Jan 2017)London Apache Kafka Meetup (Jan 2017)
London Apache Kafka Meetup (Jan 2017)Landoop Ltd
 
Not Only Streams for Akademia JLabs
Not Only Streams for Akademia JLabsNot Only Streams for Akademia JLabs
Not Only Streams for Akademia JLabsKonrad Malawski
 
UX, ethnography and possibilities: for Libraries, Museums and Archives
UX, ethnography and possibilities: for Libraries, Museums and ArchivesUX, ethnography and possibilities: for Libraries, Museums and Archives
UX, ethnography and possibilities: for Libraries, Museums and ArchivesNed Potter
 
Processing IoT Data with Apache Kafka
Processing IoT Data with Apache KafkaProcessing IoT Data with Apache Kafka
Processing IoT Data with Apache KafkaMatthew Howlett
 
Designing Teams for Emerging Challenges
Designing Teams for Emerging ChallengesDesigning Teams for Emerging Challenges
Designing Teams for Emerging ChallengesAaron Irizarry
 
IoT Connected Brewery
IoT Connected BreweryIoT Connected Brewery
IoT Connected BreweryJason Hubbard
 
Fusion Middleware Live Application Development Demo Oracle Open World 2012
Fusion Middleware Live Application Development Demo Oracle Open World 2012Fusion Middleware Live Application Development Demo Oracle Open World 2012
Fusion Middleware Live Application Development Demo Oracle Open World 2012Lucas Jellema
 
Five Cool Use Cases for the Spring Component of the SOA Suite 11g
Five Cool Use Cases for the Spring Component of the SOA Suite 11gFive Cool Use Cases for the Spring Component of the SOA Suite 11g
Five Cool Use Cases for the Spring Component of the SOA Suite 11gGuido Schmutz
 

Destaque (20)

Real-Time Event & Stream Processing on MS Azure
Real-Time Event & Stream Processing on MS AzureReal-Time Event & Stream Processing on MS Azure
Real-Time Event & Stream Processing on MS Azure
 
IoT Innovation Lab Berlin @relayr - Kay Lerch on Getting basics right for you...
IoT Innovation Lab Berlin @relayr - Kay Lerch on Getting basics right for you...IoT Innovation Lab Berlin @relayr - Kay Lerch on Getting basics right for you...
IoT Innovation Lab Berlin @relayr - Kay Lerch on Getting basics right for you...
 
Dataflow with Apache NiFi - Apache NiFi Meetup - 2016 Hadoop Summit - San Jose
Dataflow with Apache NiFi - Apache NiFi Meetup - 2016 Hadoop Summit - San JoseDataflow with Apache NiFi - Apache NiFi Meetup - 2016 Hadoop Summit - San Jose
Dataflow with Apache NiFi - Apache NiFi Meetup - 2016 Hadoop Summit - San Jose
 
Big Data Day LA 2015 - Always-on Ingestion for Data at Scale by Arvind Prabha...
Big Data Day LA 2015 - Always-on Ingestion for Data at Scale by Arvind Prabha...Big Data Day LA 2015 - Always-on Ingestion for Data at Scale by Arvind Prabha...
Big Data Day LA 2015 - Always-on Ingestion for Data at Scale by Arvind Prabha...
 
Developing Connected Applications with AWS IoT - Technical 301
Developing Connected Applications with AWS IoT - Technical 301Developing Connected Applications with AWS IoT - Technical 301
Developing Connected Applications with AWS IoT - Technical 301
 
Lightbend Fast Data Platform
Lightbend Fast Data PlatformLightbend Fast Data Platform
Lightbend Fast Data Platform
 
How to Build Continuous Ingestion for the Internet of Things
How to Build Continuous Ingestion for the Internet of ThingsHow to Build Continuous Ingestion for the Internet of Things
How to Build Continuous Ingestion for the Internet of Things
 
Study: The Future of VR, AR and Self-Driving Cars
Study: The Future of VR, AR and Self-Driving CarsStudy: The Future of VR, AR and Self-Driving Cars
Study: The Future of VR, AR and Self-Driving Cars
 
Getting started with Azure Event Hubs and Stream Analytics services
Getting started with Azure Event Hubs and Stream Analytics servicesGetting started with Azure Event Hubs and Stream Analytics services
Getting started with Azure Event Hubs and Stream Analytics services
 
Blr hadoop meetup
Blr hadoop meetupBlr hadoop meetup
Blr hadoop meetup
 
Storm over gearpump
Storm over gearpumpStorm over gearpump
Storm over gearpump
 
London Apache Kafka Meetup (Jan 2017)
London Apache Kafka Meetup (Jan 2017)London Apache Kafka Meetup (Jan 2017)
London Apache Kafka Meetup (Jan 2017)
 
Kafka connect
Kafka connectKafka connect
Kafka connect
 
Not Only Streams for Akademia JLabs
Not Only Streams for Akademia JLabsNot Only Streams for Akademia JLabs
Not Only Streams for Akademia JLabs
 
UX, ethnography and possibilities: for Libraries, Museums and Archives
UX, ethnography and possibilities: for Libraries, Museums and ArchivesUX, ethnography and possibilities: for Libraries, Museums and Archives
UX, ethnography and possibilities: for Libraries, Museums and Archives
 
Processing IoT Data with Apache Kafka
Processing IoT Data with Apache KafkaProcessing IoT Data with Apache Kafka
Processing IoT Data with Apache Kafka
 
Designing Teams for Emerging Challenges
Designing Teams for Emerging ChallengesDesigning Teams for Emerging Challenges
Designing Teams for Emerging Challenges
 
IoT Connected Brewery
IoT Connected BreweryIoT Connected Brewery
IoT Connected Brewery
 
Fusion Middleware Live Application Development Demo Oracle Open World 2012
Fusion Middleware Live Application Development Demo Oracle Open World 2012Fusion Middleware Live Application Development Demo Oracle Open World 2012
Fusion Middleware Live Application Development Demo Oracle Open World 2012
 
Five Cool Use Cases for the Spring Component of the SOA Suite 11g
Five Cool Use Cases for the Spring Component of the SOA Suite 11gFive Cool Use Cases for the Spring Component of the SOA Suite 11g
Five Cool Use Cases for the Spring Component of the SOA Suite 11g
 

Semelhante a Comparison of various streaming technologies

Architecting and Tuning IIB/eXtreme Scale for Maximum Performance and Reliabi...
Architecting and Tuning IIB/eXtreme Scale for Maximum Performance and Reliabi...Architecting and Tuning IIB/eXtreme Scale for Maximum Performance and Reliabi...
Architecting and Tuning IIB/eXtreme Scale for Maximum Performance and Reliabi...Prolifics
 
Introduction to Structured Streaming
Introduction to Structured StreamingIntroduction to Structured Streaming
Introduction to Structured Streamingdatamantra
 
Natural Laws of Software Performance
Natural Laws of Software PerformanceNatural Laws of Software Performance
Natural Laws of Software PerformanceGibraltar Software
 
Five cool ways the JVM can run Apache Spark faster
Five cool ways the JVM can run Apache Spark fasterFive cool ways the JVM can run Apache Spark faster
Five cool ways the JVM can run Apache Spark fasterTim Ellison
 
Jstorm introduction-0.9.6
Jstorm introduction-0.9.6Jstorm introduction-0.9.6
Jstorm introduction-0.9.6longda feng
 
Apache Storm and Oracle Event Processing for Real-time Analytics
Apache Storm and Oracle Event Processing for Real-time AnalyticsApache Storm and Oracle Event Processing for Real-time Analytics
Apache Storm and Oracle Event Processing for Real-time AnalyticsPrabhu Thukkaram
 
Spring boot
Spring bootSpring boot
Spring bootsdeeg
 
NTTs Journey with Openstack-final
NTTs Journey with Openstack-finalNTTs Journey with Openstack-final
NTTs Journey with Openstack-finalshintaro mizuno
 
Why Cloud Management Makes Sense - Josh Fraser
Why Cloud Management Makes Sense - Josh FraserWhy Cloud Management Makes Sense - Josh Fraser
Why Cloud Management Makes Sense - Josh FraserAmazon Web Services
 
Azure + DataStax Enterprise (DSE) Powers Office365 Per User Store
Azure + DataStax Enterprise (DSE) Powers Office365 Per User StoreAzure + DataStax Enterprise (DSE) Powers Office365 Per User Store
Azure + DataStax Enterprise (DSE) Powers Office365 Per User StoreDataStax Academy
 
Real World Problem Solving Using Application Performance Management 10
Real World Problem Solving Using Application Performance Management 10Real World Problem Solving Using Application Performance Management 10
Real World Problem Solving Using Application Performance Management 10CA Technologies
 
Voldemort & Hadoop @ Linkedin, Hadoop User Group Jan 2010
Voldemort & Hadoop @ Linkedin, Hadoop User Group Jan 2010Voldemort & Hadoop @ Linkedin, Hadoop User Group Jan 2010
Voldemort & Hadoop @ Linkedin, Hadoop User Group Jan 2010Bhupesh Bansal
 
Hadoop and Voldemort @ LinkedIn
Hadoop and Voldemort @ LinkedInHadoop and Voldemort @ LinkedIn
Hadoop and Voldemort @ LinkedInHadoop User Group
 
Apache Storm vs. Spark Streaming - two stream processing platforms compared
Apache Storm vs. Spark Streaming - two stream processing platforms comparedApache Storm vs. Spark Streaming - two stream processing platforms compared
Apache Storm vs. Spark Streaming - two stream processing platforms comparedGuido Schmutz
 
How to Improve Performance Testing Using InfluxDB and Apache JMeter
How to Improve Performance Testing Using InfluxDB and Apache JMeterHow to Improve Performance Testing Using InfluxDB and Apache JMeter
How to Improve Performance Testing Using InfluxDB and Apache JMeterInfluxData
 
When Web Services Go Bad
When Web Services Go BadWhen Web Services Go Bad
When Web Services Go BadSteve Loughran
 
Continuent Tungsten - Scalable Saa S Data Management
Continuent Tungsten - Scalable Saa S Data ManagementContinuent Tungsten - Scalable Saa S Data Management
Continuent Tungsten - Scalable Saa S Data Managementguest2e11e8
 
Pure Systems Patterns of Expertise - John Kaemmerer and Gerry Kovan, 11th Sep...
Pure Systems Patterns of Expertise - John Kaemmerer and Gerry Kovan, 11th Sep...Pure Systems Patterns of Expertise - John Kaemmerer and Gerry Kovan, 11th Sep...
Pure Systems Patterns of Expertise - John Kaemmerer and Gerry Kovan, 11th Sep...IBM Systems UKI
 
Flink Forward San Francisco 2019: Towards Flink 2.0: Rethinking the stack and...
Flink Forward San Francisco 2019: Towards Flink 2.0: Rethinking the stack and...Flink Forward San Francisco 2019: Towards Flink 2.0: Rethinking the stack and...
Flink Forward San Francisco 2019: Towards Flink 2.0: Rethinking the stack and...Flink Forward
 

Semelhante a Comparison of various streaming technologies (20)

Architecting and Tuning IIB/eXtreme Scale for Maximum Performance and Reliabi...
Architecting and Tuning IIB/eXtreme Scale for Maximum Performance and Reliabi...Architecting and Tuning IIB/eXtreme Scale for Maximum Performance and Reliabi...
Architecting and Tuning IIB/eXtreme Scale for Maximum Performance and Reliabi...
 
Introduction to Structured Streaming
Introduction to Structured StreamingIntroduction to Structured Streaming
Introduction to Structured Streaming
 
Natural Laws of Software Performance
Natural Laws of Software PerformanceNatural Laws of Software Performance
Natural Laws of Software Performance
 
Five cool ways the JVM can run Apache Spark faster
Five cool ways the JVM can run Apache Spark fasterFive cool ways the JVM can run Apache Spark faster
Five cool ways the JVM can run Apache Spark faster
 
Jstorm introduction-0.9.6
Jstorm introduction-0.9.6Jstorm introduction-0.9.6
Jstorm introduction-0.9.6
 
Apache Storm and Oracle Event Processing for Real-time Analytics
Apache Storm and Oracle Event Processing for Real-time AnalyticsApache Storm and Oracle Event Processing for Real-time Analytics
Apache Storm and Oracle Event Processing for Real-time Analytics
 
Spring boot
Spring bootSpring boot
Spring boot
 
NTTs Journey with Openstack-final
NTTs Journey with Openstack-finalNTTs Journey with Openstack-final
NTTs Journey with Openstack-final
 
Why Cloud Management Makes Sense - Josh Fraser
Why Cloud Management Makes Sense - Josh FraserWhy Cloud Management Makes Sense - Josh Fraser
Why Cloud Management Makes Sense - Josh Fraser
 
Azure + DataStax Enterprise (DSE) Powers Office365 Per User Store
Azure + DataStax Enterprise (DSE) Powers Office365 Per User StoreAzure + DataStax Enterprise (DSE) Powers Office365 Per User Store
Azure + DataStax Enterprise (DSE) Powers Office365 Per User Store
 
Real World Problem Solving Using Application Performance Management 10
Real World Problem Solving Using Application Performance Management 10Real World Problem Solving Using Application Performance Management 10
Real World Problem Solving Using Application Performance Management 10
 
Voldemort & Hadoop @ Linkedin, Hadoop User Group Jan 2010
Voldemort & Hadoop @ Linkedin, Hadoop User Group Jan 2010Voldemort & Hadoop @ Linkedin, Hadoop User Group Jan 2010
Voldemort & Hadoop @ Linkedin, Hadoop User Group Jan 2010
 
Hadoop and Voldemort @ LinkedIn
Hadoop and Voldemort @ LinkedInHadoop and Voldemort @ LinkedIn
Hadoop and Voldemort @ LinkedIn
 
Apache Storm vs. Spark Streaming - two stream processing platforms compared
Apache Storm vs. Spark Streaming - two stream processing platforms comparedApache Storm vs. Spark Streaming - two stream processing platforms compared
Apache Storm vs. Spark Streaming - two stream processing platforms compared
 
How to Improve Performance Testing Using InfluxDB and Apache JMeter
How to Improve Performance Testing Using InfluxDB and Apache JMeterHow to Improve Performance Testing Using InfluxDB and Apache JMeter
How to Improve Performance Testing Using InfluxDB and Apache JMeter
 
When Web Services Go Bad
When Web Services Go BadWhen Web Services Go Bad
When Web Services Go Bad
 
Continuent Tungsten - Scalable Saa S Data Management
Continuent Tungsten - Scalable Saa S Data ManagementContinuent Tungsten - Scalable Saa S Data Management
Continuent Tungsten - Scalable Saa S Data Management
 
Pure Systems Patterns of Expertise - John Kaemmerer and Gerry Kovan, 11th Sep...
Pure Systems Patterns of Expertise - John Kaemmerer and Gerry Kovan, 11th Sep...Pure Systems Patterns of Expertise - John Kaemmerer and Gerry Kovan, 11th Sep...
Pure Systems Patterns of Expertise - John Kaemmerer and Gerry Kovan, 11th Sep...
 
Why meteor
Why meteorWhy meteor
Why meteor
 
Flink Forward San Francisco 2019: Towards Flink 2.0: Rethinking the stack and...
Flink Forward San Francisco 2019: Towards Flink 2.0: Rethinking the stack and...Flink Forward San Francisco 2019: Towards Flink 2.0: Rethinking the stack and...
Flink Forward San Francisco 2019: Towards Flink 2.0: Rethinking the stack and...
 

Último

MANUFACTURING PROCESS-II UNIT-5 NC MACHINE TOOLS
MANUFACTURING PROCESS-II UNIT-5 NC MACHINE TOOLSMANUFACTURING PROCESS-II UNIT-5 NC MACHINE TOOLS
MANUFACTURING PROCESS-II UNIT-5 NC MACHINE TOOLSSIVASHANKAR N
 
High Profile Call Girls Nagpur Isha Call 7001035870 Meet With Nagpur Escorts
High Profile Call Girls Nagpur Isha Call 7001035870 Meet With Nagpur EscortsHigh Profile Call Girls Nagpur Isha Call 7001035870 Meet With Nagpur Escorts
High Profile Call Girls Nagpur Isha Call 7001035870 Meet With Nagpur Escortsranjana rawat
 
Java Programming :Event Handling(Types of Events)
Java Programming :Event Handling(Types of Events)Java Programming :Event Handling(Types of Events)
Java Programming :Event Handling(Types of Events)simmis5
 
Introduction and different types of Ethernet.pptx
Introduction and different types of Ethernet.pptxIntroduction and different types of Ethernet.pptx
Introduction and different types of Ethernet.pptxupamatechverse
 
College Call Girls Nashik Nehal 7001305949 Independent Escort Service Nashik
College Call Girls Nashik Nehal 7001305949 Independent Escort Service NashikCollege Call Girls Nashik Nehal 7001305949 Independent Escort Service Nashik
College Call Girls Nashik Nehal 7001305949 Independent Escort Service NashikCall Girls in Nagpur High Profile
 
Call for Papers - African Journal of Biological Sciences, E-ISSN: 2663-2187, ...
Call for Papers - African Journal of Biological Sciences, E-ISSN: 2663-2187, ...Call for Papers - African Journal of Biological Sciences, E-ISSN: 2663-2187, ...
Call for Papers - African Journal of Biological Sciences, E-ISSN: 2663-2187, ...Christo Ananth
 
Booking open Available Pune Call Girls Koregaon Park 6297143586 Call Hot Ind...
Booking open Available Pune Call Girls Koregaon Park  6297143586 Call Hot Ind...Booking open Available Pune Call Girls Koregaon Park  6297143586 Call Hot Ind...
Booking open Available Pune Call Girls Koregaon Park 6297143586 Call Hot Ind...Call Girls in Nagpur High Profile
 
UNIT - IV - Air Compressors and its Performance
UNIT - IV - Air Compressors and its PerformanceUNIT - IV - Air Compressors and its Performance
UNIT - IV - Air Compressors and its Performancesivaprakash250
 
UNIT-III FMM. DIMENSIONAL ANALYSIS
UNIT-III FMM.        DIMENSIONAL ANALYSISUNIT-III FMM.        DIMENSIONAL ANALYSIS
UNIT-III FMM. DIMENSIONAL ANALYSISrknatarajan
 
result management system report for college project
result management system report for college projectresult management system report for college project
result management system report for college projectTonystark477637
 
Glass Ceramics: Processing and Properties
Glass Ceramics: Processing and PropertiesGlass Ceramics: Processing and Properties
Glass Ceramics: Processing and PropertiesPrabhanshu Chaturvedi
 
(PRIYA) Rajgurunagar Call Girls Just Call 7001035870 [ Cash on Delivery ] Pun...
(PRIYA) Rajgurunagar Call Girls Just Call 7001035870 [ Cash on Delivery ] Pun...(PRIYA) Rajgurunagar Call Girls Just Call 7001035870 [ Cash on Delivery ] Pun...
(PRIYA) Rajgurunagar Call Girls Just Call 7001035870 [ Cash on Delivery ] Pun...ranjana rawat
 
Online banking management system project.pdf
Online banking management system project.pdfOnline banking management system project.pdf
Online banking management system project.pdfKamal Acharya
 
Top Rated Pune Call Girls Budhwar Peth ⟟ 6297143586 ⟟ Call Me For Genuine Se...
Top Rated  Pune Call Girls Budhwar Peth ⟟ 6297143586 ⟟ Call Me For Genuine Se...Top Rated  Pune Call Girls Budhwar Peth ⟟ 6297143586 ⟟ Call Me For Genuine Se...
Top Rated Pune Call Girls Budhwar Peth ⟟ 6297143586 ⟟ Call Me For Genuine Se...Call Girls in Nagpur High Profile
 
BSides Seattle 2024 - Stopping Ethan Hunt From Taking Your Data.pptx
BSides Seattle 2024 - Stopping Ethan Hunt From Taking Your Data.pptxBSides Seattle 2024 - Stopping Ethan Hunt From Taking Your Data.pptx
BSides Seattle 2024 - Stopping Ethan Hunt From Taking Your Data.pptxfenichawla
 
MANUFACTURING PROCESS-II UNIT-1 THEORY OF METAL CUTTING
MANUFACTURING PROCESS-II UNIT-1 THEORY OF METAL CUTTINGMANUFACTURING PROCESS-II UNIT-1 THEORY OF METAL CUTTING
MANUFACTURING PROCESS-II UNIT-1 THEORY OF METAL CUTTINGSIVASHANKAR N
 
Call Girls in Nagpur Suman Call 7001035870 Meet With Nagpur Escorts
Call Girls in Nagpur Suman Call 7001035870 Meet With Nagpur EscortsCall Girls in Nagpur Suman Call 7001035870 Meet With Nagpur Escorts
Call Girls in Nagpur Suman Call 7001035870 Meet With Nagpur EscortsCall Girls in Nagpur High Profile
 

Último (20)

MANUFACTURING PROCESS-II UNIT-5 NC MACHINE TOOLS
MANUFACTURING PROCESS-II UNIT-5 NC MACHINE TOOLSMANUFACTURING PROCESS-II UNIT-5 NC MACHINE TOOLS
MANUFACTURING PROCESS-II UNIT-5 NC MACHINE TOOLS
 
High Profile Call Girls Nagpur Isha Call 7001035870 Meet With Nagpur Escorts
High Profile Call Girls Nagpur Isha Call 7001035870 Meet With Nagpur EscortsHigh Profile Call Girls Nagpur Isha Call 7001035870 Meet With Nagpur Escorts
High Profile Call Girls Nagpur Isha Call 7001035870 Meet With Nagpur Escorts
 
Java Programming :Event Handling(Types of Events)
Java Programming :Event Handling(Types of Events)Java Programming :Event Handling(Types of Events)
Java Programming :Event Handling(Types of Events)
 
Introduction and different types of Ethernet.pptx
Introduction and different types of Ethernet.pptxIntroduction and different types of Ethernet.pptx
Introduction and different types of Ethernet.pptx
 
College Call Girls Nashik Nehal 7001305949 Independent Escort Service Nashik
College Call Girls Nashik Nehal 7001305949 Independent Escort Service NashikCollege Call Girls Nashik Nehal 7001305949 Independent Escort Service Nashik
College Call Girls Nashik Nehal 7001305949 Independent Escort Service Nashik
 
Call for Papers - African Journal of Biological Sciences, E-ISSN: 2663-2187, ...
Call for Papers - African Journal of Biological Sciences, E-ISSN: 2663-2187, ...Call for Papers - African Journal of Biological Sciences, E-ISSN: 2663-2187, ...
Call for Papers - African Journal of Biological Sciences, E-ISSN: 2663-2187, ...
 
Water Industry Process Automation & Control Monthly - April 2024
Water Industry Process Automation & Control Monthly - April 2024Water Industry Process Automation & Control Monthly - April 2024
Water Industry Process Automation & Control Monthly - April 2024
 
Booking open Available Pune Call Girls Koregaon Park 6297143586 Call Hot Ind...
Booking open Available Pune Call Girls Koregaon Park  6297143586 Call Hot Ind...Booking open Available Pune Call Girls Koregaon Park  6297143586 Call Hot Ind...
Booking open Available Pune Call Girls Koregaon Park 6297143586 Call Hot Ind...
 
UNIT - IV - Air Compressors and its Performance
UNIT - IV - Air Compressors and its PerformanceUNIT - IV - Air Compressors and its Performance
UNIT - IV - Air Compressors and its Performance
 
UNIT-III FMM. DIMENSIONAL ANALYSIS
UNIT-III FMM.        DIMENSIONAL ANALYSISUNIT-III FMM.        DIMENSIONAL ANALYSIS
UNIT-III FMM. DIMENSIONAL ANALYSIS
 
result management system report for college project
result management system report for college projectresult management system report for college project
result management system report for college project
 
(INDIRA) Call Girl Aurangabad Call Now 8617697112 Aurangabad Escorts 24x7
(INDIRA) Call Girl Aurangabad Call Now 8617697112 Aurangabad Escorts 24x7(INDIRA) Call Girl Aurangabad Call Now 8617697112 Aurangabad Escorts 24x7
(INDIRA) Call Girl Aurangabad Call Now 8617697112 Aurangabad Escorts 24x7
 
Glass Ceramics: Processing and Properties
Glass Ceramics: Processing and PropertiesGlass Ceramics: Processing and Properties
Glass Ceramics: Processing and Properties
 
(PRIYA) Rajgurunagar Call Girls Just Call 7001035870 [ Cash on Delivery ] Pun...
(PRIYA) Rajgurunagar Call Girls Just Call 7001035870 [ Cash on Delivery ] Pun...(PRIYA) Rajgurunagar Call Girls Just Call 7001035870 [ Cash on Delivery ] Pun...
(PRIYA) Rajgurunagar Call Girls Just Call 7001035870 [ Cash on Delivery ] Pun...
 
Online banking management system project.pdf
Online banking management system project.pdfOnline banking management system project.pdf
Online banking management system project.pdf
 
Roadmap to Membership of RICS - Pathways and Routes
Roadmap to Membership of RICS - Pathways and RoutesRoadmap to Membership of RICS - Pathways and Routes
Roadmap to Membership of RICS - Pathways and Routes
 
Top Rated Pune Call Girls Budhwar Peth ⟟ 6297143586 ⟟ Call Me For Genuine Se...
Top Rated  Pune Call Girls Budhwar Peth ⟟ 6297143586 ⟟ Call Me For Genuine Se...Top Rated  Pune Call Girls Budhwar Peth ⟟ 6297143586 ⟟ Call Me For Genuine Se...
Top Rated Pune Call Girls Budhwar Peth ⟟ 6297143586 ⟟ Call Me For Genuine Se...
 
BSides Seattle 2024 - Stopping Ethan Hunt From Taking Your Data.pptx
BSides Seattle 2024 - Stopping Ethan Hunt From Taking Your Data.pptxBSides Seattle 2024 - Stopping Ethan Hunt From Taking Your Data.pptx
BSides Seattle 2024 - Stopping Ethan Hunt From Taking Your Data.pptx
 
MANUFACTURING PROCESS-II UNIT-1 THEORY OF METAL CUTTING
MANUFACTURING PROCESS-II UNIT-1 THEORY OF METAL CUTTINGMANUFACTURING PROCESS-II UNIT-1 THEORY OF METAL CUTTING
MANUFACTURING PROCESS-II UNIT-1 THEORY OF METAL CUTTING
 
Call Girls in Nagpur Suman Call 7001035870 Meet With Nagpur Escorts
Call Girls in Nagpur Suman Call 7001035870 Meet With Nagpur EscortsCall Girls in Nagpur Suman Call 7001035870 Meet With Nagpur Escorts
Call Girls in Nagpur Suman Call 7001035870 Meet With Nagpur Escorts
 

Comparison of various streaming technologies

  • 1. © 2015 IBM Corporation0 Power of data. Simplicity of design. Speed of innovation. IBM Spark Comparison of various streaming analytics technologies Mario Briggs Sachin Aggarwal March 12, 2016
  • 2. © 2015 IBM Corporation1 Agenda  Streaming Analytics System Architecture  Features needed in Streaming Analytic Applications (What and Why?)  Apache Storm, Trident and Flink vis-à-vis Streaming features  Spark Streaming  Summing it all up  How to benchmark Spark Streaming?
  • 3. © 2015 IBM Corporation2 What is a Streaming Analytics Application? Event Stream Collecting /Processing Event Result
  • 4. © 2015 IBM Corporation3 Streaming System Architecture  Continuous Operator Architecture – Static Scheduling Source Sink Filter, Transform, Aggregate processing Filter, Transform, Aggregate processing
  • 5. © 2015 IBM Corporation4 Task Scheduler Architecture Task Set Task Scheduler Thread + Data Task Executors Thread + Data Thread + Data
  • 6. © 2015 IBM Corporation5 Streaming System Architecture’s – Pros & Cons  Continuous Operator Architecture  No Task scheduling overheads  Task Scheduler Architecture  Dynamic Data partitions can add more parallelism
  • 7. © 2015 IBM Corporation6 Features Needed By Streaming Analytics Applications  Fault Tolerance  Message Processing Guarantees  Back Pressure  Stateful vs Stateless  Built-in Primitives vs Roll your own  Lambda Architecture  Better Resource Utilization  Tuple Level processing vs Micro-batch
  • 8. © 2015 IBM Corporation7 Fault Tolerance Typical Architecture Master Node Cluster Co-ordination Worker Worker Worker Worker What happens when a worker or node dies ? What if the worker is a receiver ? What happens master dies ?
  • 9. © 2015 IBM Corporation8 Message Processing Guarantees  At most once processing  There could be messages lost / not processed. No message will be processed in duplicate.  At least once processing  Messages can be processed in duplicate.  Exactly once processing  No Messages lost. No duplicate processing of messages  End-to-end message processing guarantees depends on processing guarantees of the 3 main elements  Source: requires reliable sources.  Processing: processing gaurantee of the streaming system.  Sink: requires Sink with atomic-write capability
  • 10. © 2015 IBM Corporation9 Back Pressure  Processing conditions in a streaming application can change over time  Unexpected slowdowns in the processing components • For e.g. Data stores being written to or looked-up became slow  Unexpected surge in rate/load of input data.  Back pressure determines what happens when the above occurs  Nothing special. Lose data or system becomes unstable and components start crashing.  Make upstream components keep repeating same work?  Or all components upstream of the slowed-down component, throttle themselves by some mechanism
  • 11. © 2015 IBM Corporation10 Stateful vs Stateless, Built-in primitives vs. Roll your own  Some of the most common logic you implement in your streaming application Calculate and maintain aggregates over time, by a Key. Join multiple input streams by a key. Lookup master data ‘table’ Rolling counts over time windows (by time or count). Trigger on thresholds breached condition.  Stateful vs Stateless is linked to fault tolerance  What happens when you maintain aggregates and the node holding that does down. Do you have to replay from start or can continue from last saved / checkpointed.
  • 12. © 2015 IBM Corporation11 Lambda Architecture  Many systems today need to do both real time as well as historical/batch processing on the same data.  Can you share ‘same’ implementation and logic across these two?  If you can’t, then chances are you have 2 separate implementations of similar logic that is not in sync  Will have different answers from realTime vs historical system
  • 13. © 2015 IBM Corporation12 Better Resource Utilization & Tuple Level vs. Micro-batch  Can your Streaming application share same clusters with other jobs via a common Resource Manager?  Can you reserve resource at the required granular level.
  • 14. © 2015 IBM Corporation13 Apache Storm – Lets understand the programming model
  • 15. © 2015 IBM Corporation14 Apache Storm – Process Model Topology : Network of spouts and Bolts
  • 16. © 2015 IBM Corporation15  True streaming i.e. message level.  Stateless  Cannot maintain state across message’s  Executor failures, require replay from start to build state (fault tolerance of executor)  No built-in primitives  All user code (no built-in aggregates/joins/grouping)  Message processing guarantee's  at most once. Yes (no tracking & replay)  at least once. Yes ( tracks every tuple and children and replays if necessary)  exactly once. No  Run on Shared clusters  No. Scheduler component (Nimbus) doesn’t work with YARN/Mesos  Hortonworks distro allows to run as Yarn application Apache Storm - Features
  • 17. © 2015 IBM Corporation16  Fault Tolerance (Master)  When Nimbus node dies, if a worker node dies then no reassignment of worker to other nodes happen.  Existing workers continue to run and if failed, ‘Supervisor’ will restarted them  No new jobs can be submitted  Back Pressure  No  Unified Programming Model  No Apache Storm - Features
  • 18. © 2015 IBM Corporation17  Batching support for Apache Storm  Stateful  Can maintain state across message’s/batches  Failure of executor, state recoverable from external stores rather than replay from start Apache Trident
  • 19. © 2015 IBM Corporation18 Apache Trident – What’s new in the program model?
  • 20. © 2015 IBM Corporation19  Built-in primitives – Yes Aggregates/grouping/function/filters/joins  Message processing guarantee's  All three levels  Exactly once (atomic txn’s to external store, using a unique batchId & gauranteed ordering of updates among batches)  Run on Shared clusters, Back Pressure  No (Same as Storm)  Fault Tolerance (Master)  Same as Storm Apache Trident
  • 21. © 2015 IBM Corporation20 Apache Storm and Trident drawbacks Source of Storm’s problems – Multiple topology’s tasks, run in a Single JVM process
  • 22. © 2015 IBM Corporation21 Apache Storm and Trident drawbacks  One Storm ‘Worker’ (a Single JVM process) runs too many different components and multiple different tasks, each requiring different resources.  Each tuple needs to pass through 4 threads in a worker  Single global queues and log files across all tasks in a worker.  hard to debug and size correctly. Thus have to oversize a lot, not efficient. • E.g. oversizing memory means when doing a stack dump will cause missed heartbeats.  Nimbus scheduler does not support resource reservation and isolation at a worker level.  Because of above, twitter runs each topology (stream application) on dedicated cluster. http://dl.acm.org/citation.cfm?id=2742788
  • 23. © 2015 IBM Corporation22 Heron Topology
  • 24. © 2015 IBM Corporation23  True streaming  Data Exchange buffer control  Stateful - Yes  Light-weight async checkpointing mechanism (Barriers)  Built-in primitives. Yes  Transformation, aggregations, windows, joins, connect, split  Checkpoint local variables.  Tumbling windows, sliding windows.  Window triggers.  Time windows (Event time, ingestion time, processing time support)  Train models, update & predict.  Message processing guarantee's  Exactly once  Sink exactly-once -> HDFS Apache Flink
  • 25. © 2015 IBM Corporation24 Apache Flink Process Model
  • 26. © 2015 IBM Corporation25  Fault Tolerance of Master  Yes. (since 0.10.0. Only for YARN & standalone)  Back Pressure  Yes. (simple watermark for transfer buffers)  Run on Shared clusters – Yes  Programming Model  Overlapping programming model (Batch does not have SQL support) Apache Flink
  • 27. © 2015 IBM Corporation26 Spark Streaming  Micro batching  Stateful  Yes. (UpdateStateByKey , mapWithState functions)  Built-in primitives – Yes Aggregates/grouping/function/filters/joins Sliding / Tumbling Windows Train models & predict in a streaming app. Event Time (slated for 2.0)  Message processing guarantee's  Exactly once  Run on Shared clusters  Yes
  • 28. © 2015 IBM Corporation27 Spark Streaming  Back Pressure  Yes  Fault Tolerance of Master  Yes. (Dstream state saved to checkpoint file, used to restart master) Programming Model  Integrate wholly with Spark (MLlib, SparkSQL/DataFrame, RDD)  Overlapping model for Batch and Streaming version of application
  • 29. © 2015 IBM Corporation28 Feature Storm Trident Flink Spark Streaming Stateful No Yes (with External Stores) YES YES Message Processing Guarantees No Exactly Once Exactly Once (with external Stores) Exactly Once Exactly Once Back Pressure No No YES YES Built-in Primitives No Yes YES YES Overlapping Programming Model No No YES YES Work with Resource Schedulers No No YES YES True Streaming YES Micro-batch YES Micro-batch Summing it all up
  • 30. © 2015 IBM Corporation29 How to benchmark Spark Streaming?
  • 31. © 2015 IBM Corporation30 Setup Information load-data Pulls data from twitter and stores as fixed dataset push-to-kafka Reads fixed dataset and push to Kafka at specific rate spark-benchmarks Reads data from Kafka and execute benchmark code flink-benchmarks –(WIP) Reads data from Kafka and execute benchmark code
  • 32. © 2015 IBM Corporation31 Listener Interface in Spark • StreamingListener-A listener interface for receiving information about an ongoing streaming computation. • Functions we need to override:- – onReceiverStarted: Called when processing of a job of a batch has completed – onReceiverError: Called when a receiver has reported an error – onReceiverStopped: Called when a receiver has been stopped – onBatchSubmitted: Called when a batch of jobs has been submitted for processing – onBatchStarted: Called when processing of a batch of jobs has started. – onBatchCompleted: Called when processing of a batch of jobs has completed – onOutputOperationStarted: Called when processing of a job of a batch has started – onOutputOperationCompleted: Called when processing of a job of a batch has completed
  • 33. © 2015 IBM Corporation32 Implementation Details Flag to check first batch  Record startTime For each batch  Update totalRecords + batchCompleted.batchInfo.numRecords  Increment batchCount If totalRecords >= recordLimit  Record Endtime  avgLatency = totalDelay / totalRecords  recordThroughput = totalRecords / totalTime
  • 34. © 2015 IBM Corporation33 Observation: Execution with various batch intervals Batch interval (ms) Batch Count Total Records Total consumed time (s) Average Latency/batch interval avg delay per batch Average records/second 1000 1,056 100,000,020 1,056 1,429 429 94688.7392386 900 1,175 100,000,020 1,058 1,362 462 94541.2998858 800 1,322 100,000,020 1,058 1,291 491 94541.2998858 700 1,487 100,000,020 1,041 1,184 484 96034.5609108 600 1,770 100,000,020 1,062 1,102 502 94159.4948532 500 1,681 100,000,020 1,042 1,104 604 95943.1594458 400 1,355 100,000,020 1,033 1,202 802 96774.6187322 300 979 100,000,020 1,062 1,712 1,412 94190.3586768 200 755 100,000,020 1,092 2,212 2,012 91577.2064378 100 718 100,000,020 1,076 2,397 2,297 92963.5183339
  • 35. © 2015 IBM Corporation34 Observation: Execution with various input rate 500 60 1144 1457733645809 1457733645982 1457734424894 1457734424894 100017510 779.085 1165.648763 128378.1744 500 54 1056 1457734624836 1457734624971 1457735504549 1457735504549 108014596 879.713 1378.846423 122783.9034 500 45 1552 1457735724327 1457735724487 1457736730486 1457736730486 112535122 1006.159 1130.425913 111846.2609 500 36 2290 1457736944320 1457736944450 1457738209266 1457738209266 120010057 1264.946 1007.914431 94873.66022 500 24 3600 1457738405811 1457738405990 1457740206047 1457740206047 120005209 1800.236 962.950761 66660.82058 500 12 6517 1457740349824 1457740349984 1457743608456 1457743608456 120000935 3258.632 875.2863643 36825.55594 500 6 12956 1457743741277 1457743741433 1457750219374 1457750219374 120000857 6478.097 766.6043024 18524.09079
  • 36. © 2015 IBM Corporation35
  • 37. © 2015 IBM Corporation36 Back up
  • 38. © 2015 IBM Corporation37

Notas do Editor

  1. Add Dynamic Allocation.
  2. https://github.com/davidkiss/storm-twitter-word-count
  3. https://github.com/Blackmist/TwitterTrending
  4. https://ci.apache.org/projects/flink/flink-docs-master/apis/streaming/index.html#controlling-latency https://ci.apache.org/projects/flink/flink-docs-release-0.10/setup/config.html#configuring-taskmanager-processing-slots
  5. https://cwiki.apache.org/confluence/display/FLINK/Data+exchange+between+tasks https://ci.apache.org/projects/flink/flink-docs-release-0.10/setup/config.html#configuring-taskmanager-processing-slots
  6. Flink uses effectively distributed blocking queues with bounded capacity The output side never puts too much data on the wire by a simple watermark mechanism. If enough data is in-flight, we wait before we copy more data to the wire until it is below a threshold. This guarantees that there is never too much data in-flight. If new data is not consumed on the receiving side (because there is no buffer available), this slows down the sender. http://data-artisans.com/how-flink-handles-backpressure/
  7. 35