SlideShare ist ein Scribd-Unternehmen logo
1 von 54
Downloaden Sie, um offline zu lesen
A Deep Dive into
Structured Streaming
Tathagata “TD” Das
@tathadas
Spark Meetup @ Bloomberg
27 September 2016
Streaming in Apache Spark
Spark Streaming changed how people write streaming
apps
2
SQL
Streamin
g
MLlib
Spark Core
GraphX
Functional, concise and
expressive
Fault-tolerant state management
Unified stack with batch
processing
More than 50% users consider most important part of Apache Spark
Streaming apps are
growing more complex
3
Streaming computations
don’t run in isolation
Need to interact with batch data,
interactive analysis, machine learning, etc.
Use case: IoT Device Monitoring
IoT events
from
Kafka
ETL into long term
storage
- Prevent data loss
- Prevent duplicates
Status monitoring
- Handle late data
- Aggregate on
windows on event
time
Interactively
debug
issues
- consistency
event stream
Anomaly detection
- Learn models offline
- Use online + continuous
learning
Use case: IoT Device Monitoring
IoT events
from
Kafka
ETL into long term
storage
- Prevent data loss
- Prevent duplicates
Status monitoring
- Handle late data
- Aggregate on
windows on event
time
Interactively
debug
issues
- consistency
event stream
Anomaly detection
- Learn models offline
- Use online + continuous
learning
Continuous Applications
Not just streaming any more
1. Processing with event-time, dealing with late data
- DStream API exposes batch time, hard to incorporate event-
time
2. Interoperate streaming with batch AND interactive
- RDD/DStream has similar API, but still requires translation
3. Reasoning about end-to-end guarantees
- Requires carefully constructing sinks that handle failures
correctly
- Data consistency in the storage while being updated
Pain points with DStreams
Structured Streaming
The simplest way to perform streaming analytics
is not having to reason about streaming at all
New Model Trigger: every 1 sec
1 2 3
Time
data up
to 1
Input data up
to 2
data up
to 3
Quer
y
Input: data from source as an
append-only table
Trigger: how frequently to check
input for new data
Query: operations on input
usual map/filter/reduce
new window, session ops
New Model Trigger: every 1 sec
1 2 3
output
for data
up to 1
Result
Quer
y
Time
data up
to 1
Input data up
to 2
output
for data
up to 2
data up
to 3
output
for data
up to 3
Result: final operated table
updated every trigger interval
Output: what part of result to write
to data sink after every
trigger
Complete output: Write full result table every
time
Output
complet
e
output
New Model Trigger: every 1 sec
1 2 3
output
for data
up to 1
Result
Quer
y
Time
data up
to 1
Input data up
to 2
output
for data
up to 2
data up
to 3
output
for data
up to 3
Output
delta
output
Result: final operated table
updated every trigger
interval
Output: what part of result to write
to data sink after every
trigger
Complete output: Write full result table
every time
Delta output: Write only the rows that
changed
in result from previous batch
Static, bounded
data
Streaming,
unbounded data
Single API !
API - Dataset/DataFrame
14
Dataset
A distributed bounded or unbounded collection of
data with a known schema
A high-level abstraction for selecting, filtering,
mapping, reducing, and aggregating structured data
Common API across Scala, Java, Python, R
Standards based:
Supports most popular
constructs in HiveQL,
ANSI SQL
Compatible: Use
JDBC to connect with
popular BI tools
15
Datasets with SQL
SELECT type, avg(signal)
FROM devices
GROUP BY type
Concise: great for ad-
hoc interactive
analysis
Interoperable: based
on R / pandas, easy to
go back and forth
16
Dynamic Datasets (DataFrames)
ctxt.table("device-data")
.groupBy("type")
.agg("type", avg("signal"))
.map(lambda …)
.collect()
17
Statically-typed Datasets
No boilerplate:
automatically convert
to/from domain objects
Safe: compile time
checks for correctness
// Compute histogram of age by name.
val hist = ds.groupBy(_.type).mapGroups {
case (type, data: Iter[DeviceData]) =>
val buckets = new Array[Int](10)
data.map(_.signal).foreach { a =>
buckets(a / 10) += 1
}
(type, buckets)
}
case class DeviceData(type:String, signal:Int)
// Convert data to Java objects
val ds: Dataset[DeviceData] =
ctx.read.json("data.json").as[Device]
Unified, Structured APIs in Spark 2.0
18
SQL DataFrames Datasets
Syntax
Errors
Analysis
Errors
Runtime Compile
Time
Runtime
Compile
Time
Compile
Time
Runtime
Batch ETL with DataFrames
input = spark.read
.format("json")
.load("source-path")
result = input
.select("device", "signal")
.where("signal > 15")
result.write
.format("parquet")
.save("dest-path")
Read from Json file
Select some devices
Write to parquet file
Streaming ETL with DataFrames
input = spark.readStream
.format("json")
.load("source-path")
result = input
.select("device", "signal")
.where("signal > 15")
result.writeStream
.format("parquet")
.start("dest-path")
Read from Json file stream
Replace read with readStream
Select some devices
Code does not change
Write to Parquet file stream
Replace save() with start()
Streaming ETL with DataFrames
read…stream() creates a streaming
DataFrame, does not start any of the
computation
write…startStream() defines where
& how to output the data and starts
the processing
input = spark.readStream
.format("json")
.load("source-path")
result = input
.select("device", "signal")
.where("signal > 15")
result.writeStream
.format("parquet")
.start("dest-path")
Streaming ETL with DataFrames
1 2 3
Result
[append-only table]
Input
Output
[append mode]
new
rows in
result of
2
new
rows in
result of
3
input = spark.readStream
.format("json")
.load("source-path")
result = input
.select("device", "signal")
.where("signal > 15")
result.writeStream
.format("parquet")
.start("dest-path")
Demo
Continuous Aggregations
Continuously compute average
signal across all devices
Continuously compute average
signal of each type of device
24
input.avg("signal")
input.groupBy("device-type")
.avg("signal")
Continuous Windowed Aggregations
25
input.groupBy(
$"device-type",
window($"event-time-col", "10 min"))
.avg("signal")
Continuously compute
average signal of each
type of device in last 10
minutes using event-
time
Simplifies event-time stream processing (not possible in
DStreams)
Works on both, streaming and batch jobs
Joining streams with static data
kafkaDataset = spark.readStream
.format("kafka")
.option("subscribe", "iot-updates")
.option("kafka.bootstrap.servers", ...)
.load()
staticDataset = ctxt.read
.jdbc("jdbc://", "iot-device-info")
joinedDataset =
kafkaDataset.join(
staticDataset, "device-type")
26
Join streaming data from Kafka
with static data via JDBC to
enrich the streaming data …
… without having to think that
you are joining streaming data
Output Modes
Defines what is outputted every time there is a
trigger
Different output modes make sense for different
queries
27
input.select("device", "signal")
.writeStream
.outputMode("append")
.format("parquet")
.start("dest-path")
Append mode with
non-aggregation
queries
input.agg(count("*"))
.writeStream
.outputMode("complete")
.format("parquet")
.start("dest-path")
Complete mode with
aggregation queries
Query Management
query = result.write
.format("parquet")
.outputMode("append")
.start("dest-path")
query.stop()
query.awaitTermination()
query.exception()
query.sourceStatuses()
query.sinkStatus()
28
query: a handle to the running streaming
computation for managing it
- Stop it, wait for it to terminate
- Get status
- Get error, if terminated
Multiple queries can be active at the same
time
Each query has unique name for keeping
track
Logically:
Dataset operations on table
(i.e. as easy to understand as batch)
Physically:
Spark automatically runs the query in
streaming fashion
(i.e. incrementally and continuously)
DataFrame
Logical Plan
Continuous,
incremental execution
Catalyst optimizer
Query Execution
Structured Streaming
High-level streaming API built on Datasets/DataFrames
Event time, windowing, sessions, sources & sinks
End-to-end exactly once semantics
Unifies streaming, interactive and batch queries
Aggregate data in a stream, then serve using JDBC
Add, remove, change queries at runtime
Build and apply ML models
What can you do with this that’s hard
with other engines?
True unification
Combines batch + streaming + interactive workloads
Same code + same super-optimized engine for everything
Flexible API tightly integrated with the engine
Choose your own tool - Dataset/DataFrame/SQL
Greater debuggability and performance
Benefits of Spark
in-memory computing, elastic scaling, fault-tolerance, straggler mitigation, …
Underneath the Hood
Batch Execution on Spark SQL
33
DataFrame
/
Dataset
Logical
Plan
Abstract
representatio
n of query
Batch Execution on Spark SQL
34
DataFrame
/
Dataset
Logical
Plan
Planne
r
SQL AST
DataFram
e
Unresolve
d Logical
Plan
Logical
Plan
Optimized
Logical
Plan
RDDs
Selected
Physical
Plan
Analysis
Logical
Optimization
Physical
Planning
CostModel
Physical
Plans
Code
Generation
CatalogDataset
Helluvalot of magic!
Batch Execution on Spark SQL
35
DataFrame
/
Dataset
Logical
Plan
Execution Plan
Planne
r
Run super-optimized
Spark jobs to compute
results
Bytecode generation
JVM intrinsics, vectorization
Operations on serialized data
Code Optimizations Memory Optimizations
Compact and fast encoding
Offheap memory
Project Tungsten - Phase 1 and 2
Continuous Incremental Execution
Planner knows how to convert
streaming logical plans to a
continuous series of incremental
execution plans, for each
processing the next chunk of
streaming data
36
DataFrame
/
Dataset
Logical
Plan
Incremental
Execution Plan
1
Incremental
Execution Plan
2
Incremental
Execution Plan
3
Planne
r
Incremental
Execution Plan
4
Continuous Incremental Execution
37
Planner
Incremental
Execution 2
Offsets: [106-197] Count: 92
Planner polls for
new data from
sources
Incremental
Execution 1
Offsets: [19-105] Count: 87
Incrementally executes
new data and writes to
sink
Continuous Aggregations
Maintain running aggregate as in-memory state
backed by WAL in file system for fault-tolerance
38
state data generated and
used across incremental
executions
Incremental
Execution 1
state:
87
Offsets: [19-105] Running Count: 87
memory
Incremental
Execution 2
state:
179
Offsets: [106-179]
Count: 87+92 =
179
Fault-tolerance
All data and metadata
in the system needs to
be recoverable /
replayable
state
Planner
source sink
Incremental
Execution 1
Incremental
Execution 2
Fault-tolerance
Fault-tolerant Planner
Tracks offsets by writing the
offset range of each
execution to a write ahead log
(WAL) in HDFS
state
Planner
source sink
Offsets written to
fault-tolerant WAL
before execution
Incremental
Execution 2
Incremental
Execution 1
Fault-tolerance
Fault-tolerant Planner
Tracks offsets by writing the
offset range of each
execution to a write ahead log
(WAL) in HDFS
state
Planner
source sink
Failed planner fails
current execution
Incremental
Execution 2
Incremental
Execution 1
Failed
Execution
Failed
Planner
Fault-tolerance
Fault-tolerant Planner
Tracks offsets by writing the
offset range of each
execution to a write ahead log
(WAL) in HDFS
Reads log to recover from
failures, and re-execute exact
range of offsets
state
Restarte
d
Planner
source sink
Offsets read back
from WAL
Incremental
Execution 1
Same executions
regenerated from
offsets
Failed
Execution
Incremental
Execution 2
Fault-tolerance
Fault-tolerant Sources
Structured streaming sources
are by design replayable (e.g.
Kafka, Kinesis, files) and
generate the exactly same
data given offsets recovered
by planner
state
Planner
sink
Incremental
Execution 1
Incremental
Execution 2
source
Replayable
source
Fault-tolerance
Fault-tolerant State
Intermediate "state data" is a
maintained in versioned, key-
value maps in Spark workers,
backed by HDFS
Planner makes sure "correct
version" of state used to re-
execute after failure
Planner
source sink
Incremental
Execution 1
Incremental
Execution 2
state
state is fault-tolerant with WAL
Fault-tolerance
Fault-tolerant Sink
Sink are by design
idempotent, and handles re-
executions to avoid double
committing the output
Planner
source
Incremental
Execution 1
Incremental
Execution 2
state
sink
Idempotent
by design
46
offset tracking in WAL
+
state management
+
fault-tolerant sources and sinks
=
end-to-end
exactly-once
guarantees
47
Fast, fault-tolerant, exactly-once
stateful stream processing
without having to reason about streaming
Blogs for Deeper Technical Dive
Blog 1:
https://databricks.com/blog/2016/07/2
8/continuous-applications-evolving-
streaming-in-apache-spark-2-0.html
Blog 2:
https://databricks.com/blog/2016/07/2
8/structured-streaming-in-apache-
spark.html
Comparison with Other Engines
49
Read the blog to understand this table
Current Status: Spark 2.0
Basic infrastructure and API
- Event time, windows,
aggregations
- Append and Complete output
modes
- Support for a subset of batch
queries
Source and sink
- Sources: Files
- Sinks: Files and in-memory table
Experimental release to
set the future direction
Not ready for production
but good to experiment
with and provide
feedback
In Progress
Future Direction
Metrics
Stability and scalability
Support for more queries
Watermarks and late data
Sessionization
More output modes
ML integrations
Make Structured
Streaming ready for
production workloads
as soon as possible
Stay tuned on our Databricks blogs for more information
and examples on Structured Streaming
Try latest version of Apache Spark
Try Apache Spark with Databricks
53
http://databricks.com/try
Structured Streaming
Making Continuous Applications
easier, faster, and smarter
Follow me @tathadas

Weitere ähnliche Inhalte

Was ist angesagt?

Deep Dive into Stateful Stream Processing in Structured Streaming with Tathag...
Deep Dive into Stateful Stream Processing in Structured Streaming with Tathag...Deep Dive into Stateful Stream Processing in Structured Streaming with Tathag...
Deep Dive into Stateful Stream Processing in Structured Streaming with Tathag...Databricks
 
Improving SparkSQL Performance by 30%: How We Optimize Parquet Pushdown and P...
Improving SparkSQL Performance by 30%: How We Optimize Parquet Pushdown and P...Improving SparkSQL Performance by 30%: How We Optimize Parquet Pushdown and P...
Improving SparkSQL Performance by 30%: How We Optimize Parquet Pushdown and P...Databricks
 
Discretized Stream - Fault-Tolerant Streaming Computation at Scale - SOSP
Discretized Stream - Fault-Tolerant Streaming Computation at Scale - SOSPDiscretized Stream - Fault-Tolerant Streaming Computation at Scale - SOSP
Discretized Stream - Fault-Tolerant Streaming Computation at Scale - SOSPTathagata Das
 
Adaptive Query Execution: Speeding Up Spark SQL at Runtime
Adaptive Query Execution: Speeding Up Spark SQL at RuntimeAdaptive Query Execution: Speeding Up Spark SQL at Runtime
Adaptive Query Execution: Speeding Up Spark SQL at RuntimeDatabricks
 
Apache Kafka Architecture & Fundamentals Explained
Apache Kafka Architecture & Fundamentals ExplainedApache Kafka Architecture & Fundamentals Explained
Apache Kafka Architecture & Fundamentals Explainedconfluent
 
Designing Structured Streaming Pipelines—How to Architect Things Right
Designing Structured Streaming Pipelines—How to Architect Things RightDesigning Structured Streaming Pipelines—How to Architect Things Right
Designing Structured Streaming Pipelines—How to Architect Things RightDatabricks
 
Speed Up Uber's Presto with Alluxio
Speed Up Uber's Presto with AlluxioSpeed Up Uber's Presto with Alluxio
Speed Up Uber's Presto with AlluxioAlluxio, Inc.
 
Large Scale Lakehouse Implementation Using Structured Streaming
Large Scale Lakehouse Implementation Using Structured StreamingLarge Scale Lakehouse Implementation Using Structured Streaming
Large Scale Lakehouse Implementation Using Structured StreamingDatabricks
 
(BDT318) How Netflix Handles Up To 8 Million Events Per Second
(BDT318) How Netflix Handles Up To 8 Million Events Per Second(BDT318) How Netflix Handles Up To 8 Million Events Per Second
(BDT318) How Netflix Handles Up To 8 Million Events Per SecondAmazon Web Services
 
Optimizing Apache Spark UDFs
Optimizing Apache Spark UDFsOptimizing Apache Spark UDFs
Optimizing Apache Spark UDFsDatabricks
 
Databricks Delta Lake and Its Benefits
Databricks Delta Lake and Its BenefitsDatabricks Delta Lake and Its Benefits
Databricks Delta Lake and Its BenefitsDatabricks
 
Delta Lake Streaming: Under the Hood
Delta Lake Streaming: Under the HoodDelta Lake Streaming: Under the Hood
Delta Lake Streaming: Under the HoodDatabricks
 
Deep Dive Into Elasticsearch
Deep Dive Into ElasticsearchDeep Dive Into Elasticsearch
Deep Dive Into ElasticsearchKnoldus Inc.
 
Arbitrary Stateful Aggregations using Structured Streaming in Apache Spark
Arbitrary Stateful Aggregations using Structured Streaming in Apache SparkArbitrary Stateful Aggregations using Structured Streaming in Apache Spark
Arbitrary Stateful Aggregations using Structured Streaming in Apache SparkDatabricks
 
Cosco: An Efficient Facebook-Scale Shuffle Service
Cosco: An Efficient Facebook-Scale Shuffle ServiceCosco: An Efficient Facebook-Scale Shuffle Service
Cosco: An Efficient Facebook-Scale Shuffle ServiceDatabricks
 
Distributed Databases Deconstructed: CockroachDB, TiDB and YugaByte DB
Distributed Databases Deconstructed: CockroachDB, TiDB and YugaByte DBDistributed Databases Deconstructed: CockroachDB, TiDB and YugaByte DB
Distributed Databases Deconstructed: CockroachDB, TiDB and YugaByte DBYugabyteDB
 
How We Optimize Spark SQL Jobs With parallel and sync IO
How We Optimize Spark SQL Jobs With parallel and sync IOHow We Optimize Spark SQL Jobs With parallel and sync IO
How We Optimize Spark SQL Jobs With parallel and sync IODatabricks
 
Matching the Scale at Tinder with Kafka
Matching the Scale at Tinder with Kafka Matching the Scale at Tinder with Kafka
Matching the Scale at Tinder with Kafka confluent
 

Was ist angesagt? (20)

Deep Dive into Stateful Stream Processing in Structured Streaming with Tathag...
Deep Dive into Stateful Stream Processing in Structured Streaming with Tathag...Deep Dive into Stateful Stream Processing in Structured Streaming with Tathag...
Deep Dive into Stateful Stream Processing in Structured Streaming with Tathag...
 
Improving SparkSQL Performance by 30%: How We Optimize Parquet Pushdown and P...
Improving SparkSQL Performance by 30%: How We Optimize Parquet Pushdown and P...Improving SparkSQL Performance by 30%: How We Optimize Parquet Pushdown and P...
Improving SparkSQL Performance by 30%: How We Optimize Parquet Pushdown and P...
 
Discretized Stream - Fault-Tolerant Streaming Computation at Scale - SOSP
Discretized Stream - Fault-Tolerant Streaming Computation at Scale - SOSPDiscretized Stream - Fault-Tolerant Streaming Computation at Scale - SOSP
Discretized Stream - Fault-Tolerant Streaming Computation at Scale - SOSP
 
Adaptive Query Execution: Speeding Up Spark SQL at Runtime
Adaptive Query Execution: Speeding Up Spark SQL at RuntimeAdaptive Query Execution: Speeding Up Spark SQL at Runtime
Adaptive Query Execution: Speeding Up Spark SQL at Runtime
 
Apache Kafka Architecture & Fundamentals Explained
Apache Kafka Architecture & Fundamentals ExplainedApache Kafka Architecture & Fundamentals Explained
Apache Kafka Architecture & Fundamentals Explained
 
Designing Structured Streaming Pipelines—How to Architect Things Right
Designing Structured Streaming Pipelines—How to Architect Things RightDesigning Structured Streaming Pipelines—How to Architect Things Right
Designing Structured Streaming Pipelines—How to Architect Things Right
 
Speed Up Uber's Presto with Alluxio
Speed Up Uber's Presto with AlluxioSpeed Up Uber's Presto with Alluxio
Speed Up Uber's Presto with Alluxio
 
Large Scale Lakehouse Implementation Using Structured Streaming
Large Scale Lakehouse Implementation Using Structured StreamingLarge Scale Lakehouse Implementation Using Structured Streaming
Large Scale Lakehouse Implementation Using Structured Streaming
 
(BDT318) How Netflix Handles Up To 8 Million Events Per Second
(BDT318) How Netflix Handles Up To 8 Million Events Per Second(BDT318) How Netflix Handles Up To 8 Million Events Per Second
(BDT318) How Netflix Handles Up To 8 Million Events Per Second
 
Optimizing Apache Spark UDFs
Optimizing Apache Spark UDFsOptimizing Apache Spark UDFs
Optimizing Apache Spark UDFs
 
Databricks Delta Lake and Its Benefits
Databricks Delta Lake and Its BenefitsDatabricks Delta Lake and Its Benefits
Databricks Delta Lake and Its Benefits
 
Delta Lake Streaming: Under the Hood
Delta Lake Streaming: Under the HoodDelta Lake Streaming: Under the Hood
Delta Lake Streaming: Under the Hood
 
Elasticsearch
ElasticsearchElasticsearch
Elasticsearch
 
Deep Dive Into Elasticsearch
Deep Dive Into ElasticsearchDeep Dive Into Elasticsearch
Deep Dive Into Elasticsearch
 
Arbitrary Stateful Aggregations using Structured Streaming in Apache Spark
Arbitrary Stateful Aggregations using Structured Streaming in Apache SparkArbitrary Stateful Aggregations using Structured Streaming in Apache Spark
Arbitrary Stateful Aggregations using Structured Streaming in Apache Spark
 
Cosco: An Efficient Facebook-Scale Shuffle Service
Cosco: An Efficient Facebook-Scale Shuffle ServiceCosco: An Efficient Facebook-Scale Shuffle Service
Cosco: An Efficient Facebook-Scale Shuffle Service
 
Distributed Databases Deconstructed: CockroachDB, TiDB and YugaByte DB
Distributed Databases Deconstructed: CockroachDB, TiDB and YugaByte DBDistributed Databases Deconstructed: CockroachDB, TiDB and YugaByte DB
Distributed Databases Deconstructed: CockroachDB, TiDB and YugaByte DB
 
How We Optimize Spark SQL Jobs With parallel and sync IO
How We Optimize Spark SQL Jobs With parallel and sync IOHow We Optimize Spark SQL Jobs With parallel and sync IO
How We Optimize Spark SQL Jobs With parallel and sync IO
 
Matching the Scale at Tinder with Kafka
Matching the Scale at Tinder with Kafka Matching the Scale at Tinder with Kafka
Matching the Scale at Tinder with Kafka
 
Elasticsearch
ElasticsearchElasticsearch
Elasticsearch
 

Andere mochten auch

What's new with Apache Spark's Structured Streaming?
What's new with Apache Spark's Structured Streaming?What's new with Apache Spark's Structured Streaming?
What's new with Apache Spark's Structured Streaming?Miklos Christine
 
Structured Streaming Using Spark 2.1
Structured Streaming Using Spark 2.1Structured Streaming Using Spark 2.1
Structured Streaming Using Spark 2.1Sigmoid
 
Introduction to Structured streaming
Introduction to Structured streamingIntroduction to Structured streaming
Introduction to Structured streamingdatamantra
 
Easy, scalable, fault tolerant stream processing with structured streaming - ...
Easy, scalable, fault tolerant stream processing with structured streaming - ...Easy, scalable, fault tolerant stream processing with structured streaming - ...
Easy, scalable, fault tolerant stream processing with structured streaming - ...Databricks
 
A Deep Dive into Structured Streaming in Apache Spark
A Deep Dive into Structured Streaming in Apache Spark A Deep Dive into Structured Streaming in Apache Spark
A Deep Dive into Structured Streaming in Apache Spark Anyscale
 
Introduction to Structured Streaming
Introduction to Structured StreamingIntroduction to Structured Streaming
Introduction to Structured Streamingdatamantra
 
Building Continuous Application with Structured Streaming and Real-Time Data ...
Building Continuous Application with Structured Streaming and Real-Time Data ...Building Continuous Application with Structured Streaming and Real-Time Data ...
Building Continuous Application with Structured Streaming and Real-Time Data ...Databricks
 
Making Structured Streaming Ready for Production
Making Structured Streaming Ready for ProductionMaking Structured Streaming Ready for Production
Making Structured Streaming Ready for ProductionDatabricks
 
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
 
Deep dive into stateful stream processing in structured streaming by Tathaga...
Deep dive into stateful stream processing in structured streaming  by Tathaga...Deep dive into stateful stream processing in structured streaming  by Tathaga...
Deep dive into stateful stream processing in structured streaming by Tathaga...Databricks
 

Andere mochten auch (10)

What's new with Apache Spark's Structured Streaming?
What's new with Apache Spark's Structured Streaming?What's new with Apache Spark's Structured Streaming?
What's new with Apache Spark's Structured Streaming?
 
Structured Streaming Using Spark 2.1
Structured Streaming Using Spark 2.1Structured Streaming Using Spark 2.1
Structured Streaming Using Spark 2.1
 
Introduction to Structured streaming
Introduction to Structured streamingIntroduction to Structured streaming
Introduction to Structured streaming
 
Easy, scalable, fault tolerant stream processing with structured streaming - ...
Easy, scalable, fault tolerant stream processing with structured streaming - ...Easy, scalable, fault tolerant stream processing with structured streaming - ...
Easy, scalable, fault tolerant stream processing with structured streaming - ...
 
A Deep Dive into Structured Streaming in Apache Spark
A Deep Dive into Structured Streaming in Apache Spark A Deep Dive into Structured Streaming in Apache Spark
A Deep Dive into Structured Streaming in Apache Spark
 
Introduction to Structured Streaming
Introduction to Structured StreamingIntroduction to Structured Streaming
Introduction to Structured Streaming
 
Building Continuous Application with Structured Streaming and Real-Time Data ...
Building Continuous Application with Structured Streaming and Real-Time Data ...Building Continuous Application with Structured Streaming and Real-Time Data ...
Building Continuous Application with Structured Streaming and Real-Time Data ...
 
Making Structured Streaming Ready for Production
Making Structured Streaming Ready for ProductionMaking Structured Streaming Ready for Production
Making Structured Streaming Ready for Production
 
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
 
Deep dive into stateful stream processing in structured streaming by Tathaga...
Deep dive into stateful stream processing in structured streaming  by Tathaga...Deep dive into stateful stream processing in structured streaming  by Tathaga...
Deep dive into stateful stream processing in structured streaming by Tathaga...
 

Ähnlich wie A Deep Dive into Structured Streaming: Apache Spark Meetup at Bloomberg 2016

Taking Spark Streaming to the Next Level with Datasets and DataFrames
Taking Spark Streaming to the Next Level with Datasets and DataFramesTaking Spark Streaming to the Next Level with Datasets and DataFrames
Taking Spark Streaming to the Next Level with Datasets and DataFramesDatabricks
 
Continuous Application with Structured Streaming 2.0
Continuous Application with Structured Streaming 2.0Continuous Application with Structured Streaming 2.0
Continuous Application with Structured Streaming 2.0Anyscale
 
The Future of Real-Time in Spark
The Future of Real-Time in SparkThe Future of Real-Time in Spark
The Future of Real-Time in SparkReynold Xin
 
Tecnicas e Instrumentos de Recoleccion de Datos
Tecnicas e Instrumentos de Recoleccion de DatosTecnicas e Instrumentos de Recoleccion de Datos
Tecnicas e Instrumentos de Recoleccion de DatosAngel Giraldo
 
The Future of Real-Time in Spark
The Future of Real-Time in SparkThe Future of Real-Time in Spark
The Future of Real-Time in SparkDatabricks
 
Easy, Scalable, Fault-tolerant stream processing with Structured Streaming in...
Easy, Scalable, Fault-tolerant stream processing with Structured Streaming in...Easy, Scalable, Fault-tolerant stream processing with Structured Streaming in...
Easy, Scalable, Fault-tolerant stream processing with Structured Streaming in...DataWorks Summit
 
Writing Continuous Applications with Structured Streaming Python APIs in Apac...
Writing Continuous Applications with Structured Streaming Python APIs in Apac...Writing Continuous Applications with Structured Streaming Python APIs in Apac...
Writing Continuous Applications with Structured Streaming Python APIs in Apac...Databricks
 
Strata NYC 2015: What's new in Spark Streaming
Strata NYC 2015: What's new in Spark StreamingStrata NYC 2015: What's new in Spark Streaming
Strata NYC 2015: What's new in Spark StreamingDatabricks
 
Kafka Summit NYC 2017 - Easy, Scalable, Fault-tolerant Stream Processing with...
Kafka Summit NYC 2017 - Easy, Scalable, Fault-tolerant Stream Processing with...Kafka Summit NYC 2017 - Easy, Scalable, Fault-tolerant Stream Processing with...
Kafka Summit NYC 2017 - Easy, Scalable, Fault-tolerant Stream Processing with...confluent
 
Spark Streaming Recipes and "Exactly Once" Semantics Revised
Spark Streaming Recipes and "Exactly Once" Semantics RevisedSpark Streaming Recipes and "Exactly Once" Semantics Revised
Spark Streaming Recipes and "Exactly Once" Semantics RevisedMichael Spector
 
Spark what's new what's coming
Spark what's new what's comingSpark what's new what's coming
Spark what's new what's comingDatabricks
 
Designing ETL Pipelines with Structured Streaming and Delta Lake—How to Archi...
Designing ETL Pipelines with Structured Streaming and Delta Lake—How to Archi...Designing ETL Pipelines with Structured Streaming and Delta Lake—How to Archi...
Designing ETL Pipelines with Structured Streaming and Delta Lake—How to Archi...Databricks
 
Writing Continuous Applications with Structured Streaming in PySpark
Writing Continuous Applications with Structured Streaming in PySparkWriting Continuous Applications with Structured Streaming in PySpark
Writing Continuous Applications with Structured Streaming in PySparkDatabricks
 
Introduction to Spark Streaming
Introduction to Spark StreamingIntroduction to Spark Streaming
Introduction to Spark StreamingKnoldus Inc.
 
Learning spark ch10 - Spark Streaming
Learning spark ch10 - Spark StreamingLearning spark ch10 - Spark Streaming
Learning spark ch10 - Spark Streamingphanleson
 
A Deep Dive into Stateful Stream Processing in Structured Streaming with Tath...
A Deep Dive into Stateful Stream Processing in Structured Streaming with Tath...A Deep Dive into Stateful Stream Processing in Structured Streaming with Tath...
A Deep Dive into Stateful Stream Processing in Structured Streaming with Tath...Databricks
 
Flink Forward SF 2017: Srikanth Satya & Tom Kaitchuck - Pravega: Storage Rei...
Flink Forward SF 2017: Srikanth Satya & Tom Kaitchuck -  Pravega: Storage Rei...Flink Forward SF 2017: Srikanth Satya & Tom Kaitchuck -  Pravega: Storage Rei...
Flink Forward SF 2017: Srikanth Satya & Tom Kaitchuck - Pravega: Storage Rei...Flink Forward
 
Spark (Structured) Streaming vs. Kafka Streams - two stream processing platfo...
Spark (Structured) Streaming vs. Kafka Streams - two stream processing platfo...Spark (Structured) Streaming vs. Kafka Streams - two stream processing platfo...
Spark (Structured) Streaming vs. Kafka Streams - two stream processing platfo...Guido Schmutz
 

Ähnlich wie A Deep Dive into Structured Streaming: Apache Spark Meetup at Bloomberg 2016 (20)

Taking Spark Streaming to the Next Level with Datasets and DataFrames
Taking Spark Streaming to the Next Level with Datasets and DataFramesTaking Spark Streaming to the Next Level with Datasets and DataFrames
Taking Spark Streaming to the Next Level with Datasets and DataFrames
 
Continuous Application with Structured Streaming 2.0
Continuous Application with Structured Streaming 2.0Continuous Application with Structured Streaming 2.0
Continuous Application with Structured Streaming 2.0
 
The Future of Real-Time in Spark
The Future of Real-Time in SparkThe Future of Real-Time in Spark
The Future of Real-Time in Spark
 
Tecnicas e Instrumentos de Recoleccion de Datos
Tecnicas e Instrumentos de Recoleccion de DatosTecnicas e Instrumentos de Recoleccion de Datos
Tecnicas e Instrumentos de Recoleccion de Datos
 
The Future of Real-Time in Spark
The Future of Real-Time in SparkThe Future of Real-Time in Spark
The Future of Real-Time in Spark
 
Easy, Scalable, Fault-tolerant stream processing with Structured Streaming in...
Easy, Scalable, Fault-tolerant stream processing with Structured Streaming in...Easy, Scalable, Fault-tolerant stream processing with Structured Streaming in...
Easy, Scalable, Fault-tolerant stream processing with Structured Streaming in...
 
Writing Continuous Applications with Structured Streaming Python APIs in Apac...
Writing Continuous Applications with Structured Streaming Python APIs in Apac...Writing Continuous Applications with Structured Streaming Python APIs in Apac...
Writing Continuous Applications with Structured Streaming Python APIs in Apac...
 
Strata NYC 2015: What's new in Spark Streaming
Strata NYC 2015: What's new in Spark StreamingStrata NYC 2015: What's new in Spark Streaming
Strata NYC 2015: What's new in Spark Streaming
 
Kafka Summit NYC 2017 - Easy, Scalable, Fault-tolerant Stream Processing with...
Kafka Summit NYC 2017 - Easy, Scalable, Fault-tolerant Stream Processing with...Kafka Summit NYC 2017 - Easy, Scalable, Fault-tolerant Stream Processing with...
Kafka Summit NYC 2017 - Easy, Scalable, Fault-tolerant Stream Processing with...
 
Spark Streaming Recipes and "Exactly Once" Semantics Revised
Spark Streaming Recipes and "Exactly Once" Semantics RevisedSpark Streaming Recipes and "Exactly Once" Semantics Revised
Spark Streaming Recipes and "Exactly Once" Semantics Revised
 
Spark what's new what's coming
Spark what's new what's comingSpark what's new what's coming
Spark what's new what's coming
 
Designing ETL Pipelines with Structured Streaming and Delta Lake—How to Archi...
Designing ETL Pipelines with Structured Streaming and Delta Lake—How to Archi...Designing ETL Pipelines with Structured Streaming and Delta Lake—How to Archi...
Designing ETL Pipelines with Structured Streaming and Delta Lake—How to Archi...
 
Writing Continuous Applications with Structured Streaming in PySpark
Writing Continuous Applications with Structured Streaming in PySparkWriting Continuous Applications with Structured Streaming in PySpark
Writing Continuous Applications with Structured Streaming in PySpark
 
Introduction to Spark Streaming
Introduction to Spark StreamingIntroduction to Spark Streaming
Introduction to Spark Streaming
 
Learning spark ch10 - Spark Streaming
Learning spark ch10 - Spark StreamingLearning spark ch10 - Spark Streaming
Learning spark ch10 - Spark Streaming
 
A Deep Dive into Stateful Stream Processing in Structured Streaming with Tath...
A Deep Dive into Stateful Stream Processing in Structured Streaming with Tath...A Deep Dive into Stateful Stream Processing in Structured Streaming with Tath...
A Deep Dive into Stateful Stream Processing in Structured Streaming with Tath...
 
Flink Forward SF 2017: Srikanth Satya & Tom Kaitchuck - Pravega: Storage Rei...
Flink Forward SF 2017: Srikanth Satya & Tom Kaitchuck -  Pravega: Storage Rei...Flink Forward SF 2017: Srikanth Satya & Tom Kaitchuck -  Pravega: Storage Rei...
Flink Forward SF 2017: Srikanth Satya & Tom Kaitchuck - Pravega: Storage Rei...
 
So you think you can stream.pptx
So you think you can stream.pptxSo you think you can stream.pptx
So you think you can stream.pptx
 
Stream Processing Overview
Stream Processing OverviewStream Processing Overview
Stream Processing Overview
 
Spark (Structured) Streaming vs. Kafka Streams - two stream processing platfo...
Spark (Structured) Streaming vs. Kafka Streams - two stream processing platfo...Spark (Structured) Streaming vs. Kafka Streams - two stream processing platfo...
Spark (Structured) Streaming vs. Kafka Streams - two stream processing platfo...
 

Mehr von Databricks

DW Migration Webinar-March 2022.pptx
DW Migration Webinar-March 2022.pptxDW Migration Webinar-March 2022.pptx
DW Migration Webinar-March 2022.pptxDatabricks
 
Data Lakehouse Symposium | Day 1 | Part 1
Data Lakehouse Symposium | Day 1 | Part 1Data Lakehouse Symposium | Day 1 | Part 1
Data Lakehouse Symposium | Day 1 | Part 1Databricks
 
Data Lakehouse Symposium | Day 1 | Part 2
Data Lakehouse Symposium | Day 1 | Part 2Data Lakehouse Symposium | Day 1 | Part 2
Data Lakehouse Symposium | Day 1 | Part 2Databricks
 
Data Lakehouse Symposium | Day 2
Data Lakehouse Symposium | Day 2Data Lakehouse Symposium | Day 2
Data Lakehouse Symposium | Day 2Databricks
 
Data Lakehouse Symposium | Day 4
Data Lakehouse Symposium | Day 4Data Lakehouse Symposium | Day 4
Data Lakehouse Symposium | Day 4Databricks
 
5 Critical Steps to Clean Your Data Swamp When Migrating Off of Hadoop
5 Critical Steps to Clean Your Data Swamp When Migrating Off of Hadoop5 Critical Steps to Clean Your Data Swamp When Migrating Off of Hadoop
5 Critical Steps to Clean Your Data Swamp When Migrating Off of HadoopDatabricks
 
Democratizing Data Quality Through a Centralized Platform
Democratizing Data Quality Through a Centralized PlatformDemocratizing Data Quality Through a Centralized Platform
Democratizing Data Quality Through a Centralized PlatformDatabricks
 
Learn to Use Databricks for Data Science
Learn to Use Databricks for Data ScienceLearn to Use Databricks for Data Science
Learn to Use Databricks for Data ScienceDatabricks
 
Why APM Is Not the Same As ML Monitoring
Why APM Is Not the Same As ML MonitoringWhy APM Is Not the Same As ML Monitoring
Why APM Is Not the Same As ML MonitoringDatabricks
 
The Function, the Context, and the Data—Enabling ML Ops at Stitch Fix
The Function, the Context, and the Data—Enabling ML Ops at Stitch FixThe Function, the Context, and the Data—Enabling ML Ops at Stitch Fix
The Function, the Context, and the Data—Enabling ML Ops at Stitch FixDatabricks
 
Stage Level Scheduling Improving Big Data and AI Integration
Stage Level Scheduling Improving Big Data and AI IntegrationStage Level Scheduling Improving Big Data and AI Integration
Stage Level Scheduling Improving Big Data and AI IntegrationDatabricks
 
Simplify Data Conversion from Spark to TensorFlow and PyTorch
Simplify Data Conversion from Spark to TensorFlow and PyTorchSimplify Data Conversion from Spark to TensorFlow and PyTorch
Simplify Data Conversion from Spark to TensorFlow and PyTorchDatabricks
 
Scaling your Data Pipelines with Apache Spark on Kubernetes
Scaling your Data Pipelines with Apache Spark on KubernetesScaling your Data Pipelines with Apache Spark on Kubernetes
Scaling your Data Pipelines with Apache Spark on KubernetesDatabricks
 
Scaling and Unifying SciKit Learn and Apache Spark Pipelines
Scaling and Unifying SciKit Learn and Apache Spark PipelinesScaling and Unifying SciKit Learn and Apache Spark Pipelines
Scaling and Unifying SciKit Learn and Apache Spark PipelinesDatabricks
 
Sawtooth Windows for Feature Aggregations
Sawtooth Windows for Feature AggregationsSawtooth Windows for Feature Aggregations
Sawtooth Windows for Feature AggregationsDatabricks
 
Redis + Apache Spark = Swiss Army Knife Meets Kitchen Sink
Redis + Apache Spark = Swiss Army Knife Meets Kitchen SinkRedis + Apache Spark = Swiss Army Knife Meets Kitchen Sink
Redis + Apache Spark = Swiss Army Knife Meets Kitchen SinkDatabricks
 
Re-imagine Data Monitoring with whylogs and Spark
Re-imagine Data Monitoring with whylogs and SparkRe-imagine Data Monitoring with whylogs and Spark
Re-imagine Data Monitoring with whylogs and SparkDatabricks
 
Raven: End-to-end Optimization of ML Prediction Queries
Raven: End-to-end Optimization of ML Prediction QueriesRaven: End-to-end Optimization of ML Prediction Queries
Raven: End-to-end Optimization of ML Prediction QueriesDatabricks
 
Processing Large Datasets for ADAS Applications using Apache Spark
Processing Large Datasets for ADAS Applications using Apache SparkProcessing Large Datasets for ADAS Applications using Apache Spark
Processing Large Datasets for ADAS Applications using Apache SparkDatabricks
 
Massive Data Processing in Adobe Using Delta Lake
Massive Data Processing in Adobe Using Delta LakeMassive Data Processing in Adobe Using Delta Lake
Massive Data Processing in Adobe Using Delta LakeDatabricks
 

Mehr von Databricks (20)

DW Migration Webinar-March 2022.pptx
DW Migration Webinar-March 2022.pptxDW Migration Webinar-March 2022.pptx
DW Migration Webinar-March 2022.pptx
 
Data Lakehouse Symposium | Day 1 | Part 1
Data Lakehouse Symposium | Day 1 | Part 1Data Lakehouse Symposium | Day 1 | Part 1
Data Lakehouse Symposium | Day 1 | Part 1
 
Data Lakehouse Symposium | Day 1 | Part 2
Data Lakehouse Symposium | Day 1 | Part 2Data Lakehouse Symposium | Day 1 | Part 2
Data Lakehouse Symposium | Day 1 | Part 2
 
Data Lakehouse Symposium | Day 2
Data Lakehouse Symposium | Day 2Data Lakehouse Symposium | Day 2
Data Lakehouse Symposium | Day 2
 
Data Lakehouse Symposium | Day 4
Data Lakehouse Symposium | Day 4Data Lakehouse Symposium | Day 4
Data Lakehouse Symposium | Day 4
 
5 Critical Steps to Clean Your Data Swamp When Migrating Off of Hadoop
5 Critical Steps to Clean Your Data Swamp When Migrating Off of Hadoop5 Critical Steps to Clean Your Data Swamp When Migrating Off of Hadoop
5 Critical Steps to Clean Your Data Swamp When Migrating Off of Hadoop
 
Democratizing Data Quality Through a Centralized Platform
Democratizing Data Quality Through a Centralized PlatformDemocratizing Data Quality Through a Centralized Platform
Democratizing Data Quality Through a Centralized Platform
 
Learn to Use Databricks for Data Science
Learn to Use Databricks for Data ScienceLearn to Use Databricks for Data Science
Learn to Use Databricks for Data Science
 
Why APM Is Not the Same As ML Monitoring
Why APM Is Not the Same As ML MonitoringWhy APM Is Not the Same As ML Monitoring
Why APM Is Not the Same As ML Monitoring
 
The Function, the Context, and the Data—Enabling ML Ops at Stitch Fix
The Function, the Context, and the Data—Enabling ML Ops at Stitch FixThe Function, the Context, and the Data—Enabling ML Ops at Stitch Fix
The Function, the Context, and the Data—Enabling ML Ops at Stitch Fix
 
Stage Level Scheduling Improving Big Data and AI Integration
Stage Level Scheduling Improving Big Data and AI IntegrationStage Level Scheduling Improving Big Data and AI Integration
Stage Level Scheduling Improving Big Data and AI Integration
 
Simplify Data Conversion from Spark to TensorFlow and PyTorch
Simplify Data Conversion from Spark to TensorFlow and PyTorchSimplify Data Conversion from Spark to TensorFlow and PyTorch
Simplify Data Conversion from Spark to TensorFlow and PyTorch
 
Scaling your Data Pipelines with Apache Spark on Kubernetes
Scaling your Data Pipelines with Apache Spark on KubernetesScaling your Data Pipelines with Apache Spark on Kubernetes
Scaling your Data Pipelines with Apache Spark on Kubernetes
 
Scaling and Unifying SciKit Learn and Apache Spark Pipelines
Scaling and Unifying SciKit Learn and Apache Spark PipelinesScaling and Unifying SciKit Learn and Apache Spark Pipelines
Scaling and Unifying SciKit Learn and Apache Spark Pipelines
 
Sawtooth Windows for Feature Aggregations
Sawtooth Windows for Feature AggregationsSawtooth Windows for Feature Aggregations
Sawtooth Windows for Feature Aggregations
 
Redis + Apache Spark = Swiss Army Knife Meets Kitchen Sink
Redis + Apache Spark = Swiss Army Knife Meets Kitchen SinkRedis + Apache Spark = Swiss Army Knife Meets Kitchen Sink
Redis + Apache Spark = Swiss Army Knife Meets Kitchen Sink
 
Re-imagine Data Monitoring with whylogs and Spark
Re-imagine Data Monitoring with whylogs and SparkRe-imagine Data Monitoring with whylogs and Spark
Re-imagine Data Monitoring with whylogs and Spark
 
Raven: End-to-end Optimization of ML Prediction Queries
Raven: End-to-end Optimization of ML Prediction QueriesRaven: End-to-end Optimization of ML Prediction Queries
Raven: End-to-end Optimization of ML Prediction Queries
 
Processing Large Datasets for ADAS Applications using Apache Spark
Processing Large Datasets for ADAS Applications using Apache SparkProcessing Large Datasets for ADAS Applications using Apache Spark
Processing Large Datasets for ADAS Applications using Apache Spark
 
Massive Data Processing in Adobe Using Delta Lake
Massive Data Processing in Adobe Using Delta LakeMassive Data Processing in Adobe Using Delta Lake
Massive Data Processing in Adobe Using Delta Lake
 

Kürzlich hochgeladen

Comparing Linux OS Image Update Models - EOSS 2024.pdf
Comparing Linux OS Image Update Models - EOSS 2024.pdfComparing Linux OS Image Update Models - EOSS 2024.pdf
Comparing Linux OS Image Update Models - EOSS 2024.pdfDrew Moseley
 
Salesforce Implementation Services PPT By ABSYZ
Salesforce Implementation Services PPT By ABSYZSalesforce Implementation Services PPT By ABSYZ
Salesforce Implementation Services PPT By ABSYZABSYZ Inc
 
VictoriaMetrics Q1 Meet Up '24 - Community & News Update
VictoriaMetrics Q1 Meet Up '24 - Community & News UpdateVictoriaMetrics Q1 Meet Up '24 - Community & News Update
VictoriaMetrics Q1 Meet Up '24 - Community & News UpdateVictoriaMetrics
 
Post Quantum Cryptography – The Impact on Identity
Post Quantum Cryptography – The Impact on IdentityPost Quantum Cryptography – The Impact on Identity
Post Quantum Cryptography – The Impact on Identityteam-WIBU
 
OpenChain AI Study Group - Europe and Asia Recap - 2024-04-11 - Full Recording
OpenChain AI Study Group - Europe and Asia Recap - 2024-04-11 - Full RecordingOpenChain AI Study Group - Europe and Asia Recap - 2024-04-11 - Full Recording
OpenChain AI Study Group - Europe and Asia Recap - 2024-04-11 - Full RecordingShane Coughlan
 
Ronisha Informatics Private Limited Catalogue
Ronisha Informatics Private Limited CatalogueRonisha Informatics Private Limited Catalogue
Ronisha Informatics Private Limited Catalogueitservices996
 
Real-time Tracking and Monitoring with Cargo Cloud Solutions.pptx
Real-time Tracking and Monitoring with Cargo Cloud Solutions.pptxReal-time Tracking and Monitoring with Cargo Cloud Solutions.pptx
Real-time Tracking and Monitoring with Cargo Cloud Solutions.pptxRTS corp
 
JavaLand 2024 - Going serverless with Quarkus GraalVM native images and AWS L...
JavaLand 2024 - Going serverless with Quarkus GraalVM native images and AWS L...JavaLand 2024 - Going serverless with Quarkus GraalVM native images and AWS L...
JavaLand 2024 - Going serverless with Quarkus GraalVM native images and AWS L...Bert Jan Schrijver
 
Strategies for using alternative queries to mitigate zero results
Strategies for using alternative queries to mitigate zero resultsStrategies for using alternative queries to mitigate zero results
Strategies for using alternative queries to mitigate zero resultsJean Silva
 
UI5ers live - Custom Controls wrapping 3rd-party libs.pptx
UI5ers live - Custom Controls wrapping 3rd-party libs.pptxUI5ers live - Custom Controls wrapping 3rd-party libs.pptx
UI5ers live - Custom Controls wrapping 3rd-party libs.pptxAndreas Kunz
 
Leveraging AI for Mobile App Testing on Real Devices | Applitools + Kobiton
Leveraging AI for Mobile App Testing on Real Devices | Applitools + KobitonLeveraging AI for Mobile App Testing on Real Devices | Applitools + Kobiton
Leveraging AI for Mobile App Testing on Real Devices | Applitools + KobitonApplitools
 
Amazon Bedrock in Action - presentation of the Bedrock's capabilities
Amazon Bedrock in Action - presentation of the Bedrock's capabilitiesAmazon Bedrock in Action - presentation of the Bedrock's capabilities
Amazon Bedrock in Action - presentation of the Bedrock's capabilitiesKrzysztofKkol1
 
Precise and Complete Requirements? An Elusive Goal
Precise and Complete Requirements? An Elusive GoalPrecise and Complete Requirements? An Elusive Goal
Precise and Complete Requirements? An Elusive GoalLionel Briand
 
Machine Learning Software Engineering Patterns and Their Engineering
Machine Learning Software Engineering Patterns and Their EngineeringMachine Learning Software Engineering Patterns and Their Engineering
Machine Learning Software Engineering Patterns and Their EngineeringHironori Washizaki
 
What’s New in VictoriaMetrics: Q1 2024 Updates
What’s New in VictoriaMetrics: Q1 2024 UpdatesWhat’s New in VictoriaMetrics: Q1 2024 Updates
What’s New in VictoriaMetrics: Q1 2024 UpdatesVictoriaMetrics
 
The Role of IoT and Sensor Technology in Cargo Cloud Solutions.pptx
The Role of IoT and Sensor Technology in Cargo Cloud Solutions.pptxThe Role of IoT and Sensor Technology in Cargo Cloud Solutions.pptx
The Role of IoT and Sensor Technology in Cargo Cloud Solutions.pptxRTS corp
 
Enhancing Supply Chain Visibility with Cargo Cloud Solutions.pdf
Enhancing Supply Chain Visibility with Cargo Cloud Solutions.pdfEnhancing Supply Chain Visibility with Cargo Cloud Solutions.pdf
Enhancing Supply Chain Visibility with Cargo Cloud Solutions.pdfRTS corp
 
Large Language Models for Test Case Evolution and Repair
Large Language Models for Test Case Evolution and RepairLarge Language Models for Test Case Evolution and Repair
Large Language Models for Test Case Evolution and RepairLionel Briand
 
OpenChain Education Work Group Monthly Meeting - 2024-04-10 - Full Recording
OpenChain Education Work Group Monthly Meeting - 2024-04-10 - Full RecordingOpenChain Education Work Group Monthly Meeting - 2024-04-10 - Full Recording
OpenChain Education Work Group Monthly Meeting - 2024-04-10 - Full RecordingShane Coughlan
 
Revolutionizing the Digital Transformation Office - Leveraging OnePlan’s AI a...
Revolutionizing the Digital Transformation Office - Leveraging OnePlan’s AI a...Revolutionizing the Digital Transformation Office - Leveraging OnePlan’s AI a...
Revolutionizing the Digital Transformation Office - Leveraging OnePlan’s AI a...OnePlan Solutions
 

Kürzlich hochgeladen (20)

Comparing Linux OS Image Update Models - EOSS 2024.pdf
Comparing Linux OS Image Update Models - EOSS 2024.pdfComparing Linux OS Image Update Models - EOSS 2024.pdf
Comparing Linux OS Image Update Models - EOSS 2024.pdf
 
Salesforce Implementation Services PPT By ABSYZ
Salesforce Implementation Services PPT By ABSYZSalesforce Implementation Services PPT By ABSYZ
Salesforce Implementation Services PPT By ABSYZ
 
VictoriaMetrics Q1 Meet Up '24 - Community & News Update
VictoriaMetrics Q1 Meet Up '24 - Community & News UpdateVictoriaMetrics Q1 Meet Up '24 - Community & News Update
VictoriaMetrics Q1 Meet Up '24 - Community & News Update
 
Post Quantum Cryptography – The Impact on Identity
Post Quantum Cryptography – The Impact on IdentityPost Quantum Cryptography – The Impact on Identity
Post Quantum Cryptography – The Impact on Identity
 
OpenChain AI Study Group - Europe and Asia Recap - 2024-04-11 - Full Recording
OpenChain AI Study Group - Europe and Asia Recap - 2024-04-11 - Full RecordingOpenChain AI Study Group - Europe and Asia Recap - 2024-04-11 - Full Recording
OpenChain AI Study Group - Europe and Asia Recap - 2024-04-11 - Full Recording
 
Ronisha Informatics Private Limited Catalogue
Ronisha Informatics Private Limited CatalogueRonisha Informatics Private Limited Catalogue
Ronisha Informatics Private Limited Catalogue
 
Real-time Tracking and Monitoring with Cargo Cloud Solutions.pptx
Real-time Tracking and Monitoring with Cargo Cloud Solutions.pptxReal-time Tracking and Monitoring with Cargo Cloud Solutions.pptx
Real-time Tracking and Monitoring with Cargo Cloud Solutions.pptx
 
JavaLand 2024 - Going serverless with Quarkus GraalVM native images and AWS L...
JavaLand 2024 - Going serverless with Quarkus GraalVM native images and AWS L...JavaLand 2024 - Going serverless with Quarkus GraalVM native images and AWS L...
JavaLand 2024 - Going serverless with Quarkus GraalVM native images and AWS L...
 
Strategies for using alternative queries to mitigate zero results
Strategies for using alternative queries to mitigate zero resultsStrategies for using alternative queries to mitigate zero results
Strategies for using alternative queries to mitigate zero results
 
UI5ers live - Custom Controls wrapping 3rd-party libs.pptx
UI5ers live - Custom Controls wrapping 3rd-party libs.pptxUI5ers live - Custom Controls wrapping 3rd-party libs.pptx
UI5ers live - Custom Controls wrapping 3rd-party libs.pptx
 
Leveraging AI for Mobile App Testing on Real Devices | Applitools + Kobiton
Leveraging AI for Mobile App Testing on Real Devices | Applitools + KobitonLeveraging AI for Mobile App Testing on Real Devices | Applitools + Kobiton
Leveraging AI for Mobile App Testing on Real Devices | Applitools + Kobiton
 
Amazon Bedrock in Action - presentation of the Bedrock's capabilities
Amazon Bedrock in Action - presentation of the Bedrock's capabilitiesAmazon Bedrock in Action - presentation of the Bedrock's capabilities
Amazon Bedrock in Action - presentation of the Bedrock's capabilities
 
Precise and Complete Requirements? An Elusive Goal
Precise and Complete Requirements? An Elusive GoalPrecise and Complete Requirements? An Elusive Goal
Precise and Complete Requirements? An Elusive Goal
 
Machine Learning Software Engineering Patterns and Their Engineering
Machine Learning Software Engineering Patterns and Their EngineeringMachine Learning Software Engineering Patterns and Their Engineering
Machine Learning Software Engineering Patterns and Their Engineering
 
What’s New in VictoriaMetrics: Q1 2024 Updates
What’s New in VictoriaMetrics: Q1 2024 UpdatesWhat’s New in VictoriaMetrics: Q1 2024 Updates
What’s New in VictoriaMetrics: Q1 2024 Updates
 
The Role of IoT and Sensor Technology in Cargo Cloud Solutions.pptx
The Role of IoT and Sensor Technology in Cargo Cloud Solutions.pptxThe Role of IoT and Sensor Technology in Cargo Cloud Solutions.pptx
The Role of IoT and Sensor Technology in Cargo Cloud Solutions.pptx
 
Enhancing Supply Chain Visibility with Cargo Cloud Solutions.pdf
Enhancing Supply Chain Visibility with Cargo Cloud Solutions.pdfEnhancing Supply Chain Visibility with Cargo Cloud Solutions.pdf
Enhancing Supply Chain Visibility with Cargo Cloud Solutions.pdf
 
Large Language Models for Test Case Evolution and Repair
Large Language Models for Test Case Evolution and RepairLarge Language Models for Test Case Evolution and Repair
Large Language Models for Test Case Evolution and Repair
 
OpenChain Education Work Group Monthly Meeting - 2024-04-10 - Full Recording
OpenChain Education Work Group Monthly Meeting - 2024-04-10 - Full RecordingOpenChain Education Work Group Monthly Meeting - 2024-04-10 - Full Recording
OpenChain Education Work Group Monthly Meeting - 2024-04-10 - Full Recording
 
Revolutionizing the Digital Transformation Office - Leveraging OnePlan’s AI a...
Revolutionizing the Digital Transformation Office - Leveraging OnePlan’s AI a...Revolutionizing the Digital Transformation Office - Leveraging OnePlan’s AI a...
Revolutionizing the Digital Transformation Office - Leveraging OnePlan’s AI a...
 

A Deep Dive into Structured Streaming: Apache Spark Meetup at Bloomberg 2016

  • 1. A Deep Dive into Structured Streaming Tathagata “TD” Das @tathadas Spark Meetup @ Bloomberg 27 September 2016
  • 2. Streaming in Apache Spark Spark Streaming changed how people write streaming apps 2 SQL Streamin g MLlib Spark Core GraphX Functional, concise and expressive Fault-tolerant state management Unified stack with batch processing More than 50% users consider most important part of Apache Spark
  • 3. Streaming apps are growing more complex 3
  • 4. Streaming computations don’t run in isolation Need to interact with batch data, interactive analysis, machine learning, etc.
  • 5. Use case: IoT Device Monitoring IoT events from Kafka ETL into long term storage - Prevent data loss - Prevent duplicates Status monitoring - Handle late data - Aggregate on windows on event time Interactively debug issues - consistency event stream Anomaly detection - Learn models offline - Use online + continuous learning
  • 6. Use case: IoT Device Monitoring IoT events from Kafka ETL into long term storage - Prevent data loss - Prevent duplicates Status monitoring - Handle late data - Aggregate on windows on event time Interactively debug issues - consistency event stream Anomaly detection - Learn models offline - Use online + continuous learning Continuous Applications Not just streaming any more
  • 7. 1. Processing with event-time, dealing with late data - DStream API exposes batch time, hard to incorporate event- time 2. Interoperate streaming with batch AND interactive - RDD/DStream has similar API, but still requires translation 3. Reasoning about end-to-end guarantees - Requires carefully constructing sinks that handle failures correctly - Data consistency in the storage while being updated Pain points with DStreams
  • 9. The simplest way to perform streaming analytics is not having to reason about streaming at all
  • 10. New Model Trigger: every 1 sec 1 2 3 Time data up to 1 Input data up to 2 data up to 3 Quer y Input: data from source as an append-only table Trigger: how frequently to check input for new data Query: operations on input usual map/filter/reduce new window, session ops
  • 11. New Model Trigger: every 1 sec 1 2 3 output for data up to 1 Result Quer y Time data up to 1 Input data up to 2 output for data up to 2 data up to 3 output for data up to 3 Result: final operated table updated every trigger interval Output: what part of result to write to data sink after every trigger Complete output: Write full result table every time Output complet e output
  • 12. New Model Trigger: every 1 sec 1 2 3 output for data up to 1 Result Quer y Time data up to 1 Input data up to 2 output for data up to 2 data up to 3 output for data up to 3 Output delta output Result: final operated table updated every trigger interval Output: what part of result to write to data sink after every trigger Complete output: Write full result table every time Delta output: Write only the rows that changed in result from previous batch
  • 14. 14 Dataset A distributed bounded or unbounded collection of data with a known schema A high-level abstraction for selecting, filtering, mapping, reducing, and aggregating structured data Common API across Scala, Java, Python, R
  • 15. Standards based: Supports most popular constructs in HiveQL, ANSI SQL Compatible: Use JDBC to connect with popular BI tools 15 Datasets with SQL SELECT type, avg(signal) FROM devices GROUP BY type
  • 16. Concise: great for ad- hoc interactive analysis Interoperable: based on R / pandas, easy to go back and forth 16 Dynamic Datasets (DataFrames) ctxt.table("device-data") .groupBy("type") .agg("type", avg("signal")) .map(lambda …) .collect()
  • 17. 17 Statically-typed Datasets No boilerplate: automatically convert to/from domain objects Safe: compile time checks for correctness // Compute histogram of age by name. val hist = ds.groupBy(_.type).mapGroups { case (type, data: Iter[DeviceData]) => val buckets = new Array[Int](10) data.map(_.signal).foreach { a => buckets(a / 10) += 1 } (type, buckets) } case class DeviceData(type:String, signal:Int) // Convert data to Java objects val ds: Dataset[DeviceData] = ctx.read.json("data.json").as[Device]
  • 18. Unified, Structured APIs in Spark 2.0 18 SQL DataFrames Datasets Syntax Errors Analysis Errors Runtime Compile Time Runtime Compile Time Compile Time Runtime
  • 19. Batch ETL with DataFrames input = spark.read .format("json") .load("source-path") result = input .select("device", "signal") .where("signal > 15") result.write .format("parquet") .save("dest-path") Read from Json file Select some devices Write to parquet file
  • 20. Streaming ETL with DataFrames input = spark.readStream .format("json") .load("source-path") result = input .select("device", "signal") .where("signal > 15") result.writeStream .format("parquet") .start("dest-path") Read from Json file stream Replace read with readStream Select some devices Code does not change Write to Parquet file stream Replace save() with start()
  • 21. Streaming ETL with DataFrames read…stream() creates a streaming DataFrame, does not start any of the computation write…startStream() defines where & how to output the data and starts the processing input = spark.readStream .format("json") .load("source-path") result = input .select("device", "signal") .where("signal > 15") result.writeStream .format("parquet") .start("dest-path")
  • 22. Streaming ETL with DataFrames 1 2 3 Result [append-only table] Input Output [append mode] new rows in result of 2 new rows in result of 3 input = spark.readStream .format("json") .load("source-path") result = input .select("device", "signal") .where("signal > 15") result.writeStream .format("parquet") .start("dest-path")
  • 23. Demo
  • 24. Continuous Aggregations Continuously compute average signal across all devices Continuously compute average signal of each type of device 24 input.avg("signal") input.groupBy("device-type") .avg("signal")
  • 25. Continuous Windowed Aggregations 25 input.groupBy( $"device-type", window($"event-time-col", "10 min")) .avg("signal") Continuously compute average signal of each type of device in last 10 minutes using event- time Simplifies event-time stream processing (not possible in DStreams) Works on both, streaming and batch jobs
  • 26. Joining streams with static data kafkaDataset = spark.readStream .format("kafka") .option("subscribe", "iot-updates") .option("kafka.bootstrap.servers", ...) .load() staticDataset = ctxt.read .jdbc("jdbc://", "iot-device-info") joinedDataset = kafkaDataset.join( staticDataset, "device-type") 26 Join streaming data from Kafka with static data via JDBC to enrich the streaming data … … without having to think that you are joining streaming data
  • 27. Output Modes Defines what is outputted every time there is a trigger Different output modes make sense for different queries 27 input.select("device", "signal") .writeStream .outputMode("append") .format("parquet") .start("dest-path") Append mode with non-aggregation queries input.agg(count("*")) .writeStream .outputMode("complete") .format("parquet") .start("dest-path") Complete mode with aggregation queries
  • 28. Query Management query = result.write .format("parquet") .outputMode("append") .start("dest-path") query.stop() query.awaitTermination() query.exception() query.sourceStatuses() query.sinkStatus() 28 query: a handle to the running streaming computation for managing it - Stop it, wait for it to terminate - Get status - Get error, if terminated Multiple queries can be active at the same time Each query has unique name for keeping track
  • 29. Logically: Dataset operations on table (i.e. as easy to understand as batch) Physically: Spark automatically runs the query in streaming fashion (i.e. incrementally and continuously) DataFrame Logical Plan Continuous, incremental execution Catalyst optimizer Query Execution
  • 30. Structured Streaming High-level streaming API built on Datasets/DataFrames Event time, windowing, sessions, sources & sinks End-to-end exactly once semantics Unifies streaming, interactive and batch queries Aggregate data in a stream, then serve using JDBC Add, remove, change queries at runtime Build and apply ML models
  • 31. What can you do with this that’s hard with other engines? True unification Combines batch + streaming + interactive workloads Same code + same super-optimized engine for everything Flexible API tightly integrated with the engine Choose your own tool - Dataset/DataFrame/SQL Greater debuggability and performance Benefits of Spark in-memory computing, elastic scaling, fault-tolerance, straggler mitigation, …
  • 33. Batch Execution on Spark SQL 33 DataFrame / Dataset Logical Plan Abstract representatio n of query
  • 34. Batch Execution on Spark SQL 34 DataFrame / Dataset Logical Plan Planne r SQL AST DataFram e Unresolve d Logical Plan Logical Plan Optimized Logical Plan RDDs Selected Physical Plan Analysis Logical Optimization Physical Planning CostModel Physical Plans Code Generation CatalogDataset Helluvalot of magic!
  • 35. Batch Execution on Spark SQL 35 DataFrame / Dataset Logical Plan Execution Plan Planne r Run super-optimized Spark jobs to compute results Bytecode generation JVM intrinsics, vectorization Operations on serialized data Code Optimizations Memory Optimizations Compact and fast encoding Offheap memory Project Tungsten - Phase 1 and 2
  • 36. Continuous Incremental Execution Planner knows how to convert streaming logical plans to a continuous series of incremental execution plans, for each processing the next chunk of streaming data 36 DataFrame / Dataset Logical Plan Incremental Execution Plan 1 Incremental Execution Plan 2 Incremental Execution Plan 3 Planne r Incremental Execution Plan 4
  • 37. Continuous Incremental Execution 37 Planner Incremental Execution 2 Offsets: [106-197] Count: 92 Planner polls for new data from sources Incremental Execution 1 Offsets: [19-105] Count: 87 Incrementally executes new data and writes to sink
  • 38. Continuous Aggregations Maintain running aggregate as in-memory state backed by WAL in file system for fault-tolerance 38 state data generated and used across incremental executions Incremental Execution 1 state: 87 Offsets: [19-105] Running Count: 87 memory Incremental Execution 2 state: 179 Offsets: [106-179] Count: 87+92 = 179
  • 39. Fault-tolerance All data and metadata in the system needs to be recoverable / replayable state Planner source sink Incremental Execution 1 Incremental Execution 2
  • 40. Fault-tolerance Fault-tolerant Planner Tracks offsets by writing the offset range of each execution to a write ahead log (WAL) in HDFS state Planner source sink Offsets written to fault-tolerant WAL before execution Incremental Execution 2 Incremental Execution 1
  • 41. Fault-tolerance Fault-tolerant Planner Tracks offsets by writing the offset range of each execution to a write ahead log (WAL) in HDFS state Planner source sink Failed planner fails current execution Incremental Execution 2 Incremental Execution 1 Failed Execution Failed Planner
  • 42. Fault-tolerance Fault-tolerant Planner Tracks offsets by writing the offset range of each execution to a write ahead log (WAL) in HDFS Reads log to recover from failures, and re-execute exact range of offsets state Restarte d Planner source sink Offsets read back from WAL Incremental Execution 1 Same executions regenerated from offsets Failed Execution Incremental Execution 2
  • 43. Fault-tolerance Fault-tolerant Sources Structured streaming sources are by design replayable (e.g. Kafka, Kinesis, files) and generate the exactly same data given offsets recovered by planner state Planner sink Incremental Execution 1 Incremental Execution 2 source Replayable source
  • 44. Fault-tolerance Fault-tolerant State Intermediate "state data" is a maintained in versioned, key- value maps in Spark workers, backed by HDFS Planner makes sure "correct version" of state used to re- execute after failure Planner source sink Incremental Execution 1 Incremental Execution 2 state state is fault-tolerant with WAL
  • 45. Fault-tolerance Fault-tolerant Sink Sink are by design idempotent, and handles re- executions to avoid double committing the output Planner source Incremental Execution 1 Incremental Execution 2 state sink Idempotent by design
  • 46. 46 offset tracking in WAL + state management + fault-tolerant sources and sinks = end-to-end exactly-once guarantees
  • 47. 47 Fast, fault-tolerant, exactly-once stateful stream processing without having to reason about streaming
  • 48. Blogs for Deeper Technical Dive Blog 1: https://databricks.com/blog/2016/07/2 8/continuous-applications-evolving- streaming-in-apache-spark-2-0.html Blog 2: https://databricks.com/blog/2016/07/2 8/structured-streaming-in-apache- spark.html
  • 49. Comparison with Other Engines 49 Read the blog to understand this table
  • 50. Current Status: Spark 2.0 Basic infrastructure and API - Event time, windows, aggregations - Append and Complete output modes - Support for a subset of batch queries Source and sink - Sources: Files - Sinks: Files and in-memory table Experimental release to set the future direction Not ready for production but good to experiment with and provide feedback
  • 52. Future Direction Metrics Stability and scalability Support for more queries Watermarks and late data Sessionization More output modes ML integrations Make Structured Streaming ready for production workloads as soon as possible
  • 53. Stay tuned on our Databricks blogs for more information and examples on Structured Streaming Try latest version of Apache Spark Try Apache Spark with Databricks 53 http://databricks.com/try
  • 54. Structured Streaming Making Continuous Applications easier, faster, and smarter Follow me @tathadas