SlideShare a Scribd company logo
1 of 46
Download to read offline
A Deep Dive into
Structured Streaming
Tathagata “TD” Das
@tathadas
Spark Summit 2016
Who am I?
Project Mgmt. Committee (PMC) member of Apache Spark
Started Spark Streaming in grad school - AMPLab, UC Berkeley
Software engineerat Databricks and involved with all things
streaming in Spark
2
Streaming in Apache Spark
Spark Streaming changedhow peoplewrite streaming apps
3
SQL Streaming MLlib
Spark Core
GraphX
Functional, conciseand expressive
Fault-tolerant statemanagement
Unified stack with batch processing
More than 50%users consider most important partof Apache Spark
Streaming apps are
growing more complex
4
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
- Preventdata loss
- PreventduplicatesStatus monitoring
- Handlelate data
- Aggregateon windows
on eventtime
Interactively
debug issues
- consistency
event stream
Anomaly detection
- Learn modelsoffline
- Use online+continuous
learning
Use case: IoT Device Monitoring
IoT events
from Kafka
ETL into long term storage
- Preventdata loss
- PreventduplicatesStatus monitoring
- Handlelate data
- Aggregateon windows
on eventtime
Interactively
debug issues
- consistency
event stream
Anomaly detection
- Learn modelsoffline
- 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. Interoperatestreaming with batch AND interactive
- RDD/DStream hassimilar API, butstill requirestranslation
3. Reasoning about end-to-end guarantees
- Requirescarefully constructing sinks that handle failurescorrectly
- 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
Query
Input: data from source as an
append-only table
Trigger: howfrequently to check
input for newdata
Query: operations on input
usual map/filter/reduce
newwindow, session ops
New Model Trigger: every 1 sec
1 2 3
output
for data
up to 1
Result
Query
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 triggerinterval
Output: what part of result to write
to data sink after every trigger
Complete output: Write full result table every time
Output
complete
output
New Model Trigger: every 1 sec
1 2 3
output
for data
up to 1
Result
Query
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 triggerinterval
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
Append output: Write only new rows
*Not all output modes are feasible withall queries
Static, bounded
data
Streaming, unbounded
data
Single API !
API - Dataset/DataFrame
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.read
.format("json")
.stream("source-path")
result = input
.select("device", "signal")
.where("signal > 15")
result.write
.format("parquet")
.startStream("dest-path")
Read from Json file stream
Replace load() with stream()
Select some devices
Code does not change
Write to Parquet file stream
Replace save() with startStream()
Streaming ETL with DataFrames
input = spark.read
.format("json")
.stream("source-path")
result = input
.select("device", "signal")
.where("signal > 15")
result.write
.format("parquet")
.startStream("dest-path")
read…stream() creates a streaming
DataFrame, doesnot start any of the
computation
write…startStream() defineswhere & how
to outputthe data and starts the
processing
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.read
.format("json")
.stream("source-path")
result = input
.select("device", "signal")
.where("signal > 15")
result.write
.format("parquet")
.startStream("dest-path")
Continuous Aggregations
Continuously compute average
signal across all devices
Continuously compute average
signal of each type of device
19
input.avg("signal")
input.groupBy("device-type")
.avg("signal")
Continuous Windowed Aggregations
20
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
Simplifiesevent-time stream processing (notpossible in DStreams)
Works on both, streaming and batch jobs
Joining streams with static data
kafkaDataset = spark.read
.kafka("iot-updates")
.stream()
staticDataset = ctxt.read
.jdbc("jdbc://", "iot-device-info")
joinedDataset =
kafkaDataset.join(
staticDataset, "device-type")
21
Join streaming data from Kafka with
static data via JDBC to enrich the
streaming data …
… withouthaving to thinkthat you
are joining streaming data
Output Modes
Defines what is outputted every time there is a trigger
Different output modes make sensefor different queries
22
input.select("device", "signal")
.write
.outputMode("append")
.format("parquet")
.startStream("dest-path")
Append mode with
non-aggregation queries
input.agg(count("*"))
.write
.outputMode("complete")
.format("parquet")
.startStream("dest-path")
Complete mode with
aggregation queries
Query Management
query = result.write
.format("parquet")
.outputMode("append")
.startStream("dest-path")
query.stop()
query.awaitTermination()
query.exception()
query.sourceStatuses()
query.sinkStatus()
23
query: a handle to the running streaming
computation for managingit
- 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 keepingtrack
Logically:
Dataset operations on table
(i.e. as easyto understand as batch)
Physically:
Spark automatically runs the queryin
streaming fashion
(i.e. incrementally and continuously)
DataFrame
LogicalPlan
Continuous,
incrementalexecution
Catalyst optimizer
Query Execution
Structured Streaming
High-level streaming API built on Datasets/DataFrames
Eventtime, 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 queriesat runtime
Build and apply ML models
What can you do with this that’s hard
with other engines?
True unification
Same code + same super-optimized engine for everything
Flexible API tightly integratedwith the engine
Choose your own tool - Dataset/DataFrame/SQL
Greater debuggability and performance
Benefitsof Spark
in-memory computing, elastic scaling, fault-tolerance, straggler mitigation, …
Underneath the Hood
Batch Execution on Spark SQL
28
DataFrame/
Dataset
Logical
Plan
Abstract
representation
of query
Batch Execution on Spark SQL
29
DataFrame/
Dataset
Logical
Plan
Planner
SQL AST
DataFrame
Unresolved
Logical Plan
Logical Plan
Optimized
Logical Plan
RDDs
Selected
Physical
Plan
Analysis
Logical
Optimization
Physical
Planning
CostModel
Physical
Plans
Code
Generation
CatalogDataset
Helluvalotofmagic!
Batch Execution on Spark SQL
30
DataFrame/
Dataset
Logical
Plan
Execution PlanPlanner
Run super-optimized Spark
jobsto compute results
Bytecode generation
JVM intrinsics, vectorization
Operations on serialized data
Code Optimizations MemoryOptimizations
Compact and fastencoding
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 eachprocessing
the nextchunk of streaming data
31
DataFrame/
Dataset
Logical
Plan
Incremental
Execution Plan 1
Incremental
Execution Plan 2
Incremental
Execution Plan 3
Planner
Incremental
Execution Plan 4
Continuous Incremental Execution
32
Planner
Incremental
Execution 2
Offsets:[106-197] Count: 92
Plannerpollsfor
new data from
sources
Incremental
Execution 1
Offsets:[19-105] Count: 87
Incrementally executes
new data and writesto sink
Continuous Aggregations
Maintain runningaggregate as in-memory state
backed by WAL in file system for fault-tolerance
33
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 needsto 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
Restarted
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
Plannermakes 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 handlesre-executionsto
avoid double committing the
output
Planner
source
Incremental
Execution 1
Incremental
Execution 2
state
sink
Idempotent
by design
41
offset tracking in WAL
+
state management
+
fault-tolerant sourcesand sinks
=
end-to-end
exactly-once
guarantees
42
Fast, fault-tolerant, exactly-once
stateful stream processing
without having to reason about streaming
Release Plan: Spark 2.0 [June 2016]
Basic infrastructureand API
- Eventtime, windows,aggregations
- Append and Complete output modes
- Support for a subsetof batch queries
Sourceand sink
- Sources: Files(*Kafka coming soon
after 2.0 release)
- Sinks: Filesand in-memory table
Experimental release to set
the future direction
Not ready for production
but good to experiment
with and provide feedback
Release Plan: Spark 2.1+
Stability and scalability
Supportfor more queries
Multiple aggregations
Sessionization
More outputmodes
Watermarks and late data
Sourcesand Sinks
Public APIs
ML integrations
Make Structured
Streaming readyfor
production workloads as
soon as possible
Stay tuned on our Databricks blogsfor more information and
examples on Structured Streaming
Try latestversion of ApacheSpark and preview of Spark 2.0
Try Apache Spark with Databricks
45
http://databricks.com/try
Structured Streaming
Making Continuous Applications
easier, faster, and smarter
Follow me @tathadas
AMA @
Databricks Booth
Today: Now - 2:00 PM
Tomorrow: 12:15 PM - 1:00 PM

More Related Content

What's hot

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
 

What's hot (20)

Making Structured Streaming Ready for Production
Making Structured Streaming Ready for ProductionMaking Structured Streaming Ready for Production
Making Structured Streaming Ready for Production
 
Spark SQL Deep Dive @ Melbourne Spark Meetup
Spark SQL Deep Dive @ Melbourne Spark MeetupSpark SQL Deep Dive @ Melbourne Spark Meetup
Spark SQL Deep Dive @ Melbourne Spark Meetup
 
Apache Flink: Real-World Use Cases for Streaming Analytics
Apache Flink: Real-World Use Cases for Streaming AnalyticsApache Flink: Real-World Use Cases for Streaming Analytics
Apache Flink: Real-World Use Cases for Streaming Analytics
 
Spark with Delta Lake
Spark with Delta LakeSpark with Delta Lake
Spark with Delta Lake
 
Kafka Connect - debezium
Kafka Connect - debeziumKafka Connect - debezium
Kafka Connect - debezium
 
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
 
Apache Spark Core – Practical Optimization
Apache Spark Core – Practical OptimizationApache Spark Core – Practical Optimization
Apache Spark Core – Practical Optimization
 
Bucketing 2.0: Improve Spark SQL Performance by Removing Shuffle
Bucketing 2.0: Improve Spark SQL Performance by Removing ShuffleBucketing 2.0: Improve Spark SQL Performance by Removing Shuffle
Bucketing 2.0: Improve Spark SQL Performance by Removing Shuffle
 
Productizing Structured Streaming Jobs
Productizing Structured Streaming JobsProductizing Structured Streaming Jobs
Productizing Structured Streaming Jobs
 
Delta Lake: Optimizing Merge
Delta Lake: Optimizing MergeDelta Lake: Optimizing Merge
Delta Lake: Optimizing Merge
 
Physical Plans in Spark SQL
Physical Plans in Spark SQLPhysical Plans in Spark SQL
Physical Plans in Spark SQL
 
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...
 
HBaseCon 2015: Taming GC Pauses for Large Java Heap in HBase
HBaseCon 2015: Taming GC Pauses for Large Java Heap in HBaseHBaseCon 2015: Taming GC Pauses for Large Java Heap in HBase
HBaseCon 2015: Taming GC Pauses for Large Java Heap in HBase
 
Deep Dive into Spark SQL with Advanced Performance Tuning with Xiao Li & Wenc...
Deep Dive into Spark SQL with Advanced Performance Tuning with Xiao Li & Wenc...Deep Dive into Spark SQL with Advanced Performance Tuning with Xiao Li & Wenc...
Deep Dive into Spark SQL with Advanced Performance Tuning with Xiao Li & Wenc...
 
Near Real-Time Data Warehousing with Apache Spark and Delta Lake
Near Real-Time Data Warehousing with Apache Spark and Delta LakeNear Real-Time Data Warehousing with Apache Spark and Delta Lake
Near Real-Time Data Warehousing with Apache Spark and Delta Lake
 
A Deep Dive into Query Execution Engine of Spark SQL
A Deep Dive into Query Execution Engine of Spark SQLA Deep Dive into Query Execution Engine of Spark SQL
A Deep Dive into Query Execution Engine of Spark SQL
 
Introduction to PySpark
Introduction to PySparkIntroduction to PySpark
Introduction to PySpark
 
Apache Spark Introduction and Resilient Distributed Dataset basics and deep dive
Apache Spark Introduction and Resilient Distributed Dataset basics and deep diveApache Spark Introduction and Resilient Distributed Dataset basics and deep dive
Apache Spark Introduction and Resilient Distributed Dataset basics and deep dive
 
Apache Flink @ NYC Flink Meetup
Apache Flink @ NYC Flink MeetupApache Flink @ NYC Flink Meetup
Apache Flink @ NYC Flink Meetup
 
Streaming Data Lakes using Kafka Connect + Apache Hudi | Vinoth Chandar, Apac...
Streaming Data Lakes using Kafka Connect + Apache Hudi | Vinoth Chandar, Apac...Streaming Data Lakes using Kafka Connect + Apache Hudi | Vinoth Chandar, Apac...
Streaming Data Lakes using Kafka Connect + Apache Hudi | Vinoth Chandar, Apac...
 

Similar to Apache Spark 2.0: A Deep Dive Into Structured Streaming - by Tathagata Das

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
Angel Giraldo
 
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
 
2007 Tidc India Profiling
2007 Tidc India Profiling2007 Tidc India Profiling
2007 Tidc India Profiling
danrinkes
 
How to use Parquet as a Sasis for ETL and Analytics
How to use Parquet as a Sasis for ETL and AnalyticsHow to use Parquet as a Sasis for ETL and Analytics
How to use Parquet as a Sasis for ETL and Analytics
DataWorks Summit
 

Similar to Apache Spark 2.0: A Deep Dive Into Structured Streaming - by Tathagata Das (20)

A Deep Dive into Structured Streaming: Apache Spark Meetup at Bloomberg 2016
A Deep Dive into Structured Streaming:  Apache Spark Meetup at Bloomberg 2016 A Deep Dive into Structured Streaming:  Apache Spark Meetup at Bloomberg 2016
A Deep Dive into Structured Streaming: Apache Spark Meetup at Bloomberg 2016
 
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
 
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
 
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
 
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
 
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
 
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...
 
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?
 
2007 Tidc India Profiling
2007 Tidc India Profiling2007 Tidc India Profiling
2007 Tidc India Profiling
 
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...
 
Introduction to Structured Streaming
Introduction to Structured StreamingIntroduction to Structured Streaming
Introduction to Structured Streaming
 
Dsp lab manual 15 11-2016
Dsp lab manual 15 11-2016Dsp lab manual 15 11-2016
Dsp lab manual 15 11-2016
 
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
 
Flink 0.10 @ Bay Area Meetup (October 2015)
Flink 0.10 @ Bay Area Meetup (October 2015)Flink 0.10 @ Bay Area Meetup (October 2015)
Flink 0.10 @ Bay Area Meetup (October 2015)
 
Spark streaming
Spark streamingSpark streaming
Spark streaming
 
How to use Parquet as a Sasis for ETL and Analytics
How to use Parquet as a Sasis for ETL and AnalyticsHow to use Parquet as a Sasis for ETL and Analytics
How to use Parquet as a Sasis for ETL and Analytics
 
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
 
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 - ...
 

More from Databricks

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
Databricks
 
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
Databricks
 
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
Databricks
 
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
Databricks
 
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
Databricks
 

More from 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
 
Machine Learning CI/CD for Email Attack Detection
Machine Learning CI/CD for Email Attack DetectionMachine Learning CI/CD for Email Attack Detection
Machine Learning CI/CD for Email Attack Detection
 

Recently uploaded

%+27788225528 love spells in Colorado Springs Psychic Readings, Attraction sp...
%+27788225528 love spells in Colorado Springs Psychic Readings, Attraction sp...%+27788225528 love spells in Colorado Springs Psychic Readings, Attraction sp...
%+27788225528 love spells in Colorado Springs Psychic Readings, Attraction sp...
masabamasaba
 
%+27788225528 love spells in Boston Psychic Readings, Attraction spells,Bring...
%+27788225528 love spells in Boston Psychic Readings, Attraction spells,Bring...%+27788225528 love spells in Boston Psychic Readings, Attraction spells,Bring...
%+27788225528 love spells in Boston Psychic Readings, Attraction spells,Bring...
masabamasaba
 
Love witchcraft +27768521739 Binding love spell in Sandy Springs, GA |psychic...
Love witchcraft +27768521739 Binding love spell in Sandy Springs, GA |psychic...Love witchcraft +27768521739 Binding love spell in Sandy Springs, GA |psychic...
Love witchcraft +27768521739 Binding love spell in Sandy Springs, GA |psychic...
chiefasafspells
 

Recently uploaded (20)

WSO2CON 2024 - Building the API First Enterprise – Running an API Program, fr...
WSO2CON 2024 - Building the API First Enterprise – Running an API Program, fr...WSO2CON 2024 - Building the API First Enterprise – Running an API Program, fr...
WSO2CON 2024 - Building the API First Enterprise – Running an API Program, fr...
 
%+27788225528 love spells in Colorado Springs Psychic Readings, Attraction sp...
%+27788225528 love spells in Colorado Springs Psychic Readings, Attraction sp...%+27788225528 love spells in Colorado Springs Psychic Readings, Attraction sp...
%+27788225528 love spells in Colorado Springs Psychic Readings, Attraction sp...
 
WSO2CON 2024 - API Management Usage at La Poste and Its Impact on Business an...
WSO2CON 2024 - API Management Usage at La Poste and Its Impact on Business an...WSO2CON 2024 - API Management Usage at La Poste and Its Impact on Business an...
WSO2CON 2024 - API Management Usage at La Poste and Its Impact on Business an...
 
%in Rustenburg+277-882-255-28 abortion pills for sale in Rustenburg
%in Rustenburg+277-882-255-28 abortion pills for sale in Rustenburg%in Rustenburg+277-882-255-28 abortion pills for sale in Rustenburg
%in Rustenburg+277-882-255-28 abortion pills for sale in Rustenburg
 
%+27788225528 love spells in Boston Psychic Readings, Attraction spells,Bring...
%+27788225528 love spells in Boston Psychic Readings, Attraction spells,Bring...%+27788225528 love spells in Boston Psychic Readings, Attraction spells,Bring...
%+27788225528 love spells in Boston Psychic Readings, Attraction spells,Bring...
 
Artyushina_Guest lecture_YorkU CS May 2024.pptx
Artyushina_Guest lecture_YorkU CS May 2024.pptxArtyushina_Guest lecture_YorkU CS May 2024.pptx
Artyushina_Guest lecture_YorkU CS May 2024.pptx
 
WSO2CON 2024 - Cloud Native Middleware: Domain-Driven Design, Cell-Based Arch...
WSO2CON 2024 - Cloud Native Middleware: Domain-Driven Design, Cell-Based Arch...WSO2CON 2024 - Cloud Native Middleware: Domain-Driven Design, Cell-Based Arch...
WSO2CON 2024 - Cloud Native Middleware: Domain-Driven Design, Cell-Based Arch...
 
%in Benoni+277-882-255-28 abortion pills for sale in Benoni
%in Benoni+277-882-255-28 abortion pills for sale in Benoni%in Benoni+277-882-255-28 abortion pills for sale in Benoni
%in Benoni+277-882-255-28 abortion pills for sale in Benoni
 
%in Stilfontein+277-882-255-28 abortion pills for sale in Stilfontein
%in Stilfontein+277-882-255-28 abortion pills for sale in Stilfontein%in Stilfontein+277-882-255-28 abortion pills for sale in Stilfontein
%in Stilfontein+277-882-255-28 abortion pills for sale in Stilfontein
 
%in kaalfontein+277-882-255-28 abortion pills for sale in kaalfontein
%in kaalfontein+277-882-255-28 abortion pills for sale in kaalfontein%in kaalfontein+277-882-255-28 abortion pills for sale in kaalfontein
%in kaalfontein+277-882-255-28 abortion pills for sale in kaalfontein
 
%in tembisa+277-882-255-28 abortion pills for sale in tembisa
%in tembisa+277-882-255-28 abortion pills for sale in tembisa%in tembisa+277-882-255-28 abortion pills for sale in tembisa
%in tembisa+277-882-255-28 abortion pills for sale in tembisa
 
WSO2CON 2024 Slides - Open Source to SaaS
WSO2CON 2024 Slides - Open Source to SaaSWSO2CON 2024 Slides - Open Source to SaaS
WSO2CON 2024 Slides - Open Source to SaaS
 
Love witchcraft +27768521739 Binding love spell in Sandy Springs, GA |psychic...
Love witchcraft +27768521739 Binding love spell in Sandy Springs, GA |psychic...Love witchcraft +27768521739 Binding love spell in Sandy Springs, GA |psychic...
Love witchcraft +27768521739 Binding love spell in Sandy Springs, GA |psychic...
 
%in Hazyview+277-882-255-28 abortion pills for sale in Hazyview
%in Hazyview+277-882-255-28 abortion pills for sale in Hazyview%in Hazyview+277-882-255-28 abortion pills for sale in Hazyview
%in Hazyview+277-882-255-28 abortion pills for sale in Hazyview
 
WSO2Con2024 - WSO2's IAM Vision: Identity-Led Digital Transformation
WSO2Con2024 - WSO2's IAM Vision: Identity-Led Digital TransformationWSO2Con2024 - WSO2's IAM Vision: Identity-Led Digital Transformation
WSO2Con2024 - WSO2's IAM Vision: Identity-Led Digital Transformation
 
WSO2Con2024 - From Code To Cloud: Fast Track Your Cloud Native Journey with C...
WSO2Con2024 - From Code To Cloud: Fast Track Your Cloud Native Journey with C...WSO2Con2024 - From Code To Cloud: Fast Track Your Cloud Native Journey with C...
WSO2Con2024 - From Code To Cloud: Fast Track Your Cloud Native Journey with C...
 
VTU technical seminar 8Th Sem on Scikit-learn
VTU technical seminar 8Th Sem on Scikit-learnVTU technical seminar 8Th Sem on Scikit-learn
VTU technical seminar 8Th Sem on Scikit-learn
 
Crypto Cloud Review - How To Earn Up To $500 Per DAY Of Bitcoin 100% On AutoP...
Crypto Cloud Review - How To Earn Up To $500 Per DAY Of Bitcoin 100% On AutoP...Crypto Cloud Review - How To Earn Up To $500 Per DAY Of Bitcoin 100% On AutoP...
Crypto Cloud Review - How To Earn Up To $500 Per DAY Of Bitcoin 100% On AutoP...
 
tonesoftg
tonesoftgtonesoftg
tonesoftg
 
%in kempton park+277-882-255-28 abortion pills for sale in kempton park
%in kempton park+277-882-255-28 abortion pills for sale in kempton park %in kempton park+277-882-255-28 abortion pills for sale in kempton park
%in kempton park+277-882-255-28 abortion pills for sale in kempton park
 

Apache Spark 2.0: A Deep Dive Into Structured Streaming - by Tathagata Das

  • 1. A Deep Dive into Structured Streaming Tathagata “TD” Das @tathadas Spark Summit 2016
  • 2. Who am I? Project Mgmt. Committee (PMC) member of Apache Spark Started Spark Streaming in grad school - AMPLab, UC Berkeley Software engineerat Databricks and involved with all things streaming in Spark 2
  • 3. Streaming in Apache Spark Spark Streaming changedhow peoplewrite streaming apps 3 SQL Streaming MLlib Spark Core GraphX Functional, conciseand expressive Fault-tolerant statemanagement Unified stack with batch processing More than 50%users consider most important partof Apache Spark
  • 4. Streaming apps are growing more complex 4
  • 5. Streaming computations don’t run in isolation Need to interact with batch data, interactive analysis, machine learning, etc.
  • 6. Use case: IoT Device Monitoring IoT events from Kafka ETL into long term storage - Preventdata loss - PreventduplicatesStatus monitoring - Handlelate data - Aggregateon windows on eventtime Interactively debug issues - consistency event stream Anomaly detection - Learn modelsoffline - Use online+continuous learning
  • 7. Use case: IoT Device Monitoring IoT events from Kafka ETL into long term storage - Preventdata loss - PreventduplicatesStatus monitoring - Handlelate data - Aggregateon windows on eventtime Interactively debug issues - consistency event stream Anomaly detection - Learn modelsoffline - Use online+continuous learning Continuous Applications Not just streaming any more
  • 8. 1. Processing with event-time, dealing with late data - DStream API exposes batch time, hard to incorporate event-time 2. Interoperatestreaming with batch AND interactive - RDD/DStream hassimilar API, butstill requirestranslation 3. Reasoning about end-to-end guarantees - Requirescarefully constructing sinks that handle failurescorrectly - Data consistency in the storage while being updated Pain points with DStreams
  • 10. The simplest way to perform streaming analytics is not having to reason about streaming at all
  • 11. New Model Trigger: every 1 sec 1 2 3 Time data up to 1 Input data up to 2 data up to 3 Query Input: data from source as an append-only table Trigger: howfrequently to check input for newdata Query: operations on input usual map/filter/reduce newwindow, session ops
  • 12. New Model Trigger: every 1 sec 1 2 3 output for data up to 1 Result Query 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 triggerinterval Output: what part of result to write to data sink after every trigger Complete output: Write full result table every time Output complete output
  • 13. New Model Trigger: every 1 sec 1 2 3 output for data up to 1 Result Query 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 triggerinterval 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 Append output: Write only new rows *Not all output modes are feasible withall queries
  • 15. 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
  • 16. Streaming ETL with DataFrames input = spark.read .format("json") .stream("source-path") result = input .select("device", "signal") .where("signal > 15") result.write .format("parquet") .startStream("dest-path") Read from Json file stream Replace load() with stream() Select some devices Code does not change Write to Parquet file stream Replace save() with startStream()
  • 17. Streaming ETL with DataFrames input = spark.read .format("json") .stream("source-path") result = input .select("device", "signal") .where("signal > 15") result.write .format("parquet") .startStream("dest-path") read…stream() creates a streaming DataFrame, doesnot start any of the computation write…startStream() defineswhere & how to outputthe data and starts the processing
  • 18. 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.read .format("json") .stream("source-path") result = input .select("device", "signal") .where("signal > 15") result.write .format("parquet") .startStream("dest-path")
  • 19. Continuous Aggregations Continuously compute average signal across all devices Continuously compute average signal of each type of device 19 input.avg("signal") input.groupBy("device-type") .avg("signal")
  • 20. Continuous Windowed Aggregations 20 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 Simplifiesevent-time stream processing (notpossible in DStreams) Works on both, streaming and batch jobs
  • 21. Joining streams with static data kafkaDataset = spark.read .kafka("iot-updates") .stream() staticDataset = ctxt.read .jdbc("jdbc://", "iot-device-info") joinedDataset = kafkaDataset.join( staticDataset, "device-type") 21 Join streaming data from Kafka with static data via JDBC to enrich the streaming data … … withouthaving to thinkthat you are joining streaming data
  • 22. Output Modes Defines what is outputted every time there is a trigger Different output modes make sensefor different queries 22 input.select("device", "signal") .write .outputMode("append") .format("parquet") .startStream("dest-path") Append mode with non-aggregation queries input.agg(count("*")) .write .outputMode("complete") .format("parquet") .startStream("dest-path") Complete mode with aggregation queries
  • 23. Query Management query = result.write .format("parquet") .outputMode("append") .startStream("dest-path") query.stop() query.awaitTermination() query.exception() query.sourceStatuses() query.sinkStatus() 23 query: a handle to the running streaming computation for managingit - 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 keepingtrack
  • 24. Logically: Dataset operations on table (i.e. as easyto understand as batch) Physically: Spark automatically runs the queryin streaming fashion (i.e. incrementally and continuously) DataFrame LogicalPlan Continuous, incrementalexecution Catalyst optimizer Query Execution
  • 25. Structured Streaming High-level streaming API built on Datasets/DataFrames Eventtime, 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 queriesat runtime Build and apply ML models
  • 26. What can you do with this that’s hard with other engines? True unification Same code + same super-optimized engine for everything Flexible API tightly integratedwith the engine Choose your own tool - Dataset/DataFrame/SQL Greater debuggability and performance Benefitsof Spark in-memory computing, elastic scaling, fault-tolerance, straggler mitigation, …
  • 28. Batch Execution on Spark SQL 28 DataFrame/ Dataset Logical Plan Abstract representation of query
  • 29. Batch Execution on Spark SQL 29 DataFrame/ Dataset Logical Plan Planner SQL AST DataFrame Unresolved Logical Plan Logical Plan Optimized Logical Plan RDDs Selected Physical Plan Analysis Logical Optimization Physical Planning CostModel Physical Plans Code Generation CatalogDataset Helluvalotofmagic!
  • 30. Batch Execution on Spark SQL 30 DataFrame/ Dataset Logical Plan Execution PlanPlanner Run super-optimized Spark jobsto compute results Bytecode generation JVM intrinsics, vectorization Operations on serialized data Code Optimizations MemoryOptimizations Compact and fastencoding Offheap memory Project Tungsten -Phase 1 and 2
  • 31. Continuous Incremental Execution Planner knows how to convert streaming logical plans to a continuous series of incremental execution plans, for eachprocessing the nextchunk of streaming data 31 DataFrame/ Dataset Logical Plan Incremental Execution Plan 1 Incremental Execution Plan 2 Incremental Execution Plan 3 Planner Incremental Execution Plan 4
  • 32. Continuous Incremental Execution 32 Planner Incremental Execution 2 Offsets:[106-197] Count: 92 Plannerpollsfor new data from sources Incremental Execution 1 Offsets:[19-105] Count: 87 Incrementally executes new data and writesto sink
  • 33. Continuous Aggregations Maintain runningaggregate as in-memory state backed by WAL in file system for fault-tolerance 33 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
  • 34. Fault-tolerance All data and metadata in the system needsto be recoverable/ replayable state Planner source sink Incremental Execution 1 Incremental Execution 2
  • 35. 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
  • 36. 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
  • 37. 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 Restarted Planner source sink Offsets read back from WAL Incremental Execution 1 Same executions regenerated from offsets Failed Execution Incremental Execution 2
  • 38. 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
  • 39. Fault-tolerance Fault-tolerant State Intermediate "state data" is a maintained in versioned,key- value maps in Spark workers, backed by HDFS Plannermakes 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
  • 40. Fault-tolerance Fault-tolerant Sink Sink are by design idempotent, and handlesre-executionsto avoid double committing the output Planner source Incremental Execution 1 Incremental Execution 2 state sink Idempotent by design
  • 41. 41 offset tracking in WAL + state management + fault-tolerant sourcesand sinks = end-to-end exactly-once guarantees
  • 42. 42 Fast, fault-tolerant, exactly-once stateful stream processing without having to reason about streaming
  • 43. Release Plan: Spark 2.0 [June 2016] Basic infrastructureand API - Eventtime, windows,aggregations - Append and Complete output modes - Support for a subsetof batch queries Sourceand sink - Sources: Files(*Kafka coming soon after 2.0 release) - Sinks: Filesand in-memory table Experimental release to set the future direction Not ready for production but good to experiment with and provide feedback
  • 44. Release Plan: Spark 2.1+ Stability and scalability Supportfor more queries Multiple aggregations Sessionization More outputmodes Watermarks and late data Sourcesand Sinks Public APIs ML integrations Make Structured Streaming readyfor production workloads as soon as possible
  • 45. Stay tuned on our Databricks blogsfor more information and examples on Structured Streaming Try latestversion of ApacheSpark and preview of Spark 2.0 Try Apache Spark with Databricks 45 http://databricks.com/try
  • 46. Structured Streaming Making Continuous Applications easier, faster, and smarter Follow me @tathadas AMA @ Databricks Booth Today: Now - 2:00 PM Tomorrow: 12:15 PM - 1:00 PM