SlideShare ist ein Scribd-Unternehmen logo
1 von 34
Downloaden Sie, um offline zu lesen
이름 김병진, 권오찬
Extending Spark Streaming to
Support Complex Event
Processing
소속 삼성전자
2015. 10. 27.
Agenda
• Background
• Motivation
• Streaming SQL
• Auto Scaling
• Future Work
Background - Spark
• Apache Spark is a high performance data processing platform
• Use a distributed memory cache (RDD*)
• Process batch and stream data in the common platform
• Be developed by 700+ contributors
3x faster than Storm
(Stream, 30 Nodes)
100x faster than Hadoop
(Batch, 50 Nodes, Clustering Alg.)
*RDD: Resilient Distributed Datasets
Background – Spark RDD
• Resilient Distributed Datasets
• RDDs are immutable
• Transformations are lazy operations which build a RDD’s lineage graph
• E.g.: map, filter, join, union
• Actions launch a computation to return a value or write data
• E.g.: count, collect, reduce, save
• The scheduler builds a DAG of stages to execute
• If a task fails, spark re-runs it on another node as long as its stage’s parent are still
available
Background - Spark Modules
• Spark Streaming
• Divide stream data into micro-batches
• Compute the batches with fault tolerance
• Spark SQL
• Support SQL to handle structured data
• Provide a common way to access a variety of data
sources (Hive, Avro, Parquet, ORC, JSON, and JDBC)
Spark
Spark
Streaming
batches of
X seconds
live data stream
processed
results
DStreams
RDDs
map, reduce, count, …
Motivation – Support CEP in Spark
• Current spark is not enough to support CEP*
• Do not support continuous query language to process stream data
• Do not support auto-scaling to elastically allocate resources
• We have solved the issues:
• Extend Intel’s Streaming SQL package
• Improve performance by optimizing time-based windowed aggregation
• Support query chains by implementing “Insert Into” queries
• Implement elastic-seamless resource allocation
• Can do auto-scale in/out
*Complex Event Processing
Streaming SQL
Streaming SQL - Intel
• Streaming SQL is a third party library of Spark Packages
• Build on top of Spark Streaming and Spark SQL Catalyst
• Manipulate stream data like static structured data in database
• Process queries continuously
• Support time based windowing join/aggregation queries
http://spark-packages.org/package/Intel-bigdata/spark-streamingsql
Streaming
SQL
Streaming
SQL Query
Schema
DStream
Optimized
Logical
Plan
Streaming SQL - Plans
• Logical Plan
• Modify streaming SQL optimizer
• Add windowed aggregate logical plan
• Physical Plan
• Add windowed aggregate physical plan to call new WindowedStateDStream class
• Implement new expression functions (Count, Sum, Average, Min, Max, Distinct)
• Develop FixedSizedAggregator for efficiency which is the concept of IBM InfoSphere Streams
Samsung
Stream
Plan
Intel
Logical
Plan
Physical
Plan
Streaming SQL – Windowed State
• WindowedStateDStream class
• Modify StateDStream class to support windowed computing
• Add inverse update function to evaluate old values
• Add filter function to remove obsolete keys
Intel Samsung
Compute all elements in the window
time 1 time 2 time 3 time 4 time 5
Original
DStream
Windowed
State
DStream
time 1 time 2 time 3 time 4 time 5
Original
DStream
Windowed
DStream
Compute only Delta (old and new elements)
window-based
operation Update (in)
InvUpdate (out)
State: Array[AggregateFunction]
Count
Update
InvUpdate
Update
InvUpdate
State
Sum
Update
InvUpdate
State
Avg
Update
InvUpdate
State
Min
Update
InvUpdate
State
Max
Update
InvUpdate
State
Streaming SQL – Fixed Sized Aggregator
• Fixed-Sized Aggregator* of IBM InfoSphere Streams
• Use fixed sized binary tree
• Maintain leaf nodes as a circular buffer using front and back pointers
• Very efficient for non-invertable expressions (e.g. Min, Max)
• Example - Min Aggregator
4 7 3
4 3
3
4 7 3 2
4 2
2
7 3 2
7 2
2
9 7 3 2
7 2
2
4
4
4
4 7
4
4
*K. Tangwongsan, M. Hirzel, S. Schneider, K-L. Wu, “General incremental sliding-window aggregation,” In VLDB, 2015.
Streaming SQL – Aggregate Functions
• Windowed Aggregate Functions
• Add invUpdate function
• Reduce objects for efficient serialization
• Implement Fixed-Sized Aggregator for Min, Max functions
Aggregate Functions State Update InvUpdate
Count countValue: Long Increase countValue Decrease countValue
Sum sumValue: Any Add to sumValue Subtract to sumValue
Average sumValue: Any
countValue: Long
Increase countValue
Add to sumValue
Decrease countValue
Subtract to sumValue
Min fat: FixedSizedAggregator Insert minimum to fat Remove the oldest from fat
Max fat: FixedSizedAggregator Insert maximum to fat Remove the oldest from fat
CountDistinct distinctMap:
mutable.HashMap[Row, Long]
Increase count of distinct value
in map
Decrease count of distinct value
in map
Streaming SQL – Insert Into
• Support “Insert Into” query
• Implement Data Sources API
• Implement the insert function in Kafka relation
• Modified some physical plans to support streaming
• Modified physical planning strategies to assign a specific plan
• Convert current RDD to a DataFrame and then insert it
• Support query chaining
 The result a query can be reused by multiple queries
Example2
Kafka Kafka
INSERT INTO TABLE
Example1
SELECT duid, time
FROM ParsedTable
INSERT INTO TABLE Example2
SELECT duid, COUNT (*)
FROM Example1
GROUP BY duidParsed
Table
Example1
Kafka
Kafka Relation InsertIntoStreamSource
StreamExecutedCommandInsertIntoTable
Kafka
Logical Plan Physical Plan
Kafka Relation
DataFrame
Streaming SQL – Evaluation
• Experimental Environment
• Processing Node: Intel i5 2.67GHz, 4 Cores, 4GB per node
Spark Cluster: 7 Nodes
Kafka Cluster: 3 Nodes
• Test Query:
SELECT t.word, COUNT(t.word), SUM(t.num), AVG(t.num), MIN(t.num), MAX(t.num)
FROM (SELECT * FROM t_kafka) OVER (WINDOW 'x' SECONDS, SLIDE ‘1' SECONDS) AS t
GROUP BY t.word
• Default Set:
100 EPS, 100 Keys, 20 sec Window, 1 sec Slide, 10 Executors, 10 Reducers
Spark Cluster
Test
Agent
Kafka Cluster
Spark UI
Streaming SQL – Evaluation
• Test Result
• Show low processing delays despite of heavy event loads or large-sized windows
• Need memory optimization
0
200
400
600
800
1000
1200
1400
20 40 60 80 100
Delay(ms)
Window Size (sec)
Intel Samsung
0
1000
2000
3000
4000
5000
6000
7000
8000
9000
10000
20 40 60 80 100
Memory(KB)
Window Size (sec)
Intel Samsung
0
1000
2000
3000
4000
5000
10 100 1000 10000
Delay(ms)
EPS
Intel Samsung
0
2000
4000
6000
8000
10000
12000
10 100 1000 10000
Memory(KB)
EPS
Intel Samsung
Auto Scaling
Time-varying Event Rate in Real World
• Spark Streaming
• Data can be ingested from many sources like Kafka, Flume, Twitter, etc.
• Live input data streams are divided into batches, and they are processed
• In streaming application, event rate may change frequently over time
• How can this be dealt with?
< Event Rate > < Batch Processing Time>
High event rate Can’t be processed in real-time
Spark AS-IS
• Spark currently supports dynamic resource allocation on Yarn
(SPARK-3174) and coarse-grained Mesos (SPARK-6287)
• Existing Dynamic Resource Allocation is not optimized for streaming
• Even though event rate becomes smaller, executors may not be removed due to
scheduling policy
• It is difficult to determine appropriate configuration parameters
• Backpressure (SPARK-7398)
• Enable the Spark streaming to control the receiving rate dynamically for handling bursty input
streams
• Not real-time
Driver
Task queue Task
Executor #1
...
Executor #2
Executor #n
• Upper/lower bound for the number of executors
• spark.dynamicAllocation.minExecutors
• spark.dynamicAllocation.maxExecutors
• Scale-out condition
• schedulerBacklogTimeout < staying time of a task in task queue
• Scale-in condition
• executorIdleTimeout < staying time of an executor in idle state
• Goal
• Allocate resources to streaming applications dynamically as the rate of incoming events varies
over time
• Enable the applications to meet real-time deadlines
• Utilize the resource efficiently
• Cloud Architecture
Elastic-seamless Resource Allocation
Flint*
*Flint: our spark job manager
• Spark currently supports three cluster managers
• Standalone, Apache Mesos, Hadoop Yarn
• Spark on Mesos
• Fine-grained mode
• Each Spark task runs as a separate Mesos task.
• Launching overhead is big, so it is not suitable for streaming.
• Coarse-grained mode
• Launch only one long-running Spark task on each Mesos machine
• Cannot scale-out more than the number of Mesos machines
• Cannot control executor resource
• Only available for total resource
• Both of two modes have some problems to achieve our goal
Spark Deployment Architecture
Flint Architecture Overview
Slave 1 Slave 2 Slave n
. . .
MESOS Cluster
Cluster 1
YARN
Cluster n
YARN
. . .
. . .
NM 1
NM 2
. . .
NM n NM 1
NM 2
NM n
. . .
App 1
SPARK
App n
SPARK
EXEC 1
EXEC 2
EXEC n. . . EXEC 1
EXEC 2
EXEC n. . .
FLINT
REST
Manager
Scheduler
Application
Manager
Application
Handler
Application
Handler
Application
Handler
Deploy
Manager
Log
Manager
• Marathon, Zookeeper, ETCD, HDFS
Spark on on-demand Yarn
• Job submission process
1. Request to deploy dockerized YARN via Marathon
2. Launch ResourceManager, NodeManagers for Spark driver/executors
3. Check ResourceManager status
4. Submit Spark Job and check Job status
5. Watch Spark driver status and get Spark endpoint
Flint Architecture: Job Submission
Slave 1 Slave n
. . .
MESOS Cluster
Slave 1 Slave n
. . .
MESOS Cluster
Cluster
YARN
NM 1
NM 2
. . .
NM n
Slave 1 Slave n
. . .
MESOS Cluster
Cluster
YARN
NM 1
NM 2
. . .
NM n
App
SPARK
EXEC 1
EXEC 2
EXEC n. . .
Create Yarn Cluster
Submit Spark Job
• Scale-out Process
1. Request to increase the instance of NodeManager via Marathon
2. Launch new NodeManager
3. Scale-out Spark executor
Flint Architecture: Scale-out
Slave 1 Slave n
. . .
MESOS Cluster
Cluster
YARN
NM 1
. . .
NM n
App
SPARK
EXEC 1 EXEC n. . .
Slave 1 Slave n
. . .
MESOS Cluster
Cluster
YARN
NM 1
. . .
NM n
App
SPARK
EXEC 1 EXEC n. . .
NM 2
Slave 1 Slave n
. . .
MESOS Cluster
Cluster
YARN
NM 1
. . .
NM n
NM 2
App
SPARK
EXEC 1 EXEC n. . .
EXEC 2
Request new NodeManager Scale-out Spark executor
• Scale-in Process
1. Get Executor’s info and select spark victims
2. Inactivate spark victims and kill them after n x batch interval
3. Get Yarn victims and decommission NodeManager via ResourceManager
4. Get mesos task id of Yarn victims and kill mesos tasks of victims
Flint Architecture: Scale-in
Slave 1 Slave n
. . .
MESOS Cluster
Cluster
YARN
NM 1
. . .
NM n
NM 2
App
SPARK
EXEC 1 EXEC n. . .
EXEC 2
Slave 1 Slave n
. . .
MESOS Cluster
Cluster
YARN
NM 1
. . .
NM n
NM 2
App
SPARK
EXEC 1 EXEC n. . .
Slave 1 Slave n
. . .
MESOS Cluster
Cluster
YARN
NM 1
. . .
NM n
NM 2
App
SPARK
EXEC 1 EXEC n. . .
Scale-in Spark executor Remove NodeManager
• Auto-scaling Mechanism
• Real-time constraint
• Batch processing time < Batch interval
• Scale-out condition
• α x batch interval < batch processing delay
• Scale-in condition
• β x batch interval > batch processing delay
Flint Architecture: Auto-scaling
( 0 < β < α ≤ 1 )
Processing Time
Batch Interval
α x Batch Intervalβ x Batch Interval
Scale-in Scale-out
0 Time
• Timeout & Retry for Killed Executor
• Although Spark executor is killed by admin, Spark re-tries to connect it.
Spark Issues: Scale-in (1/3)
Scale-in
Scheduling Delay
Event Rate
Processing Delay
Driver Exec*
scale-in
Exec Task
RDD req
1st try
2nd try
3rd try
fail
Processing
Delay
• spark.shuffle.io.maxRetries = 3
• spark.shuffle.io.retryWait = 5s
• Timeout & Retry for Killed Executor
• Advertisement for killed executor
• Inactivate executor before scale-in
• Inactivate executor and kill them after n x batch interval
• During inactivating stage, the executor does not receive any task from driver.
Spark Issues: Scale-in (2/3)
Driver Exec*
scale-in
Exec Task
RDD req
fail
advertise
Fast
Fail
• Data Locality
• Spark scheduler consider data locality
• Processing/Node/Rack locality
• spark.locality.wait = 3s
• However, waiting time for locality is big burden to streaming applications
• The waiting time should be much less than batch interval for streaming
• Otherwise, streaming may not be processed in real-time
• Thus, Flint overrides “spark.locality.wait” to very small value if application type is streaming
Spark Issues: Scale-in (3/3)
Scheduler Cluster
TTT T …
3s10ms
(max)
rsc alloc
T
• Data Localization
• During scale-out process, new Yarn container performs to localize some data
(e.g. spark jar, application jar)
• Data localization incurs high disk I/O, so
• Solution
• Prefetch if possible
• Disk isolation, SSD
Spark Issues: Scale-out (1/2)
Existing
App
New
App
Disk
Access disk during localization
(50-70 MB/sec)
Performance
Degradation
• RDD Replication
• When a new executor is added, a receiver requests to replicate received RDD blocks to the
executor
• New executor does not ready to receive RDD blocks during an initialization
• At this situation, the receiver waits until new executor is ready
• Solution
• A receiver does not replicate RDD blocks to new executor during initialization
• E.g., spark.blockmanager.peer.bootstrap = 10s
Spark Issues: Scale-out (2/2)
Kafka
R
RT
R R
Executor R
R
T
R
Executor 1
R
T
R
Executor 2
RT
T
Receiver Task
Task
R RDDReplicate factor = 2
• Demo Environment
• Data source: Kafka
• Auto-scaling parameter
• α=0.4, β=0.9
• SQL query
• SELECT t.word, COUNT(DISTINCT t.num), SUM(t.num), AVG(t.num), MIN(t.num), MAX(t.num) FROM
(SELECT * FROM t_kafka) OVER (WINDOW 300 SECONDS, SLIDE 3 SECONDS) AS t GROUP BY t.word
Demo (1/2)
Task
Task
Producer
Rcvr
8 3 4 2
5 7 9 1
0 3 7 8
Task Driver
0: 23
1: 34
2: 41
…
Flint
Monitoring
Scale-in/Out
Demo (2/2)
Future Work
• Streaming SQL
• Apply Tungsten Framework
• Small-sized object, code gen, GC free memory management
• Share RDDs
• Map stream tables to RDD
• Auto Scaling
• Support to dynamically allocate resources for batch (non-streaming) applications
• Support unified schedulers for heterogeneous applications
• Batch, streaming, ad-hoc
THANK YOU!
https://github.com/samsung/spark-cep

Weitere ähnliche Inhalte

Was ist angesagt?

Spark Summit EU talk by Qifan Pu
Spark Summit EU talk by Qifan PuSpark Summit EU talk by Qifan Pu
Spark Summit EU talk by Qifan PuSpark Summit
 
Scaling Data Analytics Workloads on Databricks
Scaling Data Analytics Workloads on DatabricksScaling Data Analytics Workloads on Databricks
Scaling Data Analytics Workloads on DatabricksDatabricks
 
Tagging and Processing Data in Real Time-(Hari Shreedharan and Siddhartha Jai...
Tagging and Processing Data in Real Time-(Hari Shreedharan and Siddhartha Jai...Tagging and Processing Data in Real Time-(Hari Shreedharan and Siddhartha Jai...
Tagging and Processing Data in Real Time-(Hari Shreedharan and Siddhartha Jai...Spark Summit
 
Deep Dive Into Apache Spark Multi-User Performance Michael Feiman, Mikhail Ge...
Deep Dive Into Apache Spark Multi-User Performance Michael Feiman, Mikhail Ge...Deep Dive Into Apache Spark Multi-User Performance Michael Feiman, Mikhail Ge...
Deep Dive Into Apache Spark Multi-User Performance Michael Feiman, Mikhail Ge...Databricks
 
Apache Beam (incubating)
Apache Beam (incubating)Apache Beam (incubating)
Apache Beam (incubating)Apache Apex
 
Spark Summit EU talk by Ram Sriharsha and Vlad Feinberg
Spark Summit EU talk by Ram Sriharsha and Vlad FeinbergSpark Summit EU talk by Ram Sriharsha and Vlad Feinberg
Spark Summit EU talk by Ram Sriharsha and Vlad FeinbergSpark Summit
 
Spark Summit EU talk by Mike Percy
Spark Summit EU talk by Mike PercySpark Summit EU talk by Mike Percy
Spark Summit EU talk by Mike PercySpark Summit
 
Spark Streaming and IoT by Mike Freedman
Spark Streaming and IoT by Mike FreedmanSpark Streaming and IoT by Mike Freedman
Spark Streaming and IoT by Mike FreedmanSpark Summit
 
Challenging Web-Scale Graph Analytics with Apache Spark with Xiangrui Meng
Challenging Web-Scale Graph Analytics with Apache Spark with Xiangrui MengChallenging Web-Scale Graph Analytics with Apache Spark with Xiangrui Meng
Challenging Web-Scale Graph Analytics with Apache Spark with Xiangrui MengDatabricks
 
Why is My Stream Processing Job Slow? with Xavier Leaute
Why is My Stream Processing Job Slow? with Xavier LeauteWhy is My Stream Processing Job Slow? with Xavier Leaute
Why is My Stream Processing Job Slow? with Xavier LeauteDatabricks
 
Downscaling: The Achilles heel of Autoscaling Apache Spark Clusters
Downscaling: The Achilles heel of Autoscaling Apache Spark ClustersDownscaling: The Achilles heel of Autoscaling Apache Spark Clusters
Downscaling: The Achilles heel of Autoscaling Apache Spark ClustersDatabricks
 
Lambda architecture on Spark, Kafka for real-time large scale ML
Lambda architecture on Spark, Kafka for real-time large scale MLLambda architecture on Spark, Kafka for real-time large scale ML
Lambda architecture on Spark, Kafka for real-time large scale MLhuguk
 
Use of Spark MLib for Predicting the Offlining of Digital Media-(Christopher ...
Use of Spark MLib for Predicting the Offlining of Digital Media-(Christopher ...Use of Spark MLib for Predicting the Offlining of Digital Media-(Christopher ...
Use of Spark MLib for Predicting the Offlining of Digital Media-(Christopher ...Spark Summit
 
Spark Summit EU talk by Herman van Hovell
Spark Summit EU talk by Herman van HovellSpark Summit EU talk by Herman van Hovell
Spark Summit EU talk by Herman van HovellSpark Summit
 
Clipper: A Low-Latency Online Prediction Serving System
Clipper: A Low-Latency Online Prediction Serving SystemClipper: A Low-Latency Online Prediction Serving System
Clipper: A Low-Latency Online Prediction Serving SystemDatabricks
 
Spark Summit EU talk by Mikhail Semeniuk Hollin Wilkins
Spark Summit EU talk by Mikhail Semeniuk Hollin WilkinsSpark Summit EU talk by Mikhail Semeniuk Hollin Wilkins
Spark Summit EU talk by Mikhail Semeniuk Hollin WilkinsSpark Summit
 
Spark Summit EU talk by Josef Habdank
Spark Summit EU talk by Josef HabdankSpark Summit EU talk by Josef Habdank
Spark Summit EU talk by Josef HabdankSpark Summit
 
Next CERN Accelerator Logging Service with Jakub Wozniak
Next CERN Accelerator Logging Service with Jakub WozniakNext CERN Accelerator Logging Service with Jakub Wozniak
Next CERN Accelerator Logging Service with Jakub WozniakSpark Summit
 
Spark Summit EU talk by Kaarthik Sivashanmugam
Spark Summit EU talk by Kaarthik SivashanmugamSpark Summit EU talk by Kaarthik Sivashanmugam
Spark Summit EU talk by Kaarthik SivashanmugamSpark Summit
 

Was ist angesagt? (20)

Spark Summit EU talk by Qifan Pu
Spark Summit EU talk by Qifan PuSpark Summit EU talk by Qifan Pu
Spark Summit EU talk by Qifan Pu
 
Scaling Data Analytics Workloads on Databricks
Scaling Data Analytics Workloads on DatabricksScaling Data Analytics Workloads on Databricks
Scaling Data Analytics Workloads on Databricks
 
Tagging and Processing Data in Real Time-(Hari Shreedharan and Siddhartha Jai...
Tagging and Processing Data in Real Time-(Hari Shreedharan and Siddhartha Jai...Tagging and Processing Data in Real Time-(Hari Shreedharan and Siddhartha Jai...
Tagging and Processing Data in Real Time-(Hari Shreedharan and Siddhartha Jai...
 
Deep Dive Into Apache Spark Multi-User Performance Michael Feiman, Mikhail Ge...
Deep Dive Into Apache Spark Multi-User Performance Michael Feiman, Mikhail Ge...Deep Dive Into Apache Spark Multi-User Performance Michael Feiman, Mikhail Ge...
Deep Dive Into Apache Spark Multi-User Performance Michael Feiman, Mikhail Ge...
 
Apache Beam (incubating)
Apache Beam (incubating)Apache Beam (incubating)
Apache Beam (incubating)
 
Spark Summit EU talk by Ram Sriharsha and Vlad Feinberg
Spark Summit EU talk by Ram Sriharsha and Vlad FeinbergSpark Summit EU talk by Ram Sriharsha and Vlad Feinberg
Spark Summit EU talk by Ram Sriharsha and Vlad Feinberg
 
Spark Summit EU talk by Mike Percy
Spark Summit EU talk by Mike PercySpark Summit EU talk by Mike Percy
Spark Summit EU talk by Mike Percy
 
Spark Streaming and IoT by Mike Freedman
Spark Streaming and IoT by Mike FreedmanSpark Streaming and IoT by Mike Freedman
Spark Streaming and IoT by Mike Freedman
 
Challenging Web-Scale Graph Analytics with Apache Spark with Xiangrui Meng
Challenging Web-Scale Graph Analytics with Apache Spark with Xiangrui MengChallenging Web-Scale Graph Analytics with Apache Spark with Xiangrui Meng
Challenging Web-Scale Graph Analytics with Apache Spark with Xiangrui Meng
 
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
 
Why is My Stream Processing Job Slow? with Xavier Leaute
Why is My Stream Processing Job Slow? with Xavier LeauteWhy is My Stream Processing Job Slow? with Xavier Leaute
Why is My Stream Processing Job Slow? with Xavier Leaute
 
Downscaling: The Achilles heel of Autoscaling Apache Spark Clusters
Downscaling: The Achilles heel of Autoscaling Apache Spark ClustersDownscaling: The Achilles heel of Autoscaling Apache Spark Clusters
Downscaling: The Achilles heel of Autoscaling Apache Spark Clusters
 
Lambda architecture on Spark, Kafka for real-time large scale ML
Lambda architecture on Spark, Kafka for real-time large scale MLLambda architecture on Spark, Kafka for real-time large scale ML
Lambda architecture on Spark, Kafka for real-time large scale ML
 
Use of Spark MLib for Predicting the Offlining of Digital Media-(Christopher ...
Use of Spark MLib for Predicting the Offlining of Digital Media-(Christopher ...Use of Spark MLib for Predicting the Offlining of Digital Media-(Christopher ...
Use of Spark MLib for Predicting the Offlining of Digital Media-(Christopher ...
 
Spark Summit EU talk by Herman van Hovell
Spark Summit EU talk by Herman van HovellSpark Summit EU talk by Herman van Hovell
Spark Summit EU talk by Herman van Hovell
 
Clipper: A Low-Latency Online Prediction Serving System
Clipper: A Low-Latency Online Prediction Serving SystemClipper: A Low-Latency Online Prediction Serving System
Clipper: A Low-Latency Online Prediction Serving System
 
Spark Summit EU talk by Mikhail Semeniuk Hollin Wilkins
Spark Summit EU talk by Mikhail Semeniuk Hollin WilkinsSpark Summit EU talk by Mikhail Semeniuk Hollin Wilkins
Spark Summit EU talk by Mikhail Semeniuk Hollin Wilkins
 
Spark Summit EU talk by Josef Habdank
Spark Summit EU talk by Josef HabdankSpark Summit EU talk by Josef Habdank
Spark Summit EU talk by Josef Habdank
 
Next CERN Accelerator Logging Service with Jakub Wozniak
Next CERN Accelerator Logging Service with Jakub WozniakNext CERN Accelerator Logging Service with Jakub Wozniak
Next CERN Accelerator Logging Service with Jakub Wozniak
 
Spark Summit EU talk by Kaarthik Sivashanmugam
Spark Summit EU talk by Kaarthik SivashanmugamSpark Summit EU talk by Kaarthik Sivashanmugam
Spark Summit EU talk by Kaarthik Sivashanmugam
 

Andere mochten auch

Siddhi: A Second Look at Complex Event Processing Implementations
Siddhi: A Second Look at Complex Event Processing ImplementationsSiddhi: A Second Look at Complex Event Processing Implementations
Siddhi: A Second Look at Complex Event Processing ImplementationsSrinath Perera
 
Rule Engine & Drools
Rule Engine & DroolsRule Engine & Drools
Rule Engine & DroolsSandip Jadhav
 
Building a Data Warehouse for Business Analytics using Spark SQL-(Blagoy Kalo...
Building a Data Warehouse for Business Analytics using Spark SQL-(Blagoy Kalo...Building a Data Warehouse for Business Analytics using Spark SQL-(Blagoy Kalo...
Building a Data Warehouse for Business Analytics using Spark SQL-(Blagoy Kalo...Spark Summit
 
Apache storm vs. Spark Streaming
Apache storm vs. Spark StreamingApache storm vs. Spark Streaming
Apache storm vs. Spark StreamingP. Taylor Goetz
 
JBoss Drools - Pure Java Rule Engine
JBoss Drools - Pure Java Rule EngineJBoss Drools - Pure Java Rule Engine
JBoss Drools - Pure Java Rule EngineAnil Allewar
 
Elassandra: Elasticsearch as a Cassandra Secondary Index (Rémi Trouville, Vin...
Elassandra: Elasticsearch as a Cassandra Secondary Index (Rémi Trouville, Vin...Elassandra: Elasticsearch as a Cassandra Secondary Index (Rémi Trouville, Vin...
Elassandra: Elasticsearch as a Cassandra Secondary Index (Rémi Trouville, Vin...DataStax
 
Deep Dive with Spark Streaming - Tathagata Das - Spark Meetup 2013-06-17
Deep Dive with Spark Streaming - Tathagata  Das - Spark Meetup 2013-06-17Deep Dive with Spark Streaming - Tathagata  Das - Spark Meetup 2013-06-17
Deep Dive with Spark Streaming - Tathagata Das - Spark Meetup 2013-06-17spark-project
 
Infinit: Modern Storage Platform for Container Environments
Infinit: Modern Storage Platform for Container EnvironmentsInfinit: Modern Storage Platform for Container Environments
Infinit: Modern Storage Platform for Container EnvironmentsDocker, Inc.
 
Apache Spark 2.0: A Deep Dive Into Structured Streaming - by Tathagata Das
Apache Spark 2.0: A Deep Dive Into Structured Streaming - by Tathagata Das Apache Spark 2.0: A Deep Dive Into Structured Streaming - by Tathagata Das
Apache Spark 2.0: A Deep Dive Into Structured Streaming - by Tathagata Das Databricks
 
Hadoop Summit Europe 2014: Apache Storm Architecture
Hadoop Summit Europe 2014: Apache Storm ArchitectureHadoop Summit Europe 2014: Apache Storm Architecture
Hadoop Summit Europe 2014: Apache Storm ArchitectureP. Taylor Goetz
 
Data to Insight in a Flash: Introduction to Real-Time Analytics with WSO2 Com...
Data to Insight in a Flash: Introduction to Real-Time Analytics with WSO2 Com...Data to Insight in a Flash: Introduction to Real-Time Analytics with WSO2 Com...
Data to Insight in a Flash: Introduction to Real-Time Analytics with WSO2 Com...WSO2
 
Strtio Spark Streaming + Siddhi CEP Engine
Strtio Spark Streaming + Siddhi CEP EngineStrtio Spark Streaming + Siddhi CEP Engine
Strtio Spark Streaming + Siddhi CEP EngineMyung Ho Yun
 
C*ollege Credit: CEP Distribtued Processing on Cassandra with Storm
C*ollege Credit: CEP Distribtued Processing on Cassandra with StormC*ollege Credit: CEP Distribtued Processing on Cassandra with Storm
C*ollege Credit: CEP Distribtued Processing on Cassandra with StormDataStax
 
Introducing the WSO2 Complex Event Processor
Introducing the WSO2 Complex Event ProcessorIntroducing the WSO2 Complex Event Processor
Introducing the WSO2 Complex Event ProcessorWSO2
 
WSO2Con USA 2015: Decide and Do By Knowing With WSO2 CEP
WSO2Con USA 2015: Decide and Do By Knowing With WSO2 CEPWSO2Con USA 2015: Decide and Do By Knowing With WSO2 CEP
WSO2Con USA 2015: Decide and Do By Knowing With WSO2 CEPWSO2
 
Analyzing a Soccer Game with WSO2 CEP
Analyzing a Soccer Game with WSO2 CEPAnalyzing a Soccer Game with WSO2 CEP
Analyzing a Soccer Game with WSO2 CEPSrinath Perera
 
Developing Distributed Web Applications, Where does REST fit in?
Developing Distributed Web Applications, Where does REST fit in?Developing Distributed Web Applications, Where does REST fit in?
Developing Distributed Web Applications, Where does REST fit in?Srinath Perera
 

Andere mochten auch (20)

Flink vs. Spark
Flink vs. SparkFlink vs. Spark
Flink vs. Spark
 
Siddhi: A Second Look at Complex Event Processing Implementations
Siddhi: A Second Look at Complex Event Processing ImplementationsSiddhi: A Second Look at Complex Event Processing Implementations
Siddhi: A Second Look at Complex Event Processing Implementations
 
Rule Engine & Drools
Rule Engine & DroolsRule Engine & Drools
Rule Engine & Drools
 
Business Rules on Hadoop
Business Rules on HadoopBusiness Rules on Hadoop
Business Rules on Hadoop
 
Building a Data Warehouse for Business Analytics using Spark SQL-(Blagoy Kalo...
Building a Data Warehouse for Business Analytics using Spark SQL-(Blagoy Kalo...Building a Data Warehouse for Business Analytics using Spark SQL-(Blagoy Kalo...
Building a Data Warehouse for Business Analytics using Spark SQL-(Blagoy Kalo...
 
Apache storm vs. Spark Streaming
Apache storm vs. Spark StreamingApache storm vs. Spark Streaming
Apache storm vs. Spark Streaming
 
JBoss Drools - Pure Java Rule Engine
JBoss Drools - Pure Java Rule EngineJBoss Drools - Pure Java Rule Engine
JBoss Drools - Pure Java Rule Engine
 
Elassandra: Elasticsearch as a Cassandra Secondary Index (Rémi Trouville, Vin...
Elassandra: Elasticsearch as a Cassandra Secondary Index (Rémi Trouville, Vin...Elassandra: Elasticsearch as a Cassandra Secondary Index (Rémi Trouville, Vin...
Elassandra: Elasticsearch as a Cassandra Secondary Index (Rémi Trouville, Vin...
 
Deep Dive with Spark Streaming - Tathagata Das - Spark Meetup 2013-06-17
Deep Dive with Spark Streaming - Tathagata  Das - Spark Meetup 2013-06-17Deep Dive with Spark Streaming - Tathagata  Das - Spark Meetup 2013-06-17
Deep Dive with Spark Streaming - Tathagata Das - Spark Meetup 2013-06-17
 
Infinit: Modern Storage Platform for Container Environments
Infinit: Modern Storage Platform for Container EnvironmentsInfinit: Modern Storage Platform for Container Environments
Infinit: Modern Storage Platform for Container Environments
 
Apache Spark 2.0: A Deep Dive Into Structured Streaming - by Tathagata Das
Apache Spark 2.0: A Deep Dive Into Structured Streaming - by Tathagata Das Apache Spark 2.0: A Deep Dive Into Structured Streaming - by Tathagata Das
Apache Spark 2.0: A Deep Dive Into Structured Streaming - by Tathagata Das
 
Hadoop Summit Europe 2014: Apache Storm Architecture
Hadoop Summit Europe 2014: Apache Storm ArchitectureHadoop Summit Europe 2014: Apache Storm Architecture
Hadoop Summit Europe 2014: Apache Storm Architecture
 
Data to Insight in a Flash: Introduction to Real-Time Analytics with WSO2 Com...
Data to Insight in a Flash: Introduction to Real-Time Analytics with WSO2 Com...Data to Insight in a Flash: Introduction to Real-Time Analytics with WSO2 Com...
Data to Insight in a Flash: Introduction to Real-Time Analytics with WSO2 Com...
 
Strtio Spark Streaming + Siddhi CEP Engine
Strtio Spark Streaming + Siddhi CEP EngineStrtio Spark Streaming + Siddhi CEP Engine
Strtio Spark Streaming + Siddhi CEP Engine
 
C*ollege Credit: CEP Distribtued Processing on Cassandra with Storm
C*ollege Credit: CEP Distribtued Processing on Cassandra with StormC*ollege Credit: CEP Distribtued Processing on Cassandra with Storm
C*ollege Credit: CEP Distribtued Processing on Cassandra with Storm
 
Introducing the WSO2 Complex Event Processor
Introducing the WSO2 Complex Event ProcessorIntroducing the WSO2 Complex Event Processor
Introducing the WSO2 Complex Event Processor
 
WSO2Con USA 2015: Decide and Do By Knowing With WSO2 CEP
WSO2Con USA 2015: Decide and Do By Knowing With WSO2 CEPWSO2Con USA 2015: Decide and Do By Knowing With WSO2 CEP
WSO2Con USA 2015: Decide and Do By Knowing With WSO2 CEP
 
Analyzing a Soccer Game with WSO2 CEP
Analyzing a Soccer Game with WSO2 CEPAnalyzing a Soccer Game with WSO2 CEP
Analyzing a Soccer Game with WSO2 CEP
 
Developing Distributed Web Applications, Where does REST fit in?
Developing Distributed Web Applications, Where does REST fit in?Developing Distributed Web Applications, Where does REST fit in?
Developing Distributed Web Applications, Where does REST fit in?
 
CEP: from Esper back to Akka
CEP: from Esper back to AkkaCEP: from Esper back to Akka
CEP: from Esper back to Akka
 

Ähnlich wie Extending Spark Streaming to Support Complex Event Processing

Headaches and Breakthroughs in Building Continuous Applications
Headaches and Breakthroughs in Building Continuous ApplicationsHeadaches and Breakthroughs in Building Continuous Applications
Headaches and Breakthroughs in Building Continuous ApplicationsDatabricks
 
Infrastructure at Scale: Apache Kafka, Twitter Storm & Elastic Search (ARC303...
Infrastructure at Scale: Apache Kafka, Twitter Storm & Elastic Search (ARC303...Infrastructure at Scale: Apache Kafka, Twitter Storm & Elastic Search (ARC303...
Infrastructure at Scale: Apache Kafka, Twitter Storm & Elastic Search (ARC303...Amazon Web Services
 
Spark + AI Summit 2019: Headaches and Breakthroughs in Building Continuous Ap...
Spark + AI Summit 2019: Headaches and Breakthroughs in Building Continuous Ap...Spark + AI Summit 2019: Headaches and Breakthroughs in Building Continuous Ap...
Spark + AI Summit 2019: Headaches and Breakthroughs in Building Continuous Ap...Landon Robinson
 
Apache Spark - A High Level overview
Apache Spark - A High Level overviewApache Spark - A High Level overview
Apache Spark - A High Level overviewKaran Alang
 
AWS re:Invent presentation: Unmeltable Infrastructure at Scale by Loggly
AWS re:Invent presentation: Unmeltable Infrastructure at Scale by Loggly AWS re:Invent presentation: Unmeltable Infrastructure at Scale by Loggly
AWS re:Invent presentation: Unmeltable Infrastructure at Scale by Loggly SolarWinds Loggly
 
Cloud Security Monitoring and Spark Analytics
Cloud Security Monitoring and Spark AnalyticsCloud Security Monitoring and Spark Analytics
Cloud Security Monitoring and Spark Analyticsamesar0
 
Streaming data analytics (Kinesis, EMR/Spark) - Pop-up Loft Tel Aviv
Streaming data analytics (Kinesis, EMR/Spark) - Pop-up Loft Tel Aviv Streaming data analytics (Kinesis, EMR/Spark) - Pop-up Loft Tel Aviv
Streaming data analytics (Kinesis, EMR/Spark) - Pop-up Loft Tel Aviv Amazon Web Services
 
Spark Saturday: Spark SQL & DataFrame Workshop with Apache Spark 2.3
Spark Saturday: Spark SQL & DataFrame Workshop with Apache Spark 2.3Spark Saturday: Spark SQL & DataFrame Workshop with Apache Spark 2.3
Spark Saturday: Spark SQL & DataFrame Workshop with Apache Spark 2.3Databricks
 
Architectures, Frameworks and Infrastructure
Architectures, Frameworks and InfrastructureArchitectures, Frameworks and Infrastructure
Architectures, Frameworks and Infrastructureharendra_pathak
 
What no one tells you about writing a streaming app
What no one tells you about writing a streaming appWhat no one tells you about writing a streaming app
What no one tells you about writing a streaming apphadooparchbook
 
What No One Tells You About Writing a Streaming App: Spark Summit East talk b...
What No One Tells You About Writing a Streaming App: Spark Summit East talk b...What No One Tells You About Writing a Streaming App: Spark Summit East talk b...
What No One Tells You About Writing a Streaming App: Spark Summit East talk b...Spark Summit
 
Global Big Data Conference Sept 2014 AWS Kinesis Spark Streaming Approximatio...
Global Big Data Conference Sept 2014 AWS Kinesis Spark Streaming Approximatio...Global Big Data Conference Sept 2014 AWS Kinesis Spark Streaming Approximatio...
Global Big Data Conference Sept 2014 AWS Kinesis Spark Streaming Approximatio...Chris Fregly
 
Paris Data Geek - Spark Streaming
Paris Data Geek - Spark Streaming Paris Data Geek - Spark Streaming
Paris Data Geek - Spark Streaming Djamel Zouaoui
 
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
 
Spark Streaming @ Scale (Clicktale)
Spark Streaming @ Scale (Clicktale)Spark Streaming @ Scale (Clicktale)
Spark Streaming @ Scale (Clicktale)Yuval Itzchakov
 

Ähnlich wie Extending Spark Streaming to Support Complex Event Processing (20)

Apache Spark Components
Apache Spark ComponentsApache Spark Components
Apache Spark Components
 
Headaches and Breakthroughs in Building Continuous Applications
Headaches and Breakthroughs in Building Continuous ApplicationsHeadaches and Breakthroughs in Building Continuous Applications
Headaches and Breakthroughs in Building Continuous Applications
 
Infrastructure at Scale: Apache Kafka, Twitter Storm & Elastic Search (ARC303...
Infrastructure at Scale: Apache Kafka, Twitter Storm & Elastic Search (ARC303...Infrastructure at Scale: Apache Kafka, Twitter Storm & Elastic Search (ARC303...
Infrastructure at Scale: Apache Kafka, Twitter Storm & Elastic Search (ARC303...
 
Spark + AI Summit 2019: Headaches and Breakthroughs in Building Continuous Ap...
Spark + AI Summit 2019: Headaches and Breakthroughs in Building Continuous Ap...Spark + AI Summit 2019: Headaches and Breakthroughs in Building Continuous Ap...
Spark + AI Summit 2019: Headaches and Breakthroughs in Building Continuous Ap...
 
Apache Spark - A High Level overview
Apache Spark - A High Level overviewApache Spark - A High Level overview
Apache Spark - A High Level overview
 
AWS re:Invent presentation: Unmeltable Infrastructure at Scale by Loggly
AWS re:Invent presentation: Unmeltable Infrastructure at Scale by Loggly AWS re:Invent presentation: Unmeltable Infrastructure at Scale by Loggly
AWS re:Invent presentation: Unmeltable Infrastructure at Scale by Loggly
 
Cloud Security Monitoring and Spark Analytics
Cloud Security Monitoring and Spark AnalyticsCloud Security Monitoring and Spark Analytics
Cloud Security Monitoring and Spark Analytics
 
Streaming data analytics (Kinesis, EMR/Spark) - Pop-up Loft Tel Aviv
Streaming data analytics (Kinesis, EMR/Spark) - Pop-up Loft Tel Aviv Streaming data analytics (Kinesis, EMR/Spark) - Pop-up Loft Tel Aviv
Streaming data analytics (Kinesis, EMR/Spark) - Pop-up Loft Tel Aviv
 
Apache Spark Core
Apache Spark CoreApache Spark Core
Apache Spark Core
 
Spark Saturday: Spark SQL & DataFrame Workshop with Apache Spark 2.3
Spark Saturday: Spark SQL & DataFrame Workshop with Apache Spark 2.3Spark Saturday: Spark SQL & DataFrame Workshop with Apache Spark 2.3
Spark Saturday: Spark SQL & DataFrame Workshop with Apache Spark 2.3
 
Apache Spark
Apache SparkApache Spark
Apache Spark
 
Typesafe spark- Zalando meetup
Typesafe spark- Zalando meetupTypesafe spark- Zalando meetup
Typesafe spark- Zalando meetup
 
Architectures, Frameworks and Infrastructure
Architectures, Frameworks and InfrastructureArchitectures, Frameworks and Infrastructure
Architectures, Frameworks and Infrastructure
 
Glint with Apache Spark
Glint with Apache SparkGlint with Apache Spark
Glint with Apache Spark
 
What no one tells you about writing a streaming app
What no one tells you about writing a streaming appWhat no one tells you about writing a streaming app
What no one tells you about writing a streaming app
 
What No One Tells You About Writing a Streaming App: Spark Summit East talk b...
What No One Tells You About Writing a Streaming App: Spark Summit East talk b...What No One Tells You About Writing a Streaming App: Spark Summit East talk b...
What No One Tells You About Writing a Streaming App: Spark Summit East talk b...
 
Global Big Data Conference Sept 2014 AWS Kinesis Spark Streaming Approximatio...
Global Big Data Conference Sept 2014 AWS Kinesis Spark Streaming Approximatio...Global Big Data Conference Sept 2014 AWS Kinesis Spark Streaming Approximatio...
Global Big Data Conference Sept 2014 AWS Kinesis Spark Streaming Approximatio...
 
Paris Data Geek - Spark Streaming
Paris Data Geek - Spark Streaming Paris Data Geek - Spark Streaming
Paris Data Geek - Spark Streaming
 
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
 
Spark Streaming @ Scale (Clicktale)
Spark Streaming @ Scale (Clicktale)Spark Streaming @ Scale (Clicktale)
Spark Streaming @ Scale (Clicktale)
 

Kürzlich hochgeladen

CALL ON ➥8923113531 🔝Call Girls Kakori Lucknow best sexual service Online ☂️
CALL ON ➥8923113531 🔝Call Girls Kakori Lucknow best sexual service Online  ☂️CALL ON ➥8923113531 🔝Call Girls Kakori Lucknow best sexual service Online  ☂️
CALL ON ➥8923113531 🔝Call Girls Kakori Lucknow best sexual service Online ☂️anilsa9823
 
W01_panagenda_Navigating-the-Future-with-The-Hitchhikers-Guide-to-Notes-and-D...
W01_panagenda_Navigating-the-Future-with-The-Hitchhikers-Guide-to-Notes-and-D...W01_panagenda_Navigating-the-Future-with-The-Hitchhikers-Guide-to-Notes-and-D...
W01_panagenda_Navigating-the-Future-with-The-Hitchhikers-Guide-to-Notes-and-D...panagenda
 
SyndBuddy AI 2k Review 2024: Revolutionizing Content Syndication with AI
SyndBuddy AI 2k Review 2024: Revolutionizing Content Syndication with AISyndBuddy AI 2k Review 2024: Revolutionizing Content Syndication with AI
SyndBuddy AI 2k Review 2024: Revolutionizing Content Syndication with AIABDERRAOUF MEHENNI
 
Right Money Management App For Your Financial Goals
Right Money Management App For Your Financial GoalsRight Money Management App For Your Financial Goals
Right Money Management App For Your Financial GoalsJhone kinadey
 
HR Software Buyers Guide in 2024 - HRSoftware.com
HR Software Buyers Guide in 2024 - HRSoftware.comHR Software Buyers Guide in 2024 - HRSoftware.com
HR Software Buyers Guide in 2024 - HRSoftware.comFatema Valibhai
 
The Real-World Challenges of Medical Device Cybersecurity- Mitigating Vulnera...
The Real-World Challenges of Medical Device Cybersecurity- Mitigating Vulnera...The Real-World Challenges of Medical Device Cybersecurity- Mitigating Vulnera...
The Real-World Challenges of Medical Device Cybersecurity- Mitigating Vulnera...ICS
 
call girls in Vaishali (Ghaziabad) 🔝 >༒8448380779 🔝 genuine Escort Service 🔝✔️✔️
call girls in Vaishali (Ghaziabad) 🔝 >༒8448380779 🔝 genuine Escort Service 🔝✔️✔️call girls in Vaishali (Ghaziabad) 🔝 >༒8448380779 🔝 genuine Escort Service 🔝✔️✔️
call girls in Vaishali (Ghaziabad) 🔝 >༒8448380779 🔝 genuine Escort Service 🔝✔️✔️Delhi Call girls
 
How To Troubleshoot Collaboration Apps for the Modern Connected Worker
How To Troubleshoot Collaboration Apps for the Modern Connected WorkerHow To Troubleshoot Collaboration Apps for the Modern Connected Worker
How To Troubleshoot Collaboration Apps for the Modern Connected WorkerThousandEyes
 
Reassessing the Bedrock of Clinical Function Models: An Examination of Large ...
Reassessing the Bedrock of Clinical Function Models: An Examination of Large ...Reassessing the Bedrock of Clinical Function Models: An Examination of Large ...
Reassessing the Bedrock of Clinical Function Models: An Examination of Large ...harshavardhanraghave
 
Software Quality Assurance Interview Questions
Software Quality Assurance Interview QuestionsSoftware Quality Assurance Interview Questions
Software Quality Assurance Interview QuestionsArshad QA
 
Tech Tuesday-Harness the Power of Effective Resource Planning with OnePlan’s ...
Tech Tuesday-Harness the Power of Effective Resource Planning with OnePlan’s ...Tech Tuesday-Harness the Power of Effective Resource Planning with OnePlan’s ...
Tech Tuesday-Harness the Power of Effective Resource Planning with OnePlan’s ...OnePlan Solutions
 
CALL ON ➥8923113531 🔝Call Girls Badshah Nagar Lucknow best Female service
CALL ON ➥8923113531 🔝Call Girls Badshah Nagar Lucknow best Female serviceCALL ON ➥8923113531 🔝Call Girls Badshah Nagar Lucknow best Female service
CALL ON ➥8923113531 🔝Call Girls Badshah Nagar Lucknow best Female serviceanilsa9823
 
Optimizing AI for immediate response in Smart CCTV
Optimizing AI for immediate response in Smart CCTVOptimizing AI for immediate response in Smart CCTV
Optimizing AI for immediate response in Smart CCTVshikhaohhpro
 
Shapes for Sharing between Graph Data Spaces - and Epistemic Querying of RDF-...
Shapes for Sharing between Graph Data Spaces - and Epistemic Querying of RDF-...Shapes for Sharing between Graph Data Spaces - and Epistemic Querying of RDF-...
Shapes for Sharing between Graph Data Spaces - and Epistemic Querying of RDF-...Steffen Staab
 
Short Story: Unveiling the Reasoning Abilities of Large Language Models by Ke...
Short Story: Unveiling the Reasoning Abilities of Large Language Models by Ke...Short Story: Unveiling the Reasoning Abilities of Large Language Models by Ke...
Short Story: Unveiling the Reasoning Abilities of Large Language Models by Ke...kellynguyen01
 
Learn the Fundamentals of XCUITest Framework_ A Beginner's Guide.pdf
Learn the Fundamentals of XCUITest Framework_ A Beginner's Guide.pdfLearn the Fundamentals of XCUITest Framework_ A Beginner's Guide.pdf
Learn the Fundamentals of XCUITest Framework_ A Beginner's Guide.pdfkalichargn70th171
 
Try MyIntelliAccount Cloud Accounting Software As A Service Solution Risk Fre...
Try MyIntelliAccount Cloud Accounting Software As A Service Solution Risk Fre...Try MyIntelliAccount Cloud Accounting Software As A Service Solution Risk Fre...
Try MyIntelliAccount Cloud Accounting Software As A Service Solution Risk Fre...MyIntelliSource, Inc.
 
The Ultimate Test Automation Guide_ Best Practices and Tips.pdf
The Ultimate Test Automation Guide_ Best Practices and Tips.pdfThe Ultimate Test Automation Guide_ Best Practices and Tips.pdf
The Ultimate Test Automation Guide_ Best Practices and Tips.pdfkalichargn70th171
 

Kürzlich hochgeladen (20)

CALL ON ➥8923113531 🔝Call Girls Kakori Lucknow best sexual service Online ☂️
CALL ON ➥8923113531 🔝Call Girls Kakori Lucknow best sexual service Online  ☂️CALL ON ➥8923113531 🔝Call Girls Kakori Lucknow best sexual service Online  ☂️
CALL ON ➥8923113531 🔝Call Girls Kakori Lucknow best sexual service Online ☂️
 
W01_panagenda_Navigating-the-Future-with-The-Hitchhikers-Guide-to-Notes-and-D...
W01_panagenda_Navigating-the-Future-with-The-Hitchhikers-Guide-to-Notes-and-D...W01_panagenda_Navigating-the-Future-with-The-Hitchhikers-Guide-to-Notes-and-D...
W01_panagenda_Navigating-the-Future-with-The-Hitchhikers-Guide-to-Notes-and-D...
 
SyndBuddy AI 2k Review 2024: Revolutionizing Content Syndication with AI
SyndBuddy AI 2k Review 2024: Revolutionizing Content Syndication with AISyndBuddy AI 2k Review 2024: Revolutionizing Content Syndication with AI
SyndBuddy AI 2k Review 2024: Revolutionizing Content Syndication with AI
 
Right Money Management App For Your Financial Goals
Right Money Management App For Your Financial GoalsRight Money Management App For Your Financial Goals
Right Money Management App For Your Financial Goals
 
HR Software Buyers Guide in 2024 - HRSoftware.com
HR Software Buyers Guide in 2024 - HRSoftware.comHR Software Buyers Guide in 2024 - HRSoftware.com
HR Software Buyers Guide in 2024 - HRSoftware.com
 
CHEAP Call Girls in Pushp Vihar (-DELHI )🔝 9953056974🔝(=)/CALL GIRLS SERVICE
CHEAP Call Girls in Pushp Vihar (-DELHI )🔝 9953056974🔝(=)/CALL GIRLS SERVICECHEAP Call Girls in Pushp Vihar (-DELHI )🔝 9953056974🔝(=)/CALL GIRLS SERVICE
CHEAP Call Girls in Pushp Vihar (-DELHI )🔝 9953056974🔝(=)/CALL GIRLS SERVICE
 
The Real-World Challenges of Medical Device Cybersecurity- Mitigating Vulnera...
The Real-World Challenges of Medical Device Cybersecurity- Mitigating Vulnera...The Real-World Challenges of Medical Device Cybersecurity- Mitigating Vulnera...
The Real-World Challenges of Medical Device Cybersecurity- Mitigating Vulnera...
 
call girls in Vaishali (Ghaziabad) 🔝 >༒8448380779 🔝 genuine Escort Service 🔝✔️✔️
call girls in Vaishali (Ghaziabad) 🔝 >༒8448380779 🔝 genuine Escort Service 🔝✔️✔️call girls in Vaishali (Ghaziabad) 🔝 >༒8448380779 🔝 genuine Escort Service 🔝✔️✔️
call girls in Vaishali (Ghaziabad) 🔝 >༒8448380779 🔝 genuine Escort Service 🔝✔️✔️
 
How To Troubleshoot Collaboration Apps for the Modern Connected Worker
How To Troubleshoot Collaboration Apps for the Modern Connected WorkerHow To Troubleshoot Collaboration Apps for the Modern Connected Worker
How To Troubleshoot Collaboration Apps for the Modern Connected Worker
 
Reassessing the Bedrock of Clinical Function Models: An Examination of Large ...
Reassessing the Bedrock of Clinical Function Models: An Examination of Large ...Reassessing the Bedrock of Clinical Function Models: An Examination of Large ...
Reassessing the Bedrock of Clinical Function Models: An Examination of Large ...
 
Software Quality Assurance Interview Questions
Software Quality Assurance Interview QuestionsSoftware Quality Assurance Interview Questions
Software Quality Assurance Interview Questions
 
Tech Tuesday-Harness the Power of Effective Resource Planning with OnePlan’s ...
Tech Tuesday-Harness the Power of Effective Resource Planning with OnePlan’s ...Tech Tuesday-Harness the Power of Effective Resource Planning with OnePlan’s ...
Tech Tuesday-Harness the Power of Effective Resource Planning with OnePlan’s ...
 
CALL ON ➥8923113531 🔝Call Girls Badshah Nagar Lucknow best Female service
CALL ON ➥8923113531 🔝Call Girls Badshah Nagar Lucknow best Female serviceCALL ON ➥8923113531 🔝Call Girls Badshah Nagar Lucknow best Female service
CALL ON ➥8923113531 🔝Call Girls Badshah Nagar Lucknow best Female service
 
Optimizing AI for immediate response in Smart CCTV
Optimizing AI for immediate response in Smart CCTVOptimizing AI for immediate response in Smart CCTV
Optimizing AI for immediate response in Smart CCTV
 
Shapes for Sharing between Graph Data Spaces - and Epistemic Querying of RDF-...
Shapes for Sharing between Graph Data Spaces - and Epistemic Querying of RDF-...Shapes for Sharing between Graph Data Spaces - and Epistemic Querying of RDF-...
Shapes for Sharing between Graph Data Spaces - and Epistemic Querying of RDF-...
 
Short Story: Unveiling the Reasoning Abilities of Large Language Models by Ke...
Short Story: Unveiling the Reasoning Abilities of Large Language Models by Ke...Short Story: Unveiling the Reasoning Abilities of Large Language Models by Ke...
Short Story: Unveiling the Reasoning Abilities of Large Language Models by Ke...
 
Learn the Fundamentals of XCUITest Framework_ A Beginner's Guide.pdf
Learn the Fundamentals of XCUITest Framework_ A Beginner's Guide.pdfLearn the Fundamentals of XCUITest Framework_ A Beginner's Guide.pdf
Learn the Fundamentals of XCUITest Framework_ A Beginner's Guide.pdf
 
Try MyIntelliAccount Cloud Accounting Software As A Service Solution Risk Fre...
Try MyIntelliAccount Cloud Accounting Software As A Service Solution Risk Fre...Try MyIntelliAccount Cloud Accounting Software As A Service Solution Risk Fre...
Try MyIntelliAccount Cloud Accounting Software As A Service Solution Risk Fre...
 
The Ultimate Test Automation Guide_ Best Practices and Tips.pdf
The Ultimate Test Automation Guide_ Best Practices and Tips.pdfThe Ultimate Test Automation Guide_ Best Practices and Tips.pdf
The Ultimate Test Automation Guide_ Best Practices and Tips.pdf
 
Microsoft AI Transformation Partner Playbook.pdf
Microsoft AI Transformation Partner Playbook.pdfMicrosoft AI Transformation Partner Playbook.pdf
Microsoft AI Transformation Partner Playbook.pdf
 

Extending Spark Streaming to Support Complex Event Processing

  • 1. 이름 김병진, 권오찬 Extending Spark Streaming to Support Complex Event Processing 소속 삼성전자 2015. 10. 27.
  • 2. Agenda • Background • Motivation • Streaming SQL • Auto Scaling • Future Work
  • 3. Background - Spark • Apache Spark is a high performance data processing platform • Use a distributed memory cache (RDD*) • Process batch and stream data in the common platform • Be developed by 700+ contributors 3x faster than Storm (Stream, 30 Nodes) 100x faster than Hadoop (Batch, 50 Nodes, Clustering Alg.) *RDD: Resilient Distributed Datasets
  • 4. Background – Spark RDD • Resilient Distributed Datasets • RDDs are immutable • Transformations are lazy operations which build a RDD’s lineage graph • E.g.: map, filter, join, union • Actions launch a computation to return a value or write data • E.g.: count, collect, reduce, save • The scheduler builds a DAG of stages to execute • If a task fails, spark re-runs it on another node as long as its stage’s parent are still available
  • 5. Background - Spark Modules • Spark Streaming • Divide stream data into micro-batches • Compute the batches with fault tolerance • Spark SQL • Support SQL to handle structured data • Provide a common way to access a variety of data sources (Hive, Avro, Parquet, ORC, JSON, and JDBC) Spark Spark Streaming batches of X seconds live data stream processed results DStreams RDDs map, reduce, count, …
  • 6. Motivation – Support CEP in Spark • Current spark is not enough to support CEP* • Do not support continuous query language to process stream data • Do not support auto-scaling to elastically allocate resources • We have solved the issues: • Extend Intel’s Streaming SQL package • Improve performance by optimizing time-based windowed aggregation • Support query chains by implementing “Insert Into” queries • Implement elastic-seamless resource allocation • Can do auto-scale in/out *Complex Event Processing
  • 8. Streaming SQL - Intel • Streaming SQL is a third party library of Spark Packages • Build on top of Spark Streaming and Spark SQL Catalyst • Manipulate stream data like static structured data in database • Process queries continuously • Support time based windowing join/aggregation queries http://spark-packages.org/package/Intel-bigdata/spark-streamingsql Streaming SQL
  • 9. Streaming SQL Query Schema DStream Optimized Logical Plan Streaming SQL - Plans • Logical Plan • Modify streaming SQL optimizer • Add windowed aggregate logical plan • Physical Plan • Add windowed aggregate physical plan to call new WindowedStateDStream class • Implement new expression functions (Count, Sum, Average, Min, Max, Distinct) • Develop FixedSizedAggregator for efficiency which is the concept of IBM InfoSphere Streams Samsung Stream Plan Intel Logical Plan Physical Plan
  • 10. Streaming SQL – Windowed State • WindowedStateDStream class • Modify StateDStream class to support windowed computing • Add inverse update function to evaluate old values • Add filter function to remove obsolete keys Intel Samsung Compute all elements in the window time 1 time 2 time 3 time 4 time 5 Original DStream Windowed State DStream time 1 time 2 time 3 time 4 time 5 Original DStream Windowed DStream Compute only Delta (old and new elements) window-based operation Update (in) InvUpdate (out) State: Array[AggregateFunction] Count Update InvUpdate Update InvUpdate State Sum Update InvUpdate State Avg Update InvUpdate State Min Update InvUpdate State Max Update InvUpdate State
  • 11. Streaming SQL – Fixed Sized Aggregator • Fixed-Sized Aggregator* of IBM InfoSphere Streams • Use fixed sized binary tree • Maintain leaf nodes as a circular buffer using front and back pointers • Very efficient for non-invertable expressions (e.g. Min, Max) • Example - Min Aggregator 4 7 3 4 3 3 4 7 3 2 4 2 2 7 3 2 7 2 2 9 7 3 2 7 2 2 4 4 4 4 7 4 4 *K. Tangwongsan, M. Hirzel, S. Schneider, K-L. Wu, “General incremental sliding-window aggregation,” In VLDB, 2015.
  • 12. Streaming SQL – Aggregate Functions • Windowed Aggregate Functions • Add invUpdate function • Reduce objects for efficient serialization • Implement Fixed-Sized Aggregator for Min, Max functions Aggregate Functions State Update InvUpdate Count countValue: Long Increase countValue Decrease countValue Sum sumValue: Any Add to sumValue Subtract to sumValue Average sumValue: Any countValue: Long Increase countValue Add to sumValue Decrease countValue Subtract to sumValue Min fat: FixedSizedAggregator Insert minimum to fat Remove the oldest from fat Max fat: FixedSizedAggregator Insert maximum to fat Remove the oldest from fat CountDistinct distinctMap: mutable.HashMap[Row, Long] Increase count of distinct value in map Decrease count of distinct value in map
  • 13. Streaming SQL – Insert Into • Support “Insert Into” query • Implement Data Sources API • Implement the insert function in Kafka relation • Modified some physical plans to support streaming • Modified physical planning strategies to assign a specific plan • Convert current RDD to a DataFrame and then insert it • Support query chaining  The result a query can be reused by multiple queries Example2 Kafka Kafka INSERT INTO TABLE Example1 SELECT duid, time FROM ParsedTable INSERT INTO TABLE Example2 SELECT duid, COUNT (*) FROM Example1 GROUP BY duidParsed Table Example1 Kafka Kafka Relation InsertIntoStreamSource StreamExecutedCommandInsertIntoTable Kafka Logical Plan Physical Plan Kafka Relation DataFrame
  • 14. Streaming SQL – Evaluation • Experimental Environment • Processing Node: Intel i5 2.67GHz, 4 Cores, 4GB per node Spark Cluster: 7 Nodes Kafka Cluster: 3 Nodes • Test Query: SELECT t.word, COUNT(t.word), SUM(t.num), AVG(t.num), MIN(t.num), MAX(t.num) FROM (SELECT * FROM t_kafka) OVER (WINDOW 'x' SECONDS, SLIDE ‘1' SECONDS) AS t GROUP BY t.word • Default Set: 100 EPS, 100 Keys, 20 sec Window, 1 sec Slide, 10 Executors, 10 Reducers Spark Cluster Test Agent Kafka Cluster Spark UI
  • 15. Streaming SQL – Evaluation • Test Result • Show low processing delays despite of heavy event loads or large-sized windows • Need memory optimization 0 200 400 600 800 1000 1200 1400 20 40 60 80 100 Delay(ms) Window Size (sec) Intel Samsung 0 1000 2000 3000 4000 5000 6000 7000 8000 9000 10000 20 40 60 80 100 Memory(KB) Window Size (sec) Intel Samsung 0 1000 2000 3000 4000 5000 10 100 1000 10000 Delay(ms) EPS Intel Samsung 0 2000 4000 6000 8000 10000 12000 10 100 1000 10000 Memory(KB) EPS Intel Samsung
  • 17. Time-varying Event Rate in Real World • Spark Streaming • Data can be ingested from many sources like Kafka, Flume, Twitter, etc. • Live input data streams are divided into batches, and they are processed • In streaming application, event rate may change frequently over time • How can this be dealt with? < Event Rate > < Batch Processing Time> High event rate Can’t be processed in real-time
  • 18. Spark AS-IS • Spark currently supports dynamic resource allocation on Yarn (SPARK-3174) and coarse-grained Mesos (SPARK-6287) • Existing Dynamic Resource Allocation is not optimized for streaming • Even though event rate becomes smaller, executors may not be removed due to scheduling policy • It is difficult to determine appropriate configuration parameters • Backpressure (SPARK-7398) • Enable the Spark streaming to control the receiving rate dynamically for handling bursty input streams • Not real-time Driver Task queue Task Executor #1 ... Executor #2 Executor #n • Upper/lower bound for the number of executors • spark.dynamicAllocation.minExecutors • spark.dynamicAllocation.maxExecutors • Scale-out condition • schedulerBacklogTimeout < staying time of a task in task queue • Scale-in condition • executorIdleTimeout < staying time of an executor in idle state
  • 19. • Goal • Allocate resources to streaming applications dynamically as the rate of incoming events varies over time • Enable the applications to meet real-time deadlines • Utilize the resource efficiently • Cloud Architecture Elastic-seamless Resource Allocation Flint* *Flint: our spark job manager
  • 20. • Spark currently supports three cluster managers • Standalone, Apache Mesos, Hadoop Yarn • Spark on Mesos • Fine-grained mode • Each Spark task runs as a separate Mesos task. • Launching overhead is big, so it is not suitable for streaming. • Coarse-grained mode • Launch only one long-running Spark task on each Mesos machine • Cannot scale-out more than the number of Mesos machines • Cannot control executor resource • Only available for total resource • Both of two modes have some problems to achieve our goal Spark Deployment Architecture
  • 21. Flint Architecture Overview Slave 1 Slave 2 Slave n . . . MESOS Cluster Cluster 1 YARN Cluster n YARN . . . . . . NM 1 NM 2 . . . NM n NM 1 NM 2 NM n . . . App 1 SPARK App n SPARK EXEC 1 EXEC 2 EXEC n. . . EXEC 1 EXEC 2 EXEC n. . . FLINT REST Manager Scheduler Application Manager Application Handler Application Handler Application Handler Deploy Manager Log Manager • Marathon, Zookeeper, ETCD, HDFS Spark on on-demand Yarn
  • 22. • Job submission process 1. Request to deploy dockerized YARN via Marathon 2. Launch ResourceManager, NodeManagers for Spark driver/executors 3. Check ResourceManager status 4. Submit Spark Job and check Job status 5. Watch Spark driver status and get Spark endpoint Flint Architecture: Job Submission Slave 1 Slave n . . . MESOS Cluster Slave 1 Slave n . . . MESOS Cluster Cluster YARN NM 1 NM 2 . . . NM n Slave 1 Slave n . . . MESOS Cluster Cluster YARN NM 1 NM 2 . . . NM n App SPARK EXEC 1 EXEC 2 EXEC n. . . Create Yarn Cluster Submit Spark Job
  • 23. • Scale-out Process 1. Request to increase the instance of NodeManager via Marathon 2. Launch new NodeManager 3. Scale-out Spark executor Flint Architecture: Scale-out Slave 1 Slave n . . . MESOS Cluster Cluster YARN NM 1 . . . NM n App SPARK EXEC 1 EXEC n. . . Slave 1 Slave n . . . MESOS Cluster Cluster YARN NM 1 . . . NM n App SPARK EXEC 1 EXEC n. . . NM 2 Slave 1 Slave n . . . MESOS Cluster Cluster YARN NM 1 . . . NM n NM 2 App SPARK EXEC 1 EXEC n. . . EXEC 2 Request new NodeManager Scale-out Spark executor
  • 24. • Scale-in Process 1. Get Executor’s info and select spark victims 2. Inactivate spark victims and kill them after n x batch interval 3. Get Yarn victims and decommission NodeManager via ResourceManager 4. Get mesos task id of Yarn victims and kill mesos tasks of victims Flint Architecture: Scale-in Slave 1 Slave n . . . MESOS Cluster Cluster YARN NM 1 . . . NM n NM 2 App SPARK EXEC 1 EXEC n. . . EXEC 2 Slave 1 Slave n . . . MESOS Cluster Cluster YARN NM 1 . . . NM n NM 2 App SPARK EXEC 1 EXEC n. . . Slave 1 Slave n . . . MESOS Cluster Cluster YARN NM 1 . . . NM n NM 2 App SPARK EXEC 1 EXEC n. . . Scale-in Spark executor Remove NodeManager
  • 25. • Auto-scaling Mechanism • Real-time constraint • Batch processing time < Batch interval • Scale-out condition • α x batch interval < batch processing delay • Scale-in condition • β x batch interval > batch processing delay Flint Architecture: Auto-scaling ( 0 < β < α ≤ 1 ) Processing Time Batch Interval α x Batch Intervalβ x Batch Interval Scale-in Scale-out 0 Time
  • 26. • Timeout & Retry for Killed Executor • Although Spark executor is killed by admin, Spark re-tries to connect it. Spark Issues: Scale-in (1/3) Scale-in Scheduling Delay Event Rate Processing Delay Driver Exec* scale-in Exec Task RDD req 1st try 2nd try 3rd try fail Processing Delay • spark.shuffle.io.maxRetries = 3 • spark.shuffle.io.retryWait = 5s
  • 27. • Timeout & Retry for Killed Executor • Advertisement for killed executor • Inactivate executor before scale-in • Inactivate executor and kill them after n x batch interval • During inactivating stage, the executor does not receive any task from driver. Spark Issues: Scale-in (2/3) Driver Exec* scale-in Exec Task RDD req fail advertise Fast Fail
  • 28. • Data Locality • Spark scheduler consider data locality • Processing/Node/Rack locality • spark.locality.wait = 3s • However, waiting time for locality is big burden to streaming applications • The waiting time should be much less than batch interval for streaming • Otherwise, streaming may not be processed in real-time • Thus, Flint overrides “spark.locality.wait” to very small value if application type is streaming Spark Issues: Scale-in (3/3) Scheduler Cluster TTT T … 3s10ms (max) rsc alloc T
  • 29. • Data Localization • During scale-out process, new Yarn container performs to localize some data (e.g. spark jar, application jar) • Data localization incurs high disk I/O, so • Solution • Prefetch if possible • Disk isolation, SSD Spark Issues: Scale-out (1/2) Existing App New App Disk Access disk during localization (50-70 MB/sec) Performance Degradation
  • 30. • RDD Replication • When a new executor is added, a receiver requests to replicate received RDD blocks to the executor • New executor does not ready to receive RDD blocks during an initialization • At this situation, the receiver waits until new executor is ready • Solution • A receiver does not replicate RDD blocks to new executor during initialization • E.g., spark.blockmanager.peer.bootstrap = 10s Spark Issues: Scale-out (2/2) Kafka R RT R R Executor R R T R Executor 1 R T R Executor 2 RT T Receiver Task Task R RDDReplicate factor = 2
  • 31. • Demo Environment • Data source: Kafka • Auto-scaling parameter • α=0.4, β=0.9 • SQL query • SELECT t.word, COUNT(DISTINCT t.num), SUM(t.num), AVG(t.num), MIN(t.num), MAX(t.num) FROM (SELECT * FROM t_kafka) OVER (WINDOW 300 SECONDS, SLIDE 3 SECONDS) AS t GROUP BY t.word Demo (1/2) Task Task Producer Rcvr 8 3 4 2 5 7 9 1 0 3 7 8 Task Driver 0: 23 1: 34 2: 41 … Flint Monitoring Scale-in/Out
  • 33. Future Work • Streaming SQL • Apply Tungsten Framework • Small-sized object, code gen, GC free memory management • Share RDDs • Map stream tables to RDD • Auto Scaling • Support to dynamically allocate resources for batch (non-streaming) applications • Support unified schedulers for heterogeneous applications • Batch, streaming, ad-hoc