SlideShare a Scribd company logo
1 of 40
Download to read offline
Taking Spark Streaming to the next
level with Datasets and DataFrames
Tathagata “TD” Das
@tathadas
Strata San Jose 2016
Streaming in Spark
Spark Streaming changedhow peoplewrite streaming apps
2
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 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
Devices
ETL into long term storage
- Preventdata loss
- PreventduplicatesStatus monitoring
- Handlelate data
- Process based on
eventtime
Interactively
debug issues
- consistency
event stream
Anomaly detection
- Learn modelsoffline
- Use online+continuous
learning
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
- Hard to interoperate with DataFrames and Datasets
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
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
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
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 table
Dataset, DataFrame
Streaming,
unbounded table
Single API !
Batch ETL with DataFrame
input = ctxt.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 DataFrame
input = ctxt.read
.format("json")
.stream("source-path")
result = input
.select("device", "signal")
.where("signal > 15")
result.write
.format("parquet")
.outputMode("append")
.startStream("dest-path")
Read from Json file stream
Select some devices
Write to parquet file stream
Streaming ETL with DataFrame
input = ctxt.read
.format("json")
.stream("source-path")
result = input
.select("device", "signal")
.where("signal > 15")
result.write
.format("parquet")
.outputMode("append")
.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 DataFrame
input = ctxt.read
.format("json")
.stream("source-path")
result = input
.select("device", "signal")
.where("signal > 15")
result.write
.format("parquet")
.outputMode("append")
.startStream("dest-path")
1 2 3
Result
[append-only table]
Input
Output
[append mode]
new rows
in result
of 2
new rows
in result
of 3
Continuous Aggregations
Continuously compute average
signal of each type of device
17
input.groupBy("device-type")
.avg("signal")
input.groupBy(
window("event-time", "10min"),
"device type")
.avg("age")
Continuously compute average signal of
each type of device in last10 minutesof
eventtime
- Windowing is just a type of aggregation
- Simple API for event time based windowing
Query Management
query = result.write
.format("parquet")
.outputMode("append")
.startStream("dest-path")
query.stop()
query.awaitTermination()
query.exception()
query.sourceStatuses()
query.sinkStatus()
18
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:
DataFrame 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
Execution
Structured Streaming
High-level streaming API built on SparkSQL engine
Runs the same computation as batch queriesin 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
Advantages over DStreams
1. Processingwith event-time,dealingwith late data
2. Exactly same API for batch,streaming,and interactive
3. End-to-endexactly-once guaranteesfromthe system
4. Performance through SQL optimizations
- Logical plan optimizations, Tungsten, Codegen, etc.
- Faster state management for stateful stream processing
21
Underneath the Hood
Batch Execution on Spark SQL
23
DataFrame/
Dataset
Logical
Plan
Execution PlanPlanner
Logical plan
optimization
Execution plan
generation
RDD job
Tungsten
Code generation
Abstract
representation
of query
Continuous Incremental Execution
Planner extendedto be aware of the
streaminglogical plans
Planner generatesa continuousseriesof
incrementalexecutionplans, each
processingthe next chunk ofstreaming data
24
DataFrame/
Dataset
Logical
Plan
Incremental
Execution Plan 1
Incremental
Execution Plan 2
Incremental
Execution Plan 3
Planner
Incremental
Execution Plan 4
Streaming Source
A streaming data source where
- Records can be uniquely identified by a offset
- Arbitrary segmentof the stream data can be read
based on an offset range
- Files,Kafka, Kinesissupported
More restricted than DStream Receivers
Can ensure end-to-end exactly-once guarantees
25
Logical Plan
Sources
Sink
Operations
Streaming Sink
Allows output to pushed to externalstorage
Each batch of output should be written to
storage atomically and idempotently
Both neededto ensureend-to-end exactly-
once guarantees,AND data consistency
26
Logical Plan
Sources
Sink
Operations
Incremental Execution
Every trigger interval
- Plannerask source for
next chunkof inputdata
- Generatesexecution
plan with the input
- Generate output data
- Handsto sink for
pushing itout
27
Logical Plan
Sources
Sink
Ops
Incremental
Execution Plan
Input
get new data
as input
Output
push output
generate
outputPlanner
Offset Tracking with WAL
Plannersaves the next
offset range in a write-
ahead log (WAL) on
HDFS/S3 before
incremental execution
starts
28
Logical Plan
Sources
Incremental
Execution Plan
Planner
processed
offsets
Offset WAL
in-progress
offsets
save offset range to WAL
before processing starts
Input
Recovery from WAL
29
Logical Plan
Sources
Incremental
Execution PlanRestarted
Planner
processed
offsets
in-progress
offsets
Offset WAL
recover last offset range
and restart computation
After failure, restarted
plannerrecoverslast
offset range from WAL
and restarts failed
execution
Output exactly same as
it would been without
failure
Input
30
Streaming source
+
Streaming sink
+
Offset tracking
=
End-to-end
exactly-once
guarantees
Stateful Stream Processing
Streaming aggregationsrequire maintaining intermediate
"state" data acrossbatches
31
compute
aggregates 3
Dog
Cat 2
Dog 1
Cat 3
Dog 1
Cow 1
state data
compute
aggregates 2
Cat, Cow
compute
aggregates 1
Cat, Dog, Cat
aggregates
State Management
State needsto be fault-tolerant
Cat, Dog, Cat Cat, Cow Dog
Cat 2
Dog 1
Cat 3
Dog 1
Cow 1aggregates
both, input + previous
state needed to
recover from failure
Cat 3
Dog 1
Cow 1
Dog
compute
aggregates 1
compute
aggregates 2
compute
aggregates 3
Old State Management with DStreams
DStreams (i.e. updateStateByKey,mapWithState)represented
state data as RDDs
Leveraged RDD lineage for fault-tolerance and RDD
checkpointing to prevent unbounded
RDD checkpointing savesall state data to HDFS/S3
Inefficientwhen update ratesare low
No "incremental"checkpointing
33
New State Management: State Store
State Store API: any versionedkey value store that
- Allowsa set of key-value updatesto be transactionally committed
with a version number(i.e. incremental checkpointing)
- Allowsa specific version of the key-value data to be retrieved
34
Incremental
Execution 1
state store, v1
Incremental
Execution 2
Incremental
Execution 3
state store, v2
Spark cluster
HDFS-backed State Store
Implementation of State Store API in Spark 2.0:
In-memory hashmap backed by files in HDFS
35
Driver Executor
Executor
hash
map
hash
map
- Each executorhas hashmap(s)
containing versioned data
- Updatescommitted as delta
filesin HDFS/S3
- Delta filesperiodically
collapsed into snapshotsto
improve recovery deltafiles
in HDFS
k1 v1
k2 v2
k1 v1
k2 v2
Fault recovery of HDFS State Store
State Store version also stored in the WAL
36
On fault recovery,
- Recoverinputdata from
source using lastoffsets
- Recoverstate data using
last store version
Incremental
Execution
In
progress WAL
sources
store files
recovered
state
recovered
input
37
Fast, fault-tolerant, exactly-once
stateful stream processing
without having to reason about streaming
Plan
Spark 2.0
Basic infrastructureand API
- Eventtime, windows,aggregations
Files as sourcesand sinks
- Kafka as source, SQL as sink(?)
Plan
Spark 2.1+
More supportfor late data
Dynamic scaling
Sourceand sinks public API
- Extends DataSource API
More sourcesand sinks
ML integrations
Spark XXX
"Truestreaming" engine
- API already agnostic to
micro-batch
40
Simple and fast real-time analytics
• Develop Productively
• Execute Efficiently
• Update Automatically
Questions?
Learn More
Today, 1:50-2:30 AMA
Followme @tathadas

More Related Content

What's hot

Jump Start with Apache Spark 2.0 on Databricks
Jump Start with Apache Spark 2.0 on DatabricksJump Start with Apache Spark 2.0 on Databricks
Jump Start with Apache Spark 2.0 on DatabricksDatabricks
 
From Pipelines to Refineries: Scaling Big Data Applications
From Pipelines to Refineries: Scaling Big Data ApplicationsFrom Pipelines to Refineries: Scaling Big Data Applications
From Pipelines to Refineries: Scaling Big Data ApplicationsDatabricks
 
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
 
A look ahead at spark 2.0
A look ahead at spark 2.0 A look ahead at spark 2.0
A look ahead at spark 2.0 Databricks
 
Spark Summit EU 2015: Lessons from 300+ production users
Spark Summit EU 2015: Lessons from 300+ production usersSpark Summit EU 2015: Lessons from 300+ production users
Spark Summit EU 2015: Lessons from 300+ production usersDatabricks
 
Parallelize R Code Using Apache Spark
Parallelize R Code Using Apache Spark Parallelize R Code Using Apache Spark
Parallelize R Code Using Apache Spark Databricks
 
Spark stream - Kafka
Spark stream - Kafka Spark stream - Kafka
Spark stream - Kafka Dori Waldman
 
Robust and Scalable ETL over Cloud Storage with Apache Spark
Robust and Scalable ETL over Cloud Storage with Apache SparkRobust and Scalable ETL over Cloud Storage with Apache Spark
Robust and Scalable ETL over Cloud Storage with Apache SparkDatabricks
 
A look under the hood at Apache Spark's API and engine evolutions
A look under the hood at Apache Spark's API and engine evolutionsA look under the hood at Apache Spark's API and engine evolutions
A look under the hood at Apache Spark's API and engine evolutionsDatabricks
 
Exceptions are the Norm: Dealing with Bad Actors in ETL
Exceptions are the Norm: Dealing with Bad Actors in ETLExceptions are the Norm: Dealing with Bad Actors in ETL
Exceptions are the Norm: Dealing with Bad Actors in ETLDatabricks
 
Intro to Spark development
 Intro to Spark development  Intro to Spark development
Intro to Spark development Spark Summit
 
Stanford CS347 Guest Lecture: Apache Spark
Stanford CS347 Guest Lecture: Apache SparkStanford CS347 Guest Lecture: Apache Spark
Stanford CS347 Guest Lecture: Apache SparkReynold Xin
 
Introduction to Apache Spark Developer Training
Introduction to Apache Spark Developer TrainingIntroduction to Apache Spark Developer Training
Introduction to Apache Spark Developer TrainingCloudera, Inc.
 
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 Summit EU talk by Qifan Pu
Spark Summit EU talk by Qifan PuSpark Summit EU talk by Qifan Pu
Spark Summit EU talk by Qifan PuSpark Summit
 
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
 
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 Databricks
 
Productizing Structured Streaming Jobs
Productizing Structured Streaming JobsProductizing Structured Streaming Jobs
Productizing Structured Streaming JobsDatabricks
 
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
 
Temporal Operators For Spark Streaming And Its Application For Office365 Serv...
Temporal Operators For Spark Streaming And Its Application For Office365 Serv...Temporal Operators For Spark Streaming And Its Application For Office365 Serv...
Temporal Operators For Spark Streaming And Its Application For Office365 Serv...Jen Aman
 

What's hot (20)

Jump Start with Apache Spark 2.0 on Databricks
Jump Start with Apache Spark 2.0 on DatabricksJump Start with Apache Spark 2.0 on Databricks
Jump Start with Apache Spark 2.0 on Databricks
 
From Pipelines to Refineries: Scaling Big Data Applications
From Pipelines to Refineries: Scaling Big Data ApplicationsFrom Pipelines to Refineries: Scaling Big Data Applications
From Pipelines to Refineries: Scaling Big Data Applications
 
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
 
A look ahead at spark 2.0
A look ahead at spark 2.0 A look ahead at spark 2.0
A look ahead at spark 2.0
 
Spark Summit EU 2015: Lessons from 300+ production users
Spark Summit EU 2015: Lessons from 300+ production usersSpark Summit EU 2015: Lessons from 300+ production users
Spark Summit EU 2015: Lessons from 300+ production users
 
Parallelize R Code Using Apache Spark
Parallelize R Code Using Apache Spark Parallelize R Code Using Apache Spark
Parallelize R Code Using Apache Spark
 
Spark stream - Kafka
Spark stream - Kafka Spark stream - Kafka
Spark stream - Kafka
 
Robust and Scalable ETL over Cloud Storage with Apache Spark
Robust and Scalable ETL over Cloud Storage with Apache SparkRobust and Scalable ETL over Cloud Storage with Apache Spark
Robust and Scalable ETL over Cloud Storage with Apache Spark
 
A look under the hood at Apache Spark's API and engine evolutions
A look under the hood at Apache Spark's API and engine evolutionsA look under the hood at Apache Spark's API and engine evolutions
A look under the hood at Apache Spark's API and engine evolutions
 
Exceptions are the Norm: Dealing with Bad Actors in ETL
Exceptions are the Norm: Dealing with Bad Actors in ETLExceptions are the Norm: Dealing with Bad Actors in ETL
Exceptions are the Norm: Dealing with Bad Actors in ETL
 
Intro to Spark development
 Intro to Spark development  Intro to Spark development
Intro to Spark development
 
Stanford CS347 Guest Lecture: Apache Spark
Stanford CS347 Guest Lecture: Apache SparkStanford CS347 Guest Lecture: Apache Spark
Stanford CS347 Guest Lecture: Apache Spark
 
Introduction to Apache Spark Developer Training
Introduction to Apache Spark Developer TrainingIntroduction to Apache Spark Developer Training
Introduction to Apache Spark Developer Training
 
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 Summit EU talk by Qifan Pu
Spark Summit EU talk by Qifan PuSpark Summit EU talk by Qifan Pu
Spark Summit EU talk by Qifan Pu
 
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 ...
 
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
 
Productizing Structured Streaming Jobs
Productizing Structured Streaming JobsProductizing Structured Streaming Jobs
Productizing Structured Streaming Jobs
 
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 - ...
 
Temporal Operators For Spark Streaming And Its Application For Office365 Serv...
Temporal Operators For Spark Streaming And Its Application For Office365 Serv...Temporal Operators For Spark Streaming And Its Application For Office365 Serv...
Temporal Operators For Spark Streaming And Its Application For Office365 Serv...
 

Viewers also liked

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
 
Spark streaming with kafka
Spark streaming with kafkaSpark streaming with kafka
Spark streaming with kafkaDori Waldman
 
Structuring Spark: DataFrames, Datasets, and Streaming by Michael Armbrust
Structuring Spark: DataFrames, Datasets, and Streaming by Michael ArmbrustStructuring Spark: DataFrames, Datasets, and Streaming by Michael Armbrust
Structuring Spark: DataFrames, Datasets, and Streaming by Michael ArmbrustSpark Summit
 
Real-Time Spark: From Interactive Queries to Streaming
Real-Time Spark: From Interactive Queries to StreamingReal-Time Spark: From Interactive Queries to Streaming
Real-Time Spark: From Interactive Queries to StreamingDatabricks
 
Think Like Spark
Think Like SparkThink Like Spark
Think Like SparkAlpine Data
 
Building a modern Application with DataFrames
Building a modern Application with DataFramesBuilding a modern Application with DataFrames
Building a modern Application with DataFramesDatabricks
 
700 Queries Per Second with Updates: Spark As A Real-Time Web Service
700 Queries Per Second with Updates: Spark As A Real-Time Web Service700 Queries Per Second with Updates: Spark As A Real-Time Web Service
700 Queries Per Second with Updates: Spark As A Real-Time Web ServiceSpark Summit
 
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
 
Deep Dive with Spark Streaming - Tathagata Das - Spark Meetup 2013-06-17
Deep Dive with Spark Streaming - Tathagata  Das - Spark Meetup 2013-06-17Deep Dive with Spark Streaming - Tathagata  Das - Spark Meetup 2013-06-17
Deep Dive with Spark Streaming - Tathagata Das - Spark Meetup 2013-06-17spark-project
 
Keeping Spark on Track: Productionizing Spark for ETL
Keeping Spark on Track: Productionizing Spark for ETLKeeping Spark on Track: Productionizing Spark for ETL
Keeping Spark on Track: Productionizing Spark for ETLDatabricks
 
Practical Large Scale Experiences with Spark 2.0 Machine Learning: Spark Summ...
Practical Large Scale Experiences with Spark 2.0 Machine Learning: Spark Summ...Practical Large Scale Experiences with Spark 2.0 Machine Learning: Spark Summ...
Practical Large Scale Experiences with Spark 2.0 Machine Learning: Spark Summ...Spark Summit
 
2016 Spark Summit East Keynote: Matei Zaharia
2016 Spark Summit East Keynote: Matei Zaharia2016 Spark Summit East Keynote: Matei Zaharia
2016 Spark Summit East Keynote: Matei ZahariaDatabricks
 
Realtime Analytical Query Processing and Predictive Model Building on High Di...
Realtime Analytical Query Processing and Predictive Model Building on High Di...Realtime Analytical Query Processing and Predictive Model Building on High Di...
Realtime Analytical Query Processing and Predictive Model Building on High Di...Spark Summit
 
Parallelizing Existing R Packages with SparkR
Parallelizing Existing R Packages with SparkRParallelizing Existing R Packages with SparkR
Parallelizing Existing R Packages with SparkRDatabricks
 
Insights Without Tradeoffs Using Structured Streaming keynote by Michael Armb...
Insights Without Tradeoffs Using Structured Streaming keynote by Michael Armb...Insights Without Tradeoffs Using Structured Streaming keynote by Michael Armb...
Insights Without Tradeoffs Using Structured Streaming keynote by Michael Armb...Spark Summit
 
What No One Tells You About Writing a Streaming App: Spark Summit East talk b...
What No One Tells You About Writing a Streaming App: Spark Summit East talk b...What No One Tells You About Writing a Streaming App: Spark Summit East talk b...
What No One Tells You About Writing a Streaming App: Spark Summit East talk b...Spark Summit
 
How to deploy spark instance using ansible 2.0 in fiware lab v2
How to deploy spark instance using ansible 2.0 in fiware lab v2How to deploy spark instance using ansible 2.0 in fiware lab v2
How to deploy spark instance using ansible 2.0 in fiware lab v2Fernando Lopez Aguilar
 

Viewers also liked (20)

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
 
Spark streaming with kafka
Spark streaming with kafkaSpark streaming with kafka
Spark streaming with kafka
 
Structuring Spark: DataFrames, Datasets, and Streaming by Michael Armbrust
Structuring Spark: DataFrames, Datasets, and Streaming by Michael ArmbrustStructuring Spark: DataFrames, Datasets, and Streaming by Michael Armbrust
Structuring Spark: DataFrames, Datasets, and Streaming by Michael Armbrust
 
Real-Time Spark: From Interactive Queries to Streaming
Real-Time Spark: From Interactive Queries to StreamingReal-Time Spark: From Interactive Queries to Streaming
Real-Time Spark: From Interactive Queries to Streaming
 
Think Like Spark
Think Like SparkThink Like Spark
Think Like Spark
 
Building a modern Application with DataFrames
Building a modern Application with DataFramesBuilding a modern Application with DataFrames
Building a modern Application with DataFrames
 
700 Queries Per Second with Updates: Spark As A Real-Time Web Service
700 Queries Per Second with Updates: Spark As A Real-Time Web Service700 Queries Per Second with Updates: Spark As A Real-Time Web Service
700 Queries Per Second with Updates: Spark As A Real-Time Web Service
 
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
 
Deep Dive with Spark Streaming - Tathagata Das - Spark Meetup 2013-06-17
Deep Dive with Spark Streaming - Tathagata  Das - Spark Meetup 2013-06-17Deep Dive with Spark Streaming - Tathagata  Das - Spark Meetup 2013-06-17
Deep Dive with Spark Streaming - Tathagata Das - Spark Meetup 2013-06-17
 
Keeping Spark on Track: Productionizing Spark for ETL
Keeping Spark on Track: Productionizing Spark for ETLKeeping Spark on Track: Productionizing Spark for ETL
Keeping Spark on Track: Productionizing Spark for ETL
 
Practical Large Scale Experiences with Spark 2.0 Machine Learning: Spark Summ...
Practical Large Scale Experiences with Spark 2.0 Machine Learning: Spark Summ...Practical Large Scale Experiences with Spark 2.0 Machine Learning: Spark Summ...
Practical Large Scale Experiences with Spark 2.0 Machine Learning: Spark Summ...
 
2016 Spark Summit East Keynote: Matei Zaharia
2016 Spark Summit East Keynote: Matei Zaharia2016 Spark Summit East Keynote: Matei Zaharia
2016 Spark Summit East Keynote: Matei Zaharia
 
Realtime Analytical Query Processing and Predictive Model Building on High Di...
Realtime Analytical Query Processing and Predictive Model Building on High Di...Realtime Analytical Query Processing and Predictive Model Building on High Di...
Realtime Analytical Query Processing and Predictive Model Building on High Di...
 
Parallelizing Existing R Packages with SparkR
Parallelizing Existing R Packages with SparkRParallelizing Existing R Packages with SparkR
Parallelizing Existing R Packages with SparkR
 
Insights Without Tradeoffs Using Structured Streaming keynote by Michael Armb...
Insights Without Tradeoffs Using Structured Streaming keynote by Michael Armb...Insights Without Tradeoffs Using Structured Streaming keynote by Michael Armb...
Insights Without Tradeoffs Using Structured Streaming keynote by Michael Armb...
 
What No One Tells You About Writing a Streaming App: Spark Summit East talk b...
What No One Tells You About Writing a Streaming App: Spark Summit East talk b...What No One Tells You About Writing a Streaming App: Spark Summit East talk b...
What No One Tells You About Writing a Streaming App: Spark Summit East talk b...
 
Intro to Apache Spark
Intro to Apache SparkIntro to Apache Spark
Intro to Apache Spark
 
How to deploy spark instance using ansible 2.0 in fiware lab v2
How to deploy spark instance using ansible 2.0 in fiware lab v2How to deploy spark instance using ansible 2.0 in fiware lab v2
How to deploy spark instance using ansible 2.0 in fiware lab v2
 
MLeap: Release Spark ML Pipelines
MLeap: Release Spark ML PipelinesMLeap: Release Spark ML Pipelines
MLeap: Release Spark ML Pipelines
 
Performance
PerformancePerformance
Performance
 

Similar to Taking 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.0Anyscale
 
Guest Lecture on Spark Streaming in Stanford CME 323: Distributed Algorithms ...
Guest Lecture on Spark Streaming in Stanford CME 323: Distributed Algorithms ...Guest Lecture on Spark Streaming in Stanford CME 323: Distributed Algorithms ...
Guest Lecture on Spark Streaming in Stanford CME 323: Distributed Algorithms ...Tathagata Das
 
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
 
strata_spark_streaming.ppt
strata_spark_streaming.pptstrata_spark_streaming.ppt
strata_spark_streaming.pptrveiga100
 
Flexible and Real-Time Stream Processing with Apache Flink
Flexible and Real-Time Stream Processing with Apache FlinkFlexible and Real-Time Stream Processing with Apache Flink
Flexible and Real-Time Stream Processing with Apache FlinkDataWorks Summit
 
Flink Streaming Hadoop Summit San Jose
Flink Streaming Hadoop Summit San JoseFlink Streaming Hadoop Summit San Jose
Flink Streaming Hadoop Summit San JoseKostas Tzoumas
 
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
 
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)Stephan Ewen
 
strata_spark_streaming.ppt
strata_spark_streaming.pptstrata_spark_streaming.ppt
strata_spark_streaming.pptAbhijitManna19
 
strata_spark_streaming.ppt
strata_spark_streaming.pptstrata_spark_streaming.ppt
strata_spark_streaming.pptsnowflakebatch
 
strata spark streaming strata spark streamingsrata spark streaming
strata spark streaming strata spark streamingsrata spark streamingstrata spark streaming strata spark streamingsrata spark streaming
strata spark streaming strata spark streamingsrata spark streamingShidrokhGoudarzi1
 
Moving Towards a Streaming Architecture
Moving Towards a Streaming ArchitectureMoving Towards a Streaming Architecture
Moving Towards a Streaming ArchitectureGabriele Modena
 
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
 
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 - ...Anyscale
 
Intro to Apache Apex - Next Gen Platform for Ingest and Transform
Intro to Apache Apex - Next Gen Platform for Ingest and TransformIntro to Apache Apex - Next Gen Platform for Ingest and Transform
Intro to Apache Apex - Next Gen Platform for Ingest and TransformApache Apex
 
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
 

Similar to Taking Streaming to the Next Level with Datasets and DataFrames (20)

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
 
Guest Lecture on Spark Streaming in Stanford CME 323: Distributed Algorithms ...
Guest Lecture on Spark Streaming in Stanford CME 323: Distributed Algorithms ...Guest Lecture on Spark Streaming in Stanford CME 323: Distributed Algorithms ...
Guest Lecture on Spark Streaming in Stanford CME 323: Distributed Algorithms ...
 
Stream Processing Overview
Stream Processing OverviewStream Processing Overview
Stream Processing Overview
 
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...
 
Spark streaming
Spark streamingSpark streaming
Spark streaming
 
strata_spark_streaming.ppt
strata_spark_streaming.pptstrata_spark_streaming.ppt
strata_spark_streaming.ppt
 
Flexible and Real-Time Stream Processing with Apache Flink
Flexible and Real-Time Stream Processing with Apache FlinkFlexible and Real-Time Stream Processing with Apache Flink
Flexible and Real-Time Stream Processing with Apache Flink
 
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
 
Flink Streaming Hadoop Summit San Jose
Flink Streaming Hadoop Summit San JoseFlink Streaming Hadoop Summit San Jose
Flink Streaming Hadoop Summit San Jose
 
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?
 
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)
 
strata_spark_streaming.ppt
strata_spark_streaming.pptstrata_spark_streaming.ppt
strata_spark_streaming.ppt
 
strata_spark_streaming.ppt
strata_spark_streaming.pptstrata_spark_streaming.ppt
strata_spark_streaming.ppt
 
strata spark streaming strata spark streamingsrata spark streaming
strata spark streaming strata spark streamingsrata spark streamingstrata spark streaming strata spark streamingsrata spark streaming
strata spark streaming strata spark streamingsrata spark streaming
 
Moving Towards a Streaming Architecture
Moving Towards a Streaming ArchitectureMoving Towards a Streaming Architecture
Moving Towards a Streaming Architecture
 
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...
 
Interpreting the Data:Parallel Analysis with Sawzall
Interpreting the Data:Parallel Analysis with SawzallInterpreting the Data:Parallel Analysis with Sawzall
Interpreting the Data:Parallel Analysis with Sawzall
 
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 - ...
 
Intro to Apache Apex - Next Gen Platform for Ingest and Transform
Intro to Apache Apex - Next Gen Platform for Ingest and TransformIntro to Apache Apex - Next Gen Platform for Ingest and Transform
Intro to Apache Apex - Next Gen Platform for Ingest and Transform
 
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...
 

More from 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
 

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

Recently uploaded

How to Track Employee Performance A Comprehensive Guide.pdf
How to Track Employee Performance A Comprehensive Guide.pdfHow to Track Employee Performance A Comprehensive Guide.pdf
How to Track Employee Performance A Comprehensive Guide.pdfLivetecs LLC
 
What is Advanced Excel and what are some best practices for designing and cre...
What is Advanced Excel and what are some best practices for designing and cre...What is Advanced Excel and what are some best practices for designing and cre...
What is Advanced Excel and what are some best practices for designing and cre...Technogeeks
 
Software Project Health Check: Best Practices and Techniques for Your Product...
Software Project Health Check: Best Practices and Techniques for Your Product...Software Project Health Check: Best Practices and Techniques for Your Product...
Software Project Health Check: Best Practices and Techniques for Your Product...Velvetech LLC
 
KnowAPIs-UnknownPerf-jaxMainz-2024 (1).pptx
KnowAPIs-UnknownPerf-jaxMainz-2024 (1).pptxKnowAPIs-UnknownPerf-jaxMainz-2024 (1).pptx
KnowAPIs-UnknownPerf-jaxMainz-2024 (1).pptxTier1 app
 
Odoo 14 - eLearning Module In Odoo 14 Enterprise
Odoo 14 - eLearning Module In Odoo 14 EnterpriseOdoo 14 - eLearning Module In Odoo 14 Enterprise
Odoo 14 - eLearning Module In Odoo 14 Enterprisepreethippts
 
BATTLEFIELD ORM: TIPS, TACTICS AND STRATEGIES FOR CONQUERING YOUR DATABASE
BATTLEFIELD ORM: TIPS, TACTICS AND STRATEGIES FOR CONQUERING YOUR DATABASEBATTLEFIELD ORM: TIPS, TACTICS AND STRATEGIES FOR CONQUERING YOUR DATABASE
BATTLEFIELD ORM: TIPS, TACTICS AND STRATEGIES FOR CONQUERING YOUR DATABASEOrtus Solutions, Corp
 
英国UN学位证,北安普顿大学毕业证书1:1制作
英国UN学位证,北安普顿大学毕业证书1:1制作英国UN学位证,北安普顿大学毕业证书1:1制作
英国UN学位证,北安普顿大学毕业证书1:1制作qr0udbr0
 
Maximizing Efficiency and Profitability with OnePlan’s Professional Service A...
Maximizing Efficiency and Profitability with OnePlan’s Professional Service A...Maximizing Efficiency and Profitability with OnePlan’s Professional Service A...
Maximizing Efficiency and Profitability with OnePlan’s Professional Service A...OnePlan Solutions
 
Call Us🔝>༒+91-9711147426⇛Call In girls karol bagh (Delhi)
Call Us🔝>༒+91-9711147426⇛Call In girls karol bagh (Delhi)Call Us🔝>༒+91-9711147426⇛Call In girls karol bagh (Delhi)
Call Us🔝>༒+91-9711147426⇛Call In girls karol bagh (Delhi)jennyeacort
 
What is Fashion PLM and Why Do You Need It
What is Fashion PLM and Why Do You Need ItWhat is Fashion PLM and Why Do You Need It
What is Fashion PLM and Why Do You Need ItWave PLM
 
What are the key points to focus on before starting to learn ETL Development....
What are the key points to focus on before starting to learn ETL Development....What are the key points to focus on before starting to learn ETL Development....
What are the key points to focus on before starting to learn ETL Development....kzayra69
 
Folding Cheat Sheet #4 - fourth in a series
Folding Cheat Sheet #4 - fourth in a seriesFolding Cheat Sheet #4 - fourth in a series
Folding Cheat Sheet #4 - fourth in a seriesPhilip Schwarz
 
办理学位证(UQ文凭证书)昆士兰大学毕业证成绩单原版一模一样
办理学位证(UQ文凭证书)昆士兰大学毕业证成绩单原版一模一样办理学位证(UQ文凭证书)昆士兰大学毕业证成绩单原版一模一样
办理学位证(UQ文凭证书)昆士兰大学毕业证成绩单原版一模一样umasea
 
Catch the Wave: SAP Event-Driven and Data Streaming for the Intelligence Ente...
Catch the Wave: SAP Event-Driven and Data Streaming for the Intelligence Ente...Catch the Wave: SAP Event-Driven and Data Streaming for the Intelligence Ente...
Catch the Wave: SAP Event-Driven and Data Streaming for the Intelligence Ente...confluent
 
Unveiling Design Patterns: A Visual Guide with UML Diagrams
Unveiling Design Patterns: A Visual Guide with UML DiagramsUnveiling Design Patterns: A Visual Guide with UML Diagrams
Unveiling Design Patterns: A Visual Guide with UML DiagramsAhmed Mohamed
 
Taming Distributed Systems: Key Insights from Wix's Large-Scale Experience - ...
Taming Distributed Systems: Key Insights from Wix's Large-Scale Experience - ...Taming Distributed Systems: Key Insights from Wix's Large-Scale Experience - ...
Taming Distributed Systems: Key Insights from Wix's Large-Scale Experience - ...Natan Silnitsky
 
Cloud Data Center Network Construction - IEEE
Cloud Data Center Network Construction - IEEECloud Data Center Network Construction - IEEE
Cloud Data Center Network Construction - IEEEVICTOR MAESTRE RAMIREZ
 
A healthy diet for your Java application Devoxx France.pdf
A healthy diet for your Java application Devoxx France.pdfA healthy diet for your Java application Devoxx France.pdf
A healthy diet for your Java application Devoxx France.pdfMarharyta Nedzelska
 
Global Identity Enrolment and Verification Pro Solution - Cizo Technology Ser...
Global Identity Enrolment and Verification Pro Solution - Cizo Technology Ser...Global Identity Enrolment and Verification Pro Solution - Cizo Technology Ser...
Global Identity Enrolment and Verification Pro Solution - Cizo Technology Ser...Cizo Technology Services
 

Recently uploaded (20)

How to Track Employee Performance A Comprehensive Guide.pdf
How to Track Employee Performance A Comprehensive Guide.pdfHow to Track Employee Performance A Comprehensive Guide.pdf
How to Track Employee Performance A Comprehensive Guide.pdf
 
What is Advanced Excel and what are some best practices for designing and cre...
What is Advanced Excel and what are some best practices for designing and cre...What is Advanced Excel and what are some best practices for designing and cre...
What is Advanced Excel and what are some best practices for designing and cre...
 
Software Project Health Check: Best Practices and Techniques for Your Product...
Software Project Health Check: Best Practices and Techniques for Your Product...Software Project Health Check: Best Practices and Techniques for Your Product...
Software Project Health Check: Best Practices and Techniques for Your Product...
 
KnowAPIs-UnknownPerf-jaxMainz-2024 (1).pptx
KnowAPIs-UnknownPerf-jaxMainz-2024 (1).pptxKnowAPIs-UnknownPerf-jaxMainz-2024 (1).pptx
KnowAPIs-UnknownPerf-jaxMainz-2024 (1).pptx
 
Odoo 14 - eLearning Module In Odoo 14 Enterprise
Odoo 14 - eLearning Module In Odoo 14 EnterpriseOdoo 14 - eLearning Module In Odoo 14 Enterprise
Odoo 14 - eLearning Module In Odoo 14 Enterprise
 
2.pdf Ejercicios de programación competitiva
2.pdf Ejercicios de programación competitiva2.pdf Ejercicios de programación competitiva
2.pdf Ejercicios de programación competitiva
 
BATTLEFIELD ORM: TIPS, TACTICS AND STRATEGIES FOR CONQUERING YOUR DATABASE
BATTLEFIELD ORM: TIPS, TACTICS AND STRATEGIES FOR CONQUERING YOUR DATABASEBATTLEFIELD ORM: TIPS, TACTICS AND STRATEGIES FOR CONQUERING YOUR DATABASE
BATTLEFIELD ORM: TIPS, TACTICS AND STRATEGIES FOR CONQUERING YOUR DATABASE
 
英国UN学位证,北安普顿大学毕业证书1:1制作
英国UN学位证,北安普顿大学毕业证书1:1制作英国UN学位证,北安普顿大学毕业证书1:1制作
英国UN学位证,北安普顿大学毕业证书1:1制作
 
Maximizing Efficiency and Profitability with OnePlan’s Professional Service A...
Maximizing Efficiency and Profitability with OnePlan’s Professional Service A...Maximizing Efficiency and Profitability with OnePlan’s Professional Service A...
Maximizing Efficiency and Profitability with OnePlan’s Professional Service A...
 
Call Us🔝>༒+91-9711147426⇛Call In girls karol bagh (Delhi)
Call Us🔝>༒+91-9711147426⇛Call In girls karol bagh (Delhi)Call Us🔝>༒+91-9711147426⇛Call In girls karol bagh (Delhi)
Call Us🔝>༒+91-9711147426⇛Call In girls karol bagh (Delhi)
 
What is Fashion PLM and Why Do You Need It
What is Fashion PLM and Why Do You Need ItWhat is Fashion PLM and Why Do You Need It
What is Fashion PLM and Why Do You Need It
 
What are the key points to focus on before starting to learn ETL Development....
What are the key points to focus on before starting to learn ETL Development....What are the key points to focus on before starting to learn ETL Development....
What are the key points to focus on before starting to learn ETL Development....
 
Folding Cheat Sheet #4 - fourth in a series
Folding Cheat Sheet #4 - fourth in a seriesFolding Cheat Sheet #4 - fourth in a series
Folding Cheat Sheet #4 - fourth in a series
 
办理学位证(UQ文凭证书)昆士兰大学毕业证成绩单原版一模一样
办理学位证(UQ文凭证书)昆士兰大学毕业证成绩单原版一模一样办理学位证(UQ文凭证书)昆士兰大学毕业证成绩单原版一模一样
办理学位证(UQ文凭证书)昆士兰大学毕业证成绩单原版一模一样
 
Catch the Wave: SAP Event-Driven and Data Streaming for the Intelligence Ente...
Catch the Wave: SAP Event-Driven and Data Streaming for the Intelligence Ente...Catch the Wave: SAP Event-Driven and Data Streaming for the Intelligence Ente...
Catch the Wave: SAP Event-Driven and Data Streaming for the Intelligence Ente...
 
Unveiling Design Patterns: A Visual Guide with UML Diagrams
Unveiling Design Patterns: A Visual Guide with UML DiagramsUnveiling Design Patterns: A Visual Guide with UML Diagrams
Unveiling Design Patterns: A Visual Guide with UML Diagrams
 
Taming Distributed Systems: Key Insights from Wix's Large-Scale Experience - ...
Taming Distributed Systems: Key Insights from Wix's Large-Scale Experience - ...Taming Distributed Systems: Key Insights from Wix's Large-Scale Experience - ...
Taming Distributed Systems: Key Insights from Wix's Large-Scale Experience - ...
 
Cloud Data Center Network Construction - IEEE
Cloud Data Center Network Construction - IEEECloud Data Center Network Construction - IEEE
Cloud Data Center Network Construction - IEEE
 
A healthy diet for your Java application Devoxx France.pdf
A healthy diet for your Java application Devoxx France.pdfA healthy diet for your Java application Devoxx France.pdf
A healthy diet for your Java application Devoxx France.pdf
 
Global Identity Enrolment and Verification Pro Solution - Cizo Technology Ser...
Global Identity Enrolment and Verification Pro Solution - Cizo Technology Ser...Global Identity Enrolment and Verification Pro Solution - Cizo Technology Ser...
Global Identity Enrolment and Verification Pro Solution - Cizo Technology Ser...
 

Taking Streaming to the Next Level with Datasets and DataFrames

  • 1. Taking Spark Streaming to the next level with Datasets and DataFrames Tathagata “TD” Das @tathadas Strata San Jose 2016
  • 2. Streaming in Spark Spark Streaming changedhow peoplewrite streaming apps 2 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 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 Devices ETL into long term storage - Preventdata loss - PreventduplicatesStatus monitoring - Handlelate data - Process based on eventtime Interactively debug issues - consistency event stream Anomaly detection - Learn modelsoffline - Use online+continuous learning
  • 6. 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 - Hard to interoperate with DataFrames and Datasets 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
  • 8. The simplest way to perform streaming analytics is not having to reason about streaming at all
  • 9. 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
  • 10. 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
  • 11. 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
  • 13. Batch ETL with DataFrame input = ctxt.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
  • 14. Streaming ETL with DataFrame input = ctxt.read .format("json") .stream("source-path") result = input .select("device", "signal") .where("signal > 15") result.write .format("parquet") .outputMode("append") .startStream("dest-path") Read from Json file stream Select some devices Write to parquet file stream
  • 15. Streaming ETL with DataFrame input = ctxt.read .format("json") .stream("source-path") result = input .select("device", "signal") .where("signal > 15") result.write .format("parquet") .outputMode("append") .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
  • 16. Streaming ETL with DataFrame input = ctxt.read .format("json") .stream("source-path") result = input .select("device", "signal") .where("signal > 15") result.write .format("parquet") .outputMode("append") .startStream("dest-path") 1 2 3 Result [append-only table] Input Output [append mode] new rows in result of 2 new rows in result of 3
  • 17. Continuous Aggregations Continuously compute average signal of each type of device 17 input.groupBy("device-type") .avg("signal") input.groupBy( window("event-time", "10min"), "device type") .avg("age") Continuously compute average signal of each type of device in last10 minutesof eventtime - Windowing is just a type of aggregation - Simple API for event time based windowing
  • 18. Query Management query = result.write .format("parquet") .outputMode("append") .startStream("dest-path") query.stop() query.awaitTermination() query.exception() query.sourceStatuses() query.sinkStatus() 18 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
  • 19. Logically: DataFrame 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 Execution
  • 20. Structured Streaming High-level streaming API built on SparkSQL engine Runs the same computation as batch queriesin 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
  • 21. Advantages over DStreams 1. Processingwith event-time,dealingwith late data 2. Exactly same API for batch,streaming,and interactive 3. End-to-endexactly-once guaranteesfromthe system 4. Performance through SQL optimizations - Logical plan optimizations, Tungsten, Codegen, etc. - Faster state management for stateful stream processing 21
  • 23. Batch Execution on Spark SQL 23 DataFrame/ Dataset Logical Plan Execution PlanPlanner Logical plan optimization Execution plan generation RDD job Tungsten Code generation Abstract representation of query
  • 24. Continuous Incremental Execution Planner extendedto be aware of the streaminglogical plans Planner generatesa continuousseriesof incrementalexecutionplans, each processingthe next chunk ofstreaming data 24 DataFrame/ Dataset Logical Plan Incremental Execution Plan 1 Incremental Execution Plan 2 Incremental Execution Plan 3 Planner Incremental Execution Plan 4
  • 25. Streaming Source A streaming data source where - Records can be uniquely identified by a offset - Arbitrary segmentof the stream data can be read based on an offset range - Files,Kafka, Kinesissupported More restricted than DStream Receivers Can ensure end-to-end exactly-once guarantees 25 Logical Plan Sources Sink Operations
  • 26. Streaming Sink Allows output to pushed to externalstorage Each batch of output should be written to storage atomically and idempotently Both neededto ensureend-to-end exactly- once guarantees,AND data consistency 26 Logical Plan Sources Sink Operations
  • 27. Incremental Execution Every trigger interval - Plannerask source for next chunkof inputdata - Generatesexecution plan with the input - Generate output data - Handsto sink for pushing itout 27 Logical Plan Sources Sink Ops Incremental Execution Plan Input get new data as input Output push output generate outputPlanner
  • 28. Offset Tracking with WAL Plannersaves the next offset range in a write- ahead log (WAL) on HDFS/S3 before incremental execution starts 28 Logical Plan Sources Incremental Execution Plan Planner processed offsets Offset WAL in-progress offsets save offset range to WAL before processing starts Input
  • 29. Recovery from WAL 29 Logical Plan Sources Incremental Execution PlanRestarted Planner processed offsets in-progress offsets Offset WAL recover last offset range and restart computation After failure, restarted plannerrecoverslast offset range from WAL and restarts failed execution Output exactly same as it would been without failure Input
  • 30. 30 Streaming source + Streaming sink + Offset tracking = End-to-end exactly-once guarantees
  • 31. Stateful Stream Processing Streaming aggregationsrequire maintaining intermediate "state" data acrossbatches 31 compute aggregates 3 Dog Cat 2 Dog 1 Cat 3 Dog 1 Cow 1 state data compute aggregates 2 Cat, Cow compute aggregates 1 Cat, Dog, Cat aggregates
  • 32. State Management State needsto be fault-tolerant Cat, Dog, Cat Cat, Cow Dog Cat 2 Dog 1 Cat 3 Dog 1 Cow 1aggregates both, input + previous state needed to recover from failure Cat 3 Dog 1 Cow 1 Dog compute aggregates 1 compute aggregates 2 compute aggregates 3
  • 33. Old State Management with DStreams DStreams (i.e. updateStateByKey,mapWithState)represented state data as RDDs Leveraged RDD lineage for fault-tolerance and RDD checkpointing to prevent unbounded RDD checkpointing savesall state data to HDFS/S3 Inefficientwhen update ratesare low No "incremental"checkpointing 33
  • 34. New State Management: State Store State Store API: any versionedkey value store that - Allowsa set of key-value updatesto be transactionally committed with a version number(i.e. incremental checkpointing) - Allowsa specific version of the key-value data to be retrieved 34 Incremental Execution 1 state store, v1 Incremental Execution 2 Incremental Execution 3 state store, v2
  • 35. Spark cluster HDFS-backed State Store Implementation of State Store API in Spark 2.0: In-memory hashmap backed by files in HDFS 35 Driver Executor Executor hash map hash map - Each executorhas hashmap(s) containing versioned data - Updatescommitted as delta filesin HDFS/S3 - Delta filesperiodically collapsed into snapshotsto improve recovery deltafiles in HDFS k1 v1 k2 v2 k1 v1 k2 v2
  • 36. Fault recovery of HDFS State Store State Store version also stored in the WAL 36 On fault recovery, - Recoverinputdata from source using lastoffsets - Recoverstate data using last store version Incremental Execution In progress WAL sources store files recovered state recovered input
  • 37. 37 Fast, fault-tolerant, exactly-once stateful stream processing without having to reason about streaming
  • 38. Plan Spark 2.0 Basic infrastructureand API - Eventtime, windows,aggregations Files as sourcesand sinks - Kafka as source, SQL as sink(?)
  • 39. Plan Spark 2.1+ More supportfor late data Dynamic scaling Sourceand sinks public API - Extends DataSource API More sourcesand sinks ML integrations Spark XXX "Truestreaming" engine - API already agnostic to micro-batch
  • 40. 40 Simple and fast real-time analytics • Develop Productively • Execute Efficiently • Update Automatically Questions? Learn More Today, 1:50-2:30 AMA Followme @tathadas