SlideShare a Scribd company logo
1 of 61
Download to read offline
Easy, Scalable, Fault-tolerant
Stream Processing with
Structured Streaming
Michael Armbrust - @michaelarmbrust
Tathagata Das - @tathadas
Spark Summit 2017
6th June, San Francisco
TEAM
About Databricks
Started Spark project (now Apache Spark) at UC Berkeley in 2009
PRODUCT
Unified Analytics Platform
MISSION
Making Big Data Simple
building robust
stream processing
apps is hard
Complexities in stream processing
COMPLEX DATA
Diverse data formats
(json, avro, binary, …)
Data can be dirty,
late, out-of-order
COMPLEX SYSTEMS
Diverse storage systems
(Kafka, S3, Kinesis, RDBMS, …)
System failures
COMPLEX WORKLOADS
Combining streaming with
interactive queries
Machine learning
Structured Streaming
stream processing on Spark SQL engine
fast, scalable, fault-tolerant
rich, unified, high level APIs
deal with complex data and complex workloads
rich ecosystem of data sources
integrate with many storage systems
you
should not have to
reason about streaming
you
should write simple queries
&
Spark
should continuously update the answer
Streaming word count
Anatomy of a Streaming Query
Anatomy of a Streaming Query
spark.readStream
.format("kafka")
.option("subscribe", "input")
.load()
.groupBy($"value".cast("string"))
.count()
.writeStream
.format("kafka")
.option("topic", "output")
.trigger("1 minute")
.outputMode(OutputMode.Complete())
.option("checkpointLocation", "…")
.start()
Source
• Specify one or more locations
to read data from
• Built in support for
Files/Kafka/Socket,
pluggable.
• Can include multiple sources
of different types using
union()
Anatomy of a Streaming Query
spark.readStream
.format("kafka")
.option("subscribe", "input")
.load()
.groupBy('value.cast("string") as 'key)
.agg(count("*") as 'value)
.writeStream
.format("kafka")
.option("topic", "output")
.trigger("1 minute")
.outputMode(OutputMode.Complete())
.option("checkpointLocation", "…")
.start()
Transformation
• Using DataFrames,
Datasets and/or SQL.
• Catalyst figures out how to
execute the transformation
incrementally.
• Internal processing always
exactly-once.
DataFrames,
Datasets, SQL
input = spark.readStream
.format("kafka")
.option("subscribe", "topic")
.load()
result = input
.select("device", "signal")
.where("signal > 15")
result.writeStream
.format("parquet")
.start("dest-path")
Logical
Plan
Read from
Kafka
Project
device, signal
Filter
signal > 15
Write to
Kafka
Spark automatically streamifies!
Spark SQL converts batch-like query to a series of incremental
execution plans operating on new batches of data
Series of Incremental
Execution Plans
Kafka
Source
Optimized
Operator
codegen, off-
heap, etc.
Kafka
Sink
Optimized
Physical Plan
process
newdata
t = 1 t = 2 t = 3
process
newdata
process
newdata
Anatomy of a Streaming Query
spark.readStream
.format("kafka")
.option("subscribe", "input")
.load()
.groupBy('value.cast("string") as 'key)
.agg(count("*") as 'value)
.writeStream
.format("kafka")
.option("topic", "output")
.trigger("1 minute")
.outputMode(OutputMode.Complete())
.option("checkpointLocation", "…")
.start()
Sink
• Accepts the output of each
batch.
• When supported sinks are
transactional and exactly
once (Files).
• Use foreach to execute
arbitrary code.
Anatomy of a Streaming Query
spark.readStream
.format("kafka")
.option("subscribe", "input")
.load()
.groupBy('value.cast("string") as 'key)
.agg(count("*") as 'value)
.writeStream
.format("kafka")
.option("topic", "output")
.trigger("1 minute")
.outputMode("update")
.option("checkpointLocation", "…")
.start()
Output mode – What's output
• Complete – Output the whole answer
every time
• Update – Output changed rows
• Append – Output new rows only
Trigger – When to output
• Specified as a time, eventually
supports data size
• No trigger means as fast as possible
Anatomy of a Streaming Query
spark.readStream
.format("kafka")
.option("subscribe", "input")
.load()
.groupBy('value.cast("string") as 'key)
.agg(count("*") as 'value)
.writeStream
.format("kafka")
.option("topic", "output")
.trigger("1 minute")
.outputMode("update")
.option("checkpointLocation", "…")
.start()
Checkpoint
• Tracks the progress of a
query in persistent storage
• Can be used to restart the
query if there is a failure.
Fault-tolerance with Checkpointing
Checkpointing – tracks progress
(offsets) of consuming data from
the source and intermediate state.
Offsets and metadata saved as JSON
Can resume after changing your
streaming transformations
end-to-end
exactly-once
guarantees
process
newdata
t = 1 t = 2 t = 3
process
newdata
process
newdata
write
ahead
log
Complex Streaming ETL
Traditional ETL
Raw, dirty, un/semi-structured is data dumped as files
Periodic jobs run every few hours to convert raw data
to structured data ready for further analytics
17
file
dump
seconds hours
table
10101010
Traditional ETL
Hours of delay before taking decisions on latest data
Unacceptable when time is of essence
[intrusion detection, anomaly detection, etc.]
file
dump
seconds hours
table
10101010
Streaming ETL w/ Structured Streaming
Structured Streaming enables raw data to be available
as structured data as soon as possible
19
seconds
table
10101010
Streaming ETL w/ Structured Streaming
Example
Json data being received in Kafka
Parse nested json and flatten it
Store in structured Parquet table
Get end-to-end failure guarantees
val rawData = spark.readStream
.format("kafka")
.option("kafka.boostrap.servers",...)
.option("subscribe", "topic")
.load()
val parsedData = rawData
.selectExpr("cast (value as string) as json"))
.select(from_json("json", schema).as("data"))
.select("data.*")
val query = parsedData.writeStream
.option("checkpointLocation", "/checkpoint")
.partitionBy("date")
.format("parquet")
.start("/parquetTable")
Reading from Kafka
Specify options to configure
How?
kafka.boostrap.servers => broker1,broker2
What?
subscribe => topic1,topic2,topic3 // fixed list of topics
subscribePattern => topic* // dynamic list of topics
assign => {"topicA":[0,1] } // specific partitions
Where?
startingOffsets => latest(default) / earliest / {"topicA":{"0":23,"1":345} }
val rawData = spark.readStream
.format("kafka")
.option("kafka.boostrap.servers",...)
.option("subscribe", "topic")
.load()
Reading from Kafka
val rawData = spark.readStream
.format("kafka")
.option("kafka.boostrap.servers",...)
.option("subscribe", "topic")
.load()
rawData dataframe has
the following columns
key value topic partition offset timestamp
[binary] [binary] "topicA" 0 345 1486087873
[binary] [binary] "topicB" 3 2890 1486086721
Transforming Data
Cast binary value to string
Name it column json
val parsedData = rawData
.selectExpr("cast (value as string) as json")
.select(from_json("json", schema).as("data"))
.select("data.*")
Transforming Data
Cast binary value to string
Name it column json
Parse json string and expand into
nested columns, name it data
val parsedData = rawData
.selectExpr("cast (value as string) as json")
.select(from_json("json", schema).as("data"))
.select("data.*")
json
{ "timestamp": 1486087873, "device": "devA", …}
{ "timestamp": 1486082418, "device": "devX", …}
data (nested)
timestamp device …
1486087873 devA …
1486086721 devX …
from_json("json")
as "data"
Transforming Data
Cast binary value to string
Name it column json
Parse json string and expand into
nested columns, name it data
Flatten the nested columns
val parsedData = rawData
.selectExpr("cast (value as string) as json")
.select(from_json("json", schema).as("data"))
.select("data.*")
data (nested)
timestamp device …
1486087873 devA …
1486086721 devX …
timestamp device …
1486087873 devA …
1486086721 devX …
select("data.*")
(not nested)
Transforming Data
Cast binary value to string
Name it column json
Parse json string and expand into
nested columns, name it data
Flatten the nested columns
val parsedData = rawData
.selectExpr("cast (value as string) as json")
.select(from_json("json", schema).as("data"))
.select("data.*")
powerful built-in APIs to
perform complex data
transformations
from_json, to_json, explode, ...
100s of functions
(see our blog post)
Writing to
Save parsed data as Parquet
table in the given path
Partition files by date so that
future queries on time slices of
data is fast
e.g. query on last 48 hours of data
val query = parsedData.writeStream
.option("checkpointLocation", ...)
.partitionBy("date")
.format("parquet")
.start("/parquetTable")
Checkpointing
Enable checkpointing by
setting the checkpoint
location to save offset logs
start actually starts a
continuous running
StreamingQuery in the
Spark cluster
val query = parsedData.writeStream
.option("checkpointLocation", ...)
.format("parquet")
.partitionBy("date")
.start("/parquetTable/")
Streaming Query
query is a handle to the continuously
running StreamingQuery
Used to monitor and manage the
execution
val query = parsedData.writeStream
.option("checkpointLocation", ...)
.format("parquet")
.partitionBy("date")
.start("/parquetTable")/")
process
newdata
t = 1 t = 2 t = 3
process
newdata
process
newdata
StreamingQuery
Data Consistency on Ad-hoc Queries
Data available for complex, ad-hoc analytics within seconds
Parquet table is updated atomically, ensures prefix integrity
Even if distributed, ad-hoc queries will see either all updates from
streaming query or none, read more in our blog
https://databricks.com/blog/2016/07/28/structured-streaming-in-apache-spark.html
complex, ad-hoc
queries on
latest
data
seconds!
More Kafka Support [Spark 2.2]
Write out to Kafka
Dataframe must have binary fields
named key and value
Direct, interactive and batch
queries on Kafka
Makes Kafka even more powerful
as a storage platform!
result.writeStream
.format("kafka")
.option("topic", "output")
.start()
val df = spark
.read // not readStream
.format("kafka")
.option("subscribe", "topic")
.load()
df.registerTempTable("topicData")
spark.sql("select value from topicData")
Amazon Kinesis [Databricks Runtime 3.0]
Configure with options (similar to Kafka)
How?
region => us-west-2 / us-east-1 / ...
awsAccessKey (optional) => AKIA...
awsSecretKey (optional) => ...
What?
streamName => name-of-the-stream
Where?
initialPosition => latest(default) / earliest / trim_horizon
spark.readStream
.format("kinesis")
.option("streamName", "myStream")
.option("region", "us-west-2")
.option("awsAccessKey", ...)
.option("awsSecretKey", ...)
.load()
Working With Time
Event Time
Many use cases require aggregate statistics by event time
E.g. what's the #errors in each system in the 1 hour windows?
Many challenges
Extracting event time from data, handling late, out-of-order data
DStream APIs were insufficient for event-time stuff
Event time Aggregations
Windowing is just another type of grouping in Struct.
Streaming
number of records every hour
Support UDAFs!
parsedData
.groupBy(window("timestamp","1 hour"))
.count()
parsedData
.groupBy(
"device",
window("timestamp","10 mins"))
.avg("signal")
avg signal strength of each
device every 10 mins
Stateful Processing for Aggregations
Aggregates has to be saved as
distributed state between triggers
Each trigger reads previous state and
writes updated state
State stored in memory,
backed by write ahead log in HDFS/S3
Fault-tolerant, exactly-once guarantee!
process
newdata
t = 1
sink
src
t = 2
process
newdata
sink
src
t = 3
process
newdata
sink
src
state state
write
ahead
log
state updates
are written to
log for checkpointing
state
Automatically handles Late Data
12:00 - 13:00 1 12:00 - 13:00 3
13:00 - 14:00 1
12:00 - 13:00 3
13:00 - 14:00 2
14:00 - 15:00 5
12:00 - 13:00 5
13:00 - 14:00 2
14:00 - 15:00 5
15:00 - 16:00 4
12:00 - 13:00 3
13:00 - 14:00 2
14:00 - 15:00 6
15:00 - 16:00 4
16:00 - 17:00 3
13:00 14:00 15:00 16:00 17:00Keeping state allows
late data to update
counts of old windows
red = state updated
with late data
But size of the state increases indefinitely
if old windows are not dropped
Watermarking
Watermark - moving threshold of
how late data is expected to be
and when to drop old state
Trails behind max seen event time
Trailing gap is configurable
event time
max event time
watermark data older
than
watermark
not expected
12:30 PM
12:20 PM
trailing gap
of 10 mins
Watermarking
Data newer than watermark may
be late, but allowed to aggregate
Data older than watermark is "too
late" and dropped
Windows older than watermark
automatically deleted to limit the
amount of intermediate state
max event time
event time
watermark
late data
allowed to
aggregate
data too
late,
dropped
Watermarking
max event time
event time
watermark
allowed
lateness
of 10 mins
parsedData
.withWatermark("timestamp", "10 minutes")
.groupBy(window("timestamp","5 minutes"))
.count()
late data
allowed to
aggregate
data too
late,
dropped
Useful only in stateful operations
(streaming aggs, dropDuplicates, mapGroupsWithState, ...)
Ignored in non-stateful streaming
queries and batch queries
Watermarking
data too late,
ignored in counts,
state dropped
Processing Time12:00
12:05
12:10
12:15
12:10 12:15 12:20
12:07
12:13
12:08
EventTime
12:15
12:18
12:04
watermark updated to
12:14 - 10m = 12:04
for next trigger,
state < 12:04 deleted
data is late, but
considered in counts
parsedData
.withWatermark("timestamp", "10 minutes")
.groupBy(window("timestamp","5 minutes"))
.count()
system tracks max
observed event time
12:08
wm = 12:04
10min
12:14
More details in my blog post
Clean separation of concerns
parsedData
.withWatermark("timestamp", "10 minutes")
.groupBy(window("timestamp","5 minutes"))
.count()
.writeStream
.trigger("10 seconds")
.start()
Query Semantics
Processing Details
separated from
Clean separation of concerns
parsedData
.withWatermark("timestamp", "10 minutes")
.groupBy(window("timestamp","5 minutes"))
.count()
.writeStream
.trigger("10 seconds")
.start()
Query Semantics
How to group data by time?
(same for batch & streaming)
Processing Details
Clean separation of concerns
parsedData
.withWatermark("timestamp", "10 minutes")
.groupBy(window("timestamp","5 minutes"))
.count()
.writeStream
.trigger("10 seconds")
.start()
Query Semantics
How to group data by time?
(same for batch & streaming)
Processing Details
How late can data be?
Clean separation of concerns
parsedData
.withWatermark("timestamp", "10 minutes")
.groupBy(window("timestamp","5 minutes"))
.count()
.writeStream
.trigger("10 seconds")
.start()
Query Semantics
How to group data by time?
(same for batch & streaming)
Processing Details
How late can data be?
How often to emit updates?
Arbitrary Stateful Operations [Spark 2.2]
mapGroupsWithState
allows any user-defined
stateful function to a
user-defined state
Direct support for per-key
timeouts in event-time or
processing-time
Supports Scala and Java
46
ds.groupByKey(_.id)
.mapGroupsWithState
(timeoutConf)
(mappingWithStateFunc)
def mappingWithStateFunc(
key: K,
values: Iterator[V],
state: GroupState[S]): U = {
// update or remove state
// set timeouts
// return mapped value
}
Other interesting operations
Streaming Deduplication
Watermarks to limit state
Stream-batch Joins
Stream-stream Joins
Can use mapGroupsWithState
Direct support oming soon!
val batchData = spark.read
.format("parquet")
.load("/additional-data")
parsedData.join(batchData, "device")
parsedData.dropDuplicates("eventId")
Building Complex
Continuous Apps
Metric Processing @
Dashboards Analyze	trends	in	usage	as	they	occur
Alerts Notify	engineers	of	critical	issues
Ad-hoc	Analysis Diagnose	issues	when	they	occur
ETL Clean, normalize and store historical data
Events generated by user actions (logins, clicks, spark job updates)
Metric Processing @
Dashboards
Alerts
Ad-hoc	Analysis
ETL
Difficult with only streaming frameworks
Limited retention in
streaming storage
Inefficient for ad-hoc queries
Hard for novice users
(limited or no SQL support)
Metric Processing @
=
Metrics
Filter
ETL
Dashboards
Ad-hoc
Analysis
Alerts
Read from
rawLogs = spark.readStream
.format("kafka")
.option("kafka.bootstrap.servers", ...)
.option("subscribe", "rawLogs")
.load()
augmentedLogs = rawLogs
.withColumn("msg",
from_json($"value".cast("string"),
schema))
.select("timestamp", "msg.*")
.join(table("customers"), ["customer_id"])
DataFrames can be
reused for multiple
streams
Can build libraries of
useful DataFrames and
share code between
applications
JSON ETL
Write to
augmented
.repartition(1)
.writeStream
.format("parquet")
.option("path", "/data/metrics")
.trigger("1 minute")
.start()
Store augmented stream as efficient
columnar data for later processing
Latency: ~1 minute
Buffer data and
write one large file
every minute for
efficient reads
ETL
Dashboards
logins = spark.readStream.parquet("/data/metrics")
.where("metric = 'login'")
.groupBy(window("timestamp", "1 minute"))
.count()
display(logins) // Visualize in Databricks notebooks
Always up-to-date visualizations of
important business trends
Latency: ~1 minute to hours (configurable) Dashboards
Filter and write to
filteredLogs = augmentedLogs
.where("eventType = 'clusterHeartbeat'")
.selectExpr("to_json(struct("*")) as value")
filteredLogs.writeStream
.format("kafka")
.option("kafka.bootstrap.servers", ...)
.option("topic", "clusterHeartbeats")
.start()
Forward filtered and augmented
events back to Kafka
Latency: ~100ms average
Filter
to_json() to convert
columns back into json
string, and then save as
different Kafka topic
Simple Alerts
sparkErrors
.as[ClusterHeartBeat]
.filter(_.load > 99)
.writeStream
.foreach(new PagerdutySink(credentials))
E.g. Alert when Spark cluster load > threshold
Latency: ~100 ms
Alerts
Notify PagerDuty
Complex Alerts
sparkErrors
.as[ClusterHeartBeat]
.groupBy(_.id)
.flatMapGroupsWithState(Update, ProcessingTimeTimeout("1 minute")) {
(id: Int, events: Iterator[ClusterHeartBeat], state: GroupState[ClusterState]) =>
... // check if cluster non-responsive for a while
}
E.g. Monitor health of Spark clusters
using custom stateful logic
Latency: ~10 seconds
Alerts
React if no heartbeat
from cluster for 1 min
Ad-hoc Analysis
SELECT *
FROM parquet.`/data/metrics`
WHERE level IN ('WARN', 'ERROR')
AND customer = "…"
AND timestamp < now() – INTERVAL 1 HOUR
Trouble shoot problems as they
occur with latest information
Latency: ~1 minute
Ad-hoc
Analysis
will read latest data
when query executed
Metric Processing @
=
Metrics
Filter
ETL
Dashboards
Ad-hoc
Analysis
Alerts14+ billion records / hour
with 10 nodes
meet diverse latency requirements
as efficiently as possible
More Info
Structured Streaming Programming Guide
http://spark.apache.org/docs/latest/structured-streaming-programming-guide.html
Databricks blog posts for more focused discussions
https://databricks.com/blog/2016/07/28/structured-streaming-in-apache-spark.html
https://databricks.com/blog/2017/01/19/real-time-streaming-etl-structured-streaming-apache-spark-2-1.html
https://databricks.com/blog/2017/02/23/working-complex-data-formats-structured-streaming-apache-spark-2-1.html
https://databricks.com/blog/2017/04/26/processing-data-in-apache-kafka-with-structured-streaming-in-apache-spark-2-2.html
https://databricks.com/blog/2017/05/08/event-time-aggregation-watermarking-apache-sparks-structured-streaming.html
and more to come, stay tuned!!
UNIFIED ANALYTICS PLATFORM
Try Apache Spark in Databricks!
• Collaborative cloud environment
• Free version (community edition)
DATABRICKS RUNTIME 3.0
• Apache Spark - optimized for the cloud
• Caching and optimization layer - DBIO
• Enterprise security - DBES
Try for free today
databricks.com

More Related Content

What's hot

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
 
Deep Dive into Project Tungsten: Bringing Spark Closer to Bare Metal-(Josh Ro...
Deep Dive into Project Tungsten: Bringing Spark Closer to Bare Metal-(Josh Ro...Deep Dive into Project Tungsten: Bringing Spark Closer to Bare Metal-(Josh Ro...
Deep Dive into Project Tungsten: Bringing Spark Closer to Bare Metal-(Josh Ro...Spark Summit
 
Sparkcamp @ Strata CA: Intro to Apache Spark with Hands-on Tutorials
Sparkcamp @ Strata CA: Intro to Apache Spark with Hands-on TutorialsSparkcamp @ Strata CA: Intro to Apache Spark with Hands-on Tutorials
Sparkcamp @ Strata CA: Intro to Apache Spark with Hands-on TutorialsDatabricks
 
Deep Dive: Memory Management in Apache Spark
Deep Dive: Memory Management in Apache SparkDeep Dive: Memory Management in Apache Spark
Deep Dive: Memory Management in Apache SparkDatabricks
 
Advanced Apache Spark Meetup Spark SQL + DataFrames + Catalyst Optimizer + Da...
Advanced Apache Spark Meetup Spark SQL + DataFrames + Catalyst Optimizer + Da...Advanced Apache Spark Meetup Spark SQL + DataFrames + Catalyst Optimizer + Da...
Advanced Apache Spark Meetup Spark SQL + DataFrames + Catalyst Optimizer + Da...Chris Fregly
 
Frustration-Reduced Spark: DataFrames and the Spark Time-Series Library
Frustration-Reduced Spark: DataFrames and the Spark Time-Series LibraryFrustration-Reduced Spark: DataFrames and the Spark Time-Series Library
Frustration-Reduced Spark: DataFrames and the Spark Time-Series LibraryIlya Ganelin
 
Introducing DataFrames in Spark for Large Scale Data Science
Introducing DataFrames in Spark for Large Scale Data ScienceIntroducing DataFrames in Spark for Large Scale Data Science
Introducing DataFrames in Spark for Large Scale Data ScienceDatabricks
 
Spark Summit EU 2015: Spark DataFrames: Simple and Fast Analysis of Structure...
Spark Summit EU 2015: Spark DataFrames: Simple and Fast Analysis of Structure...Spark Summit EU 2015: Spark DataFrames: Simple and Fast Analysis of Structure...
Spark Summit EU 2015: Spark DataFrames: Simple and Fast Analysis of Structure...Databricks
 
Data Source API in Spark
Data Source API in SparkData Source API in Spark
Data Source API in SparkDatabricks
 
DataEngConf SF16 - Spark SQL Workshop
DataEngConf SF16 - Spark SQL WorkshopDataEngConf SF16 - Spark SQL Workshop
DataEngConf SF16 - Spark SQL WorkshopHakka Labs
 
Tuning and Debugging in Apache Spark
Tuning and Debugging in Apache SparkTuning and Debugging in Apache Spark
Tuning and Debugging in Apache SparkPatrick Wendell
 
Building a modern Application with DataFrames
Building a modern Application with DataFramesBuilding a modern Application with DataFrames
Building a modern Application with DataFramesSpark Summit
 
Automated Spark Deployment With Declarative Infrastructure
Automated Spark Deployment With Declarative InfrastructureAutomated Spark Deployment With Declarative Infrastructure
Automated Spark Deployment With Declarative InfrastructureSpark Summit
 
Modus operandi of Spark Streaming - Recipes for Running your Streaming Applic...
Modus operandi of Spark Streaming - Recipes for Running your Streaming Applic...Modus operandi of Spark Streaming - Recipes for Running your Streaming Applic...
Modus operandi of Spark Streaming - Recipes for Running your Streaming Applic...DataWorks Summit
 
Simplifying Big Data Analytics with Apache Spark
Simplifying Big Data Analytics with Apache SparkSimplifying Big Data Analytics with Apache Spark
Simplifying Big Data Analytics with Apache SparkDatabricks
 
Exactly-Once Streaming from Kafka-(Cody Koeninger, Kixer)
Exactly-Once Streaming from Kafka-(Cody Koeninger, Kixer)Exactly-Once Streaming from Kafka-(Cody Koeninger, Kixer)
Exactly-Once Streaming from Kafka-(Cody Koeninger, Kixer)Spark Summit
 
Frustration-Reduced PySpark: Data engineering with DataFrames
Frustration-Reduced PySpark: Data engineering with DataFramesFrustration-Reduced PySpark: Data engineering with DataFrames
Frustration-Reduced PySpark: Data engineering with DataFramesIlya Ganelin
 
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 MeetupDatabricks
 
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
 

What's hot (20)

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
 
Deep Dive into Project Tungsten: Bringing Spark Closer to Bare Metal-(Josh Ro...
Deep Dive into Project Tungsten: Bringing Spark Closer to Bare Metal-(Josh Ro...Deep Dive into Project Tungsten: Bringing Spark Closer to Bare Metal-(Josh Ro...
Deep Dive into Project Tungsten: Bringing Spark Closer to Bare Metal-(Josh Ro...
 
Sparkcamp @ Strata CA: Intro to Apache Spark with Hands-on Tutorials
Sparkcamp @ Strata CA: Intro to Apache Spark with Hands-on TutorialsSparkcamp @ Strata CA: Intro to Apache Spark with Hands-on Tutorials
Sparkcamp @ Strata CA: Intro to Apache Spark with Hands-on Tutorials
 
Deep Dive: Memory Management in Apache Spark
Deep Dive: Memory Management in Apache SparkDeep Dive: Memory Management in Apache Spark
Deep Dive: Memory Management in Apache Spark
 
Advanced Apache Spark Meetup Spark SQL + DataFrames + Catalyst Optimizer + Da...
Advanced Apache Spark Meetup Spark SQL + DataFrames + Catalyst Optimizer + Da...Advanced Apache Spark Meetup Spark SQL + DataFrames + Catalyst Optimizer + Da...
Advanced Apache Spark Meetup Spark SQL + DataFrames + Catalyst Optimizer + Da...
 
Frustration-Reduced Spark: DataFrames and the Spark Time-Series Library
Frustration-Reduced Spark: DataFrames and the Spark Time-Series LibraryFrustration-Reduced Spark: DataFrames and the Spark Time-Series Library
Frustration-Reduced Spark: DataFrames and the Spark Time-Series Library
 
Introducing DataFrames in Spark for Large Scale Data Science
Introducing DataFrames in Spark for Large Scale Data ScienceIntroducing DataFrames in Spark for Large Scale Data Science
Introducing DataFrames in Spark for Large Scale Data Science
 
Spark Summit EU 2015: Spark DataFrames: Simple and Fast Analysis of Structure...
Spark Summit EU 2015: Spark DataFrames: Simple and Fast Analysis of Structure...Spark Summit EU 2015: Spark DataFrames: Simple and Fast Analysis of Structure...
Spark Summit EU 2015: Spark DataFrames: Simple and Fast Analysis of Structure...
 
Spark etl
Spark etlSpark etl
Spark etl
 
Data Source API in Spark
Data Source API in SparkData Source API in Spark
Data Source API in Spark
 
DataEngConf SF16 - Spark SQL Workshop
DataEngConf SF16 - Spark SQL WorkshopDataEngConf SF16 - Spark SQL Workshop
DataEngConf SF16 - Spark SQL Workshop
 
Tuning and Debugging in Apache Spark
Tuning and Debugging in Apache SparkTuning and Debugging in Apache Spark
Tuning and Debugging in Apache Spark
 
Building a modern Application with DataFrames
Building a modern Application with DataFramesBuilding a modern Application with DataFrames
Building a modern Application with DataFrames
 
Automated Spark Deployment With Declarative Infrastructure
Automated Spark Deployment With Declarative InfrastructureAutomated Spark Deployment With Declarative Infrastructure
Automated Spark Deployment With Declarative Infrastructure
 
Modus operandi of Spark Streaming - Recipes for Running your Streaming Applic...
Modus operandi of Spark Streaming - Recipes for Running your Streaming Applic...Modus operandi of Spark Streaming - Recipes for Running your Streaming Applic...
Modus operandi of Spark Streaming - Recipes for Running your Streaming Applic...
 
Simplifying Big Data Analytics with Apache Spark
Simplifying Big Data Analytics with Apache SparkSimplifying Big Data Analytics with Apache Spark
Simplifying Big Data Analytics with Apache Spark
 
Exactly-Once Streaming from Kafka-(Cody Koeninger, Kixer)
Exactly-Once Streaming from Kafka-(Cody Koeninger, Kixer)Exactly-Once Streaming from Kafka-(Cody Koeninger, Kixer)
Exactly-Once Streaming from Kafka-(Cody Koeninger, Kixer)
 
Frustration-Reduced PySpark: Data engineering with DataFrames
Frustration-Reduced PySpark: Data engineering with DataFramesFrustration-Reduced PySpark: Data engineering with DataFrames
Frustration-Reduced PySpark: Data engineering with DataFrames
 
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
 
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?
 

Viewers also liked

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
 
Introduction to Structured Streaming
Introduction to Structured StreamingIntroduction to Structured Streaming
Introduction to Structured Streamingdatamantra
 
Structured Streaming Using Spark 2.1
Structured Streaming Using Spark 2.1Structured Streaming Using Spark 2.1
Structured Streaming Using Spark 2.1Sigmoid
 
A Deep Dive into Structured Streaming in Apache Spark
A Deep Dive into Structured Streaming in Apache Spark A Deep Dive into Structured Streaming in Apache Spark
A Deep Dive into Structured Streaming in Apache Spark Anyscale
 
Introduction to Structured Streaming
Introduction to Structured StreamingIntroduction to Structured Streaming
Introduction to Structured StreamingKnoldus Inc.
 
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
 
Introduction to Structured streaming
Introduction to Structured streamingIntroduction to Structured streaming
Introduction to Structured streamingdatamantra
 
Apache Spark 2.0: A Deep Dive Into Structured Streaming - by Tathagata Das
Apache Spark 2.0: A Deep Dive Into Structured Streaming - by Tathagata Das Apache Spark 2.0: A Deep Dive Into Structured Streaming - by Tathagata Das
Apache Spark 2.0: A Deep Dive Into Structured Streaming - by Tathagata Das Databricks
 
Deep dive into stateful stream processing in structured streaming by Tathaga...
Deep dive into stateful stream processing in structured streaming  by Tathaga...Deep dive into stateful stream processing in structured streaming  by Tathaga...
Deep dive into stateful stream processing in structured streaming by Tathaga...Databricks
 

Viewers also liked (9)

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 ...
 
Introduction to Structured Streaming
Introduction to Structured StreamingIntroduction to Structured Streaming
Introduction to Structured Streaming
 
Structured Streaming Using Spark 2.1
Structured Streaming Using Spark 2.1Structured Streaming Using Spark 2.1
Structured Streaming Using Spark 2.1
 
A Deep Dive into Structured Streaming in Apache Spark
A Deep Dive into Structured Streaming in Apache Spark A Deep Dive into Structured Streaming in Apache Spark
A Deep Dive into Structured Streaming in Apache Spark
 
Introduction to Structured Streaming
Introduction to Structured StreamingIntroduction to Structured Streaming
Introduction to Structured Streaming
 
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
 
Introduction to Structured streaming
Introduction to Structured streamingIntroduction to Structured streaming
Introduction to Structured streaming
 
Apache Spark 2.0: A Deep Dive Into Structured Streaming - by Tathagata Das
Apache Spark 2.0: A Deep Dive Into Structured Streaming - by Tathagata Das Apache Spark 2.0: A Deep Dive Into Structured Streaming - by Tathagata Das
Apache Spark 2.0: A Deep Dive Into Structured Streaming - by Tathagata Das
 
Deep dive into stateful stream processing in structured streaming by Tathaga...
Deep dive into stateful stream processing in structured streaming  by Tathaga...Deep dive into stateful stream processing in structured streaming  by Tathaga...
Deep dive into stateful stream processing in structured streaming by Tathaga...
 

Similar to Easy, scalable, fault tolerant stream processing with structured streaming - spark summit 2017

Writing Continuous Applications with Structured Streaming Python APIs in Apac...
Writing Continuous Applications with Structured Streaming Python APIs in Apac...Writing Continuous Applications with Structured Streaming Python APIs in Apac...
Writing Continuous Applications with Structured Streaming Python APIs in Apac...Databricks
 
Writing Continuous Applications with Structured Streaming in PySpark
Writing Continuous Applications with Structured Streaming in PySparkWriting Continuous Applications with Structured Streaming in PySpark
Writing Continuous Applications with Structured Streaming in PySparkDatabricks
 
Leveraging Azure Databricks to minimize time to insight by combining Batch an...
Leveraging Azure Databricks to minimize time to insight by combining Batch an...Leveraging Azure Databricks to minimize time to insight by combining Batch an...
Leveraging Azure Databricks to minimize time to insight by combining Batch an...Microsoft Tech Community
 
Writing Continuous Applications with Structured Streaming PySpark API
Writing Continuous Applications with Structured Streaming PySpark APIWriting Continuous Applications with Structured Streaming PySpark API
Writing Continuous Applications with Structured Streaming PySpark APIDatabricks
 
Spark Structured Streaming
Spark Structured Streaming Spark Structured Streaming
Spark Structured Streaming Revin Chalil
 
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
 
A Tale of Two APIs: Using Spark Streaming In Production
A Tale of Two APIs: Using Spark Streaming In ProductionA Tale of Two APIs: Using Spark Streaming In Production
A Tale of Two APIs: Using Spark Streaming In ProductionLightbend
 
Productizing Structured Streaming Jobs
Productizing Structured Streaming JobsProductizing Structured Streaming Jobs
Productizing Structured Streaming JobsDatabricks
 
Kafka Summit NYC 2017 - Easy, Scalable, Fault-tolerant Stream Processing with...
Kafka Summit NYC 2017 - Easy, Scalable, Fault-tolerant Stream Processing with...Kafka Summit NYC 2017 - Easy, Scalable, Fault-tolerant Stream Processing with...
Kafka Summit NYC 2017 - Easy, Scalable, Fault-tolerant Stream Processing with...confluent
 
Elasticsearch And Apache Lucene For Apache Spark And MLlib
Elasticsearch And Apache Lucene For Apache Spark And MLlibElasticsearch And Apache Lucene For Apache Spark And MLlib
Elasticsearch And Apache Lucene For Apache Spark And MLlibJen Aman
 
Designing Structured Streaming Pipelines—How to Architect Things Right
Designing Structured Streaming Pipelines—How to Architect Things RightDesigning Structured Streaming Pipelines—How to Architect Things Right
Designing Structured Streaming Pipelines—How to Architect Things RightDatabricks
 
Apache: Big Data - Starting with Apache Spark, Best Practices
Apache: Big Data - Starting with Apache Spark, Best PracticesApache: Big Data - Starting with Apache Spark, Best Practices
Apache: Big Data - Starting with Apache Spark, Best Practicesfelixcss
 
Apache Spark Structured Streaming + Apache Kafka = ♡
Apache Spark Structured Streaming + Apache Kafka = ♡Apache Spark Structured Streaming + Apache Kafka = ♡
Apache Spark Structured Streaming + Apache Kafka = ♡Bartosz Konieczny
 
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
 
Building a Unified Data Pipline in Spark / Apache Sparkを用いたBig Dataパイプラインの統一
Building a Unified Data Pipline in Spark / Apache Sparkを用いたBig Dataパイプラインの統一Building a Unified Data Pipline in Spark / Apache Sparkを用いたBig Dataパイプラインの統一
Building a Unified Data Pipline in Spark / Apache Sparkを用いたBig Dataパイプラインの統一scalaconfjp
 
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
 
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
 
From Overnight to Always On @ Jfokus 2019
From Overnight to Always On @ Jfokus 2019From Overnight to Always On @ Jfokus 2019
From Overnight to Always On @ Jfokus 2019Enno Runne
 
Spark (Structured) Streaming vs. Kafka Streams - two stream processing platfo...
Spark (Structured) Streaming vs. Kafka Streams - two stream processing platfo...Spark (Structured) Streaming vs. Kafka Streams - two stream processing platfo...
Spark (Structured) Streaming vs. Kafka Streams - two stream processing platfo...Guido Schmutz
 
Stream or not to Stream?

Stream or not to Stream?
Stream or not to Stream?

Stream or not to Stream?
Lukasz Byczynski
 

Similar to Easy, scalable, fault tolerant stream processing with structured streaming - spark summit 2017 (20)

Writing Continuous Applications with Structured Streaming Python APIs in Apac...
Writing Continuous Applications with Structured Streaming Python APIs in Apac...Writing Continuous Applications with Structured Streaming Python APIs in Apac...
Writing Continuous Applications with Structured Streaming Python APIs in Apac...
 
Writing Continuous Applications with Structured Streaming in PySpark
Writing Continuous Applications with Structured Streaming in PySparkWriting Continuous Applications with Structured Streaming in PySpark
Writing Continuous Applications with Structured Streaming in PySpark
 
Leveraging Azure Databricks to minimize time to insight by combining Batch an...
Leveraging Azure Databricks to minimize time to insight by combining Batch an...Leveraging Azure Databricks to minimize time to insight by combining Batch an...
Leveraging Azure Databricks to minimize time to insight by combining Batch an...
 
Writing Continuous Applications with Structured Streaming PySpark API
Writing Continuous Applications with Structured Streaming PySpark APIWriting Continuous Applications with Structured Streaming PySpark API
Writing Continuous Applications with Structured Streaming PySpark API
 
Spark Structured Streaming
Spark Structured Streaming Spark Structured Streaming
Spark Structured Streaming
 
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...
 
A Tale of Two APIs: Using Spark Streaming In Production
A Tale of Two APIs: Using Spark Streaming In ProductionA Tale of Two APIs: Using Spark Streaming In Production
A Tale of Two APIs: Using Spark Streaming In Production
 
Productizing Structured Streaming Jobs
Productizing Structured Streaming JobsProductizing Structured Streaming Jobs
Productizing Structured Streaming Jobs
 
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...
 
Elasticsearch And Apache Lucene For Apache Spark And MLlib
Elasticsearch And Apache Lucene For Apache Spark And MLlibElasticsearch And Apache Lucene For Apache Spark And MLlib
Elasticsearch And Apache Lucene For Apache Spark And MLlib
 
Designing Structured Streaming Pipelines—How to Architect Things Right
Designing Structured Streaming Pipelines—How to Architect Things RightDesigning Structured Streaming Pipelines—How to Architect Things Right
Designing Structured Streaming Pipelines—How to Architect Things Right
 
Apache: Big Data - Starting with Apache Spark, Best Practices
Apache: Big Data - Starting with Apache Spark, Best PracticesApache: Big Data - Starting with Apache Spark, Best Practices
Apache: Big Data - Starting with Apache Spark, Best Practices
 
Apache Spark Structured Streaming + Apache Kafka = ♡
Apache Spark Structured Streaming + Apache Kafka = ♡Apache Spark Structured Streaming + Apache Kafka = ♡
Apache Spark Structured Streaming + Apache Kafka = ♡
 
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
 
Building a Unified Data Pipline in Spark / Apache Sparkを用いたBig Dataパイプラインの統一
Building a Unified Data Pipline in Spark / Apache Sparkを用いたBig Dataパイプラインの統一Building a Unified Data Pipline in Spark / Apache Sparkを用いたBig Dataパイプラインの統一
Building a Unified Data Pipline in Spark / Apache Sparkを用いたBig Dataパイプラインの統一
 
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...
 
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...
 
From Overnight to Always On @ Jfokus 2019
From Overnight to Always On @ Jfokus 2019From Overnight to Always On @ Jfokus 2019
From Overnight to Always On @ Jfokus 2019
 
Spark (Structured) Streaming vs. Kafka Streams - two stream processing platfo...
Spark (Structured) Streaming vs. Kafka Streams - two stream processing platfo...Spark (Structured) Streaming vs. Kafka Streams - two stream processing platfo...
Spark (Structured) Streaming vs. Kafka Streams - two stream processing platfo...
 
Stream or not to Stream?

Stream or not to Stream?
Stream or not to Stream?

Stream or not to Stream?

 

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
 
Alfresco TTL#157 - Troubleshooting Made Easy: Deciphering Alfresco mTLS Confi...
Alfresco TTL#157 - Troubleshooting Made Easy: Deciphering Alfresco mTLS Confi...Alfresco TTL#157 - Troubleshooting Made Easy: Deciphering Alfresco mTLS Confi...
Alfresco TTL#157 - Troubleshooting Made Easy: Deciphering Alfresco mTLS Confi...Angel Borroy López
 
Balasore Best It Company|| Top 10 IT Company || Balasore Software company Odisha
Balasore Best It Company|| Top 10 IT Company || Balasore Software company OdishaBalasore Best It Company|| Top 10 IT Company || Balasore Software company Odisha
Balasore Best It Company|| Top 10 IT Company || Balasore Software company Odishasmiwainfosol
 
Unveiling the Future: Sylius 2.0 New Features
Unveiling the Future: Sylius 2.0 New FeaturesUnveiling the Future: Sylius 2.0 New Features
Unveiling the Future: Sylius 2.0 New FeaturesŁukasz Chruściel
 
SuccessFactors 1H 2024 Release - Sneak-Peek by Deloitte Germany
SuccessFactors 1H 2024 Release - Sneak-Peek by Deloitte GermanySuccessFactors 1H 2024 Release - Sneak-Peek by Deloitte Germany
SuccessFactors 1H 2024 Release - Sneak-Peek by Deloitte GermanyChristoph Pohl
 
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
 
Recruitment Management Software Benefits (Infographic)
Recruitment Management Software Benefits (Infographic)Recruitment Management Software Benefits (Infographic)
Recruitment Management Software Benefits (Infographic)Hr365.us smith
 
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
 
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
 
EY_Graph Database Powered Sustainability
EY_Graph Database Powered SustainabilityEY_Graph Database Powered Sustainability
EY_Graph Database Powered SustainabilityNeo4j
 
Intelligent Home Wi-Fi Solutions | ThinkPalm
Intelligent Home Wi-Fi Solutions | ThinkPalmIntelligent Home Wi-Fi Solutions | ThinkPalm
Intelligent Home Wi-Fi Solutions | ThinkPalmSujith Sukumaran
 
Introduction Computer Science - Software Design.pdf
Introduction Computer Science - Software Design.pdfIntroduction Computer Science - Software Design.pdf
Introduction Computer Science - Software Design.pdfFerryKemperman
 
Tech Tuesday - Mastering Time Management Unlock the Power of OnePlan's Timesh...
Tech Tuesday - Mastering Time Management Unlock the Power of OnePlan's Timesh...Tech Tuesday - Mastering Time Management Unlock the Power of OnePlan's Timesh...
Tech Tuesday - Mastering Time Management Unlock the Power of OnePlan's Timesh...OnePlan Solutions
 
KnowAPIs-UnknownPerf-jaxMainz-2024 (1).pptx
KnowAPIs-UnknownPerf-jaxMainz-2024 (1).pptxKnowAPIs-UnknownPerf-jaxMainz-2024 (1).pptx
KnowAPIs-UnknownPerf-jaxMainz-2024 (1).pptxTier1 app
 
办理学位证(UQ文凭证书)昆士兰大学毕业证成绩单原版一模一样
办理学位证(UQ文凭证书)昆士兰大学毕业证成绩单原版一模一样办理学位证(UQ文凭证书)昆士兰大学毕业证成绩单原版一模一样
办理学位证(UQ文凭证书)昆士兰大学毕业证成绩单原版一模一样umasea
 
MYjobs Presentation Django-based project
MYjobs Presentation Django-based projectMYjobs Presentation Django-based project
MYjobs Presentation Django-based projectAnoyGreter
 
CRM Contender Series: HubSpot vs. Salesforce
CRM Contender Series: HubSpot vs. SalesforceCRM Contender Series: HubSpot vs. Salesforce
CRM Contender Series: HubSpot vs. SalesforceBrainSell Technologies
 
Xen Safety Embedded OSS Summit April 2024 v4.pdf
Xen Safety Embedded OSS Summit April 2024 v4.pdfXen Safety Embedded OSS Summit April 2024 v4.pdf
Xen Safety Embedded OSS Summit April 2024 v4.pdfStefano Stabellini
 

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
 
Alfresco TTL#157 - Troubleshooting Made Easy: Deciphering Alfresco mTLS Confi...
Alfresco TTL#157 - Troubleshooting Made Easy: Deciphering Alfresco mTLS Confi...Alfresco TTL#157 - Troubleshooting Made Easy: Deciphering Alfresco mTLS Confi...
Alfresco TTL#157 - Troubleshooting Made Easy: Deciphering Alfresco mTLS Confi...
 
Balasore Best It Company|| Top 10 IT Company || Balasore Software company Odisha
Balasore Best It Company|| Top 10 IT Company || Balasore Software company OdishaBalasore Best It Company|| Top 10 IT Company || Balasore Software company Odisha
Balasore Best It Company|| Top 10 IT Company || Balasore Software company Odisha
 
Unveiling the Future: Sylius 2.0 New Features
Unveiling the Future: Sylius 2.0 New FeaturesUnveiling the Future: Sylius 2.0 New Features
Unveiling the Future: Sylius 2.0 New Features
 
SuccessFactors 1H 2024 Release - Sneak-Peek by Deloitte Germany
SuccessFactors 1H 2024 Release - Sneak-Peek by Deloitte GermanySuccessFactors 1H 2024 Release - Sneak-Peek by Deloitte Germany
SuccessFactors 1H 2024 Release - Sneak-Peek by Deloitte Germany
 
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 - ...
 
Recruitment Management Software Benefits (Infographic)
Recruitment Management Software Benefits (Infographic)Recruitment Management Software Benefits (Infographic)
Recruitment Management Software Benefits (Infographic)
 
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...
 
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
 
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)
 
EY_Graph Database Powered Sustainability
EY_Graph Database Powered SustainabilityEY_Graph Database Powered Sustainability
EY_Graph Database Powered Sustainability
 
Intelligent Home Wi-Fi Solutions | ThinkPalm
Intelligent Home Wi-Fi Solutions | ThinkPalmIntelligent Home Wi-Fi Solutions | ThinkPalm
Intelligent Home Wi-Fi Solutions | ThinkPalm
 
Hot Sexy call girls in Patel Nagar🔝 9953056974 🔝 escort Service
Hot Sexy call girls in Patel Nagar🔝 9953056974 🔝 escort ServiceHot Sexy call girls in Patel Nagar🔝 9953056974 🔝 escort Service
Hot Sexy call girls in Patel Nagar🔝 9953056974 🔝 escort Service
 
Introduction Computer Science - Software Design.pdf
Introduction Computer Science - Software Design.pdfIntroduction Computer Science - Software Design.pdf
Introduction Computer Science - Software Design.pdf
 
Tech Tuesday - Mastering Time Management Unlock the Power of OnePlan's Timesh...
Tech Tuesday - Mastering Time Management Unlock the Power of OnePlan's Timesh...Tech Tuesday - Mastering Time Management Unlock the Power of OnePlan's Timesh...
Tech Tuesday - Mastering Time Management Unlock the Power of OnePlan's Timesh...
 
KnowAPIs-UnknownPerf-jaxMainz-2024 (1).pptx
KnowAPIs-UnknownPerf-jaxMainz-2024 (1).pptxKnowAPIs-UnknownPerf-jaxMainz-2024 (1).pptx
KnowAPIs-UnknownPerf-jaxMainz-2024 (1).pptx
 
办理学位证(UQ文凭证书)昆士兰大学毕业证成绩单原版一模一样
办理学位证(UQ文凭证书)昆士兰大学毕业证成绩单原版一模一样办理学位证(UQ文凭证书)昆士兰大学毕业证成绩单原版一模一样
办理学位证(UQ文凭证书)昆士兰大学毕业证成绩单原版一模一样
 
MYjobs Presentation Django-based project
MYjobs Presentation Django-based projectMYjobs Presentation Django-based project
MYjobs Presentation Django-based project
 
CRM Contender Series: HubSpot vs. Salesforce
CRM Contender Series: HubSpot vs. SalesforceCRM Contender Series: HubSpot vs. Salesforce
CRM Contender Series: HubSpot vs. Salesforce
 
Xen Safety Embedded OSS Summit April 2024 v4.pdf
Xen Safety Embedded OSS Summit April 2024 v4.pdfXen Safety Embedded OSS Summit April 2024 v4.pdf
Xen Safety Embedded OSS Summit April 2024 v4.pdf
 

Easy, scalable, fault tolerant stream processing with structured streaming - spark summit 2017

  • 1. Easy, Scalable, Fault-tolerant Stream Processing with Structured Streaming Michael Armbrust - @michaelarmbrust Tathagata Das - @tathadas Spark Summit 2017 6th June, San Francisco
  • 2. TEAM About Databricks Started Spark project (now Apache Spark) at UC Berkeley in 2009 PRODUCT Unified Analytics Platform MISSION Making Big Data Simple
  • 4. Complexities in stream processing COMPLEX DATA Diverse data formats (json, avro, binary, …) Data can be dirty, late, out-of-order COMPLEX SYSTEMS Diverse storage systems (Kafka, S3, Kinesis, RDBMS, …) System failures COMPLEX WORKLOADS Combining streaming with interactive queries Machine learning
  • 5. Structured Streaming stream processing on Spark SQL engine fast, scalable, fault-tolerant rich, unified, high level APIs deal with complex data and complex workloads rich ecosystem of data sources integrate with many storage systems
  • 6. you should not have to reason about streaming
  • 7. you should write simple queries & Spark should continuously update the answer
  • 8. Streaming word count Anatomy of a Streaming Query
  • 9. Anatomy of a Streaming Query spark.readStream .format("kafka") .option("subscribe", "input") .load() .groupBy($"value".cast("string")) .count() .writeStream .format("kafka") .option("topic", "output") .trigger("1 minute") .outputMode(OutputMode.Complete()) .option("checkpointLocation", "…") .start() Source • Specify one or more locations to read data from • Built in support for Files/Kafka/Socket, pluggable. • Can include multiple sources of different types using union()
  • 10. Anatomy of a Streaming Query spark.readStream .format("kafka") .option("subscribe", "input") .load() .groupBy('value.cast("string") as 'key) .agg(count("*") as 'value) .writeStream .format("kafka") .option("topic", "output") .trigger("1 minute") .outputMode(OutputMode.Complete()) .option("checkpointLocation", "…") .start() Transformation • Using DataFrames, Datasets and/or SQL. • Catalyst figures out how to execute the transformation incrementally. • Internal processing always exactly-once.
  • 11. DataFrames, Datasets, SQL input = spark.readStream .format("kafka") .option("subscribe", "topic") .load() result = input .select("device", "signal") .where("signal > 15") result.writeStream .format("parquet") .start("dest-path") Logical Plan Read from Kafka Project device, signal Filter signal > 15 Write to Kafka Spark automatically streamifies! Spark SQL converts batch-like query to a series of incremental execution plans operating on new batches of data Series of Incremental Execution Plans Kafka Source Optimized Operator codegen, off- heap, etc. Kafka Sink Optimized Physical Plan process newdata t = 1 t = 2 t = 3 process newdata process newdata
  • 12. Anatomy of a Streaming Query spark.readStream .format("kafka") .option("subscribe", "input") .load() .groupBy('value.cast("string") as 'key) .agg(count("*") as 'value) .writeStream .format("kafka") .option("topic", "output") .trigger("1 minute") .outputMode(OutputMode.Complete()) .option("checkpointLocation", "…") .start() Sink • Accepts the output of each batch. • When supported sinks are transactional and exactly once (Files). • Use foreach to execute arbitrary code.
  • 13. Anatomy of a Streaming Query spark.readStream .format("kafka") .option("subscribe", "input") .load() .groupBy('value.cast("string") as 'key) .agg(count("*") as 'value) .writeStream .format("kafka") .option("topic", "output") .trigger("1 minute") .outputMode("update") .option("checkpointLocation", "…") .start() Output mode – What's output • Complete – Output the whole answer every time • Update – Output changed rows • Append – Output new rows only Trigger – When to output • Specified as a time, eventually supports data size • No trigger means as fast as possible
  • 14. Anatomy of a Streaming Query spark.readStream .format("kafka") .option("subscribe", "input") .load() .groupBy('value.cast("string") as 'key) .agg(count("*") as 'value) .writeStream .format("kafka") .option("topic", "output") .trigger("1 minute") .outputMode("update") .option("checkpointLocation", "…") .start() Checkpoint • Tracks the progress of a query in persistent storage • Can be used to restart the query if there is a failure.
  • 15. Fault-tolerance with Checkpointing Checkpointing – tracks progress (offsets) of consuming data from the source and intermediate state. Offsets and metadata saved as JSON Can resume after changing your streaming transformations end-to-end exactly-once guarantees process newdata t = 1 t = 2 t = 3 process newdata process newdata write ahead log
  • 17. Traditional ETL Raw, dirty, un/semi-structured is data dumped as files Periodic jobs run every few hours to convert raw data to structured data ready for further analytics 17 file dump seconds hours table 10101010
  • 18. Traditional ETL Hours of delay before taking decisions on latest data Unacceptable when time is of essence [intrusion detection, anomaly detection, etc.] file dump seconds hours table 10101010
  • 19. Streaming ETL w/ Structured Streaming Structured Streaming enables raw data to be available as structured data as soon as possible 19 seconds table 10101010
  • 20. Streaming ETL w/ Structured Streaming Example Json data being received in Kafka Parse nested json and flatten it Store in structured Parquet table Get end-to-end failure guarantees val rawData = spark.readStream .format("kafka") .option("kafka.boostrap.servers",...) .option("subscribe", "topic") .load() val parsedData = rawData .selectExpr("cast (value as string) as json")) .select(from_json("json", schema).as("data")) .select("data.*") val query = parsedData.writeStream .option("checkpointLocation", "/checkpoint") .partitionBy("date") .format("parquet") .start("/parquetTable")
  • 21. Reading from Kafka Specify options to configure How? kafka.boostrap.servers => broker1,broker2 What? subscribe => topic1,topic2,topic3 // fixed list of topics subscribePattern => topic* // dynamic list of topics assign => {"topicA":[0,1] } // specific partitions Where? startingOffsets => latest(default) / earliest / {"topicA":{"0":23,"1":345} } val rawData = spark.readStream .format("kafka") .option("kafka.boostrap.servers",...) .option("subscribe", "topic") .load()
  • 22. Reading from Kafka val rawData = spark.readStream .format("kafka") .option("kafka.boostrap.servers",...) .option("subscribe", "topic") .load() rawData dataframe has the following columns key value topic partition offset timestamp [binary] [binary] "topicA" 0 345 1486087873 [binary] [binary] "topicB" 3 2890 1486086721
  • 23. Transforming Data Cast binary value to string Name it column json val parsedData = rawData .selectExpr("cast (value as string) as json") .select(from_json("json", schema).as("data")) .select("data.*")
  • 24. Transforming Data Cast binary value to string Name it column json Parse json string and expand into nested columns, name it data val parsedData = rawData .selectExpr("cast (value as string) as json") .select(from_json("json", schema).as("data")) .select("data.*") json { "timestamp": 1486087873, "device": "devA", …} { "timestamp": 1486082418, "device": "devX", …} data (nested) timestamp device … 1486087873 devA … 1486086721 devX … from_json("json") as "data"
  • 25. Transforming Data Cast binary value to string Name it column json Parse json string and expand into nested columns, name it data Flatten the nested columns val parsedData = rawData .selectExpr("cast (value as string) as json") .select(from_json("json", schema).as("data")) .select("data.*") data (nested) timestamp device … 1486087873 devA … 1486086721 devX … timestamp device … 1486087873 devA … 1486086721 devX … select("data.*") (not nested)
  • 26. Transforming Data Cast binary value to string Name it column json Parse json string and expand into nested columns, name it data Flatten the nested columns val parsedData = rawData .selectExpr("cast (value as string) as json") .select(from_json("json", schema).as("data")) .select("data.*") powerful built-in APIs to perform complex data transformations from_json, to_json, explode, ... 100s of functions (see our blog post)
  • 27. Writing to Save parsed data as Parquet table in the given path Partition files by date so that future queries on time slices of data is fast e.g. query on last 48 hours of data val query = parsedData.writeStream .option("checkpointLocation", ...) .partitionBy("date") .format("parquet") .start("/parquetTable")
  • 28. Checkpointing Enable checkpointing by setting the checkpoint location to save offset logs start actually starts a continuous running StreamingQuery in the Spark cluster val query = parsedData.writeStream .option("checkpointLocation", ...) .format("parquet") .partitionBy("date") .start("/parquetTable/")
  • 29. Streaming Query query is a handle to the continuously running StreamingQuery Used to monitor and manage the execution val query = parsedData.writeStream .option("checkpointLocation", ...) .format("parquet") .partitionBy("date") .start("/parquetTable")/") process newdata t = 1 t = 2 t = 3 process newdata process newdata StreamingQuery
  • 30. Data Consistency on Ad-hoc Queries Data available for complex, ad-hoc analytics within seconds Parquet table is updated atomically, ensures prefix integrity Even if distributed, ad-hoc queries will see either all updates from streaming query or none, read more in our blog https://databricks.com/blog/2016/07/28/structured-streaming-in-apache-spark.html complex, ad-hoc queries on latest data seconds!
  • 31. More Kafka Support [Spark 2.2] Write out to Kafka Dataframe must have binary fields named key and value Direct, interactive and batch queries on Kafka Makes Kafka even more powerful as a storage platform! result.writeStream .format("kafka") .option("topic", "output") .start() val df = spark .read // not readStream .format("kafka") .option("subscribe", "topic") .load() df.registerTempTable("topicData") spark.sql("select value from topicData")
  • 32. Amazon Kinesis [Databricks Runtime 3.0] Configure with options (similar to Kafka) How? region => us-west-2 / us-east-1 / ... awsAccessKey (optional) => AKIA... awsSecretKey (optional) => ... What? streamName => name-of-the-stream Where? initialPosition => latest(default) / earliest / trim_horizon spark.readStream .format("kinesis") .option("streamName", "myStream") .option("region", "us-west-2") .option("awsAccessKey", ...) .option("awsSecretKey", ...) .load()
  • 34. Event Time Many use cases require aggregate statistics by event time E.g. what's the #errors in each system in the 1 hour windows? Many challenges Extracting event time from data, handling late, out-of-order data DStream APIs were insufficient for event-time stuff
  • 35. Event time Aggregations Windowing is just another type of grouping in Struct. Streaming number of records every hour Support UDAFs! parsedData .groupBy(window("timestamp","1 hour")) .count() parsedData .groupBy( "device", window("timestamp","10 mins")) .avg("signal") avg signal strength of each device every 10 mins
  • 36. Stateful Processing for Aggregations Aggregates has to be saved as distributed state between triggers Each trigger reads previous state and writes updated state State stored in memory, backed by write ahead log in HDFS/S3 Fault-tolerant, exactly-once guarantee! process newdata t = 1 sink src t = 2 process newdata sink src t = 3 process newdata sink src state state write ahead log state updates are written to log for checkpointing state
  • 37. Automatically handles Late Data 12:00 - 13:00 1 12:00 - 13:00 3 13:00 - 14:00 1 12:00 - 13:00 3 13:00 - 14:00 2 14:00 - 15:00 5 12:00 - 13:00 5 13:00 - 14:00 2 14:00 - 15:00 5 15:00 - 16:00 4 12:00 - 13:00 3 13:00 - 14:00 2 14:00 - 15:00 6 15:00 - 16:00 4 16:00 - 17:00 3 13:00 14:00 15:00 16:00 17:00Keeping state allows late data to update counts of old windows red = state updated with late data But size of the state increases indefinitely if old windows are not dropped
  • 38. Watermarking Watermark - moving threshold of how late data is expected to be and when to drop old state Trails behind max seen event time Trailing gap is configurable event time max event time watermark data older than watermark not expected 12:30 PM 12:20 PM trailing gap of 10 mins
  • 39. Watermarking Data newer than watermark may be late, but allowed to aggregate Data older than watermark is "too late" and dropped Windows older than watermark automatically deleted to limit the amount of intermediate state max event time event time watermark late data allowed to aggregate data too late, dropped
  • 40. Watermarking max event time event time watermark allowed lateness of 10 mins parsedData .withWatermark("timestamp", "10 minutes") .groupBy(window("timestamp","5 minutes")) .count() late data allowed to aggregate data too late, dropped Useful only in stateful operations (streaming aggs, dropDuplicates, mapGroupsWithState, ...) Ignored in non-stateful streaming queries and batch queries
  • 41. Watermarking data too late, ignored in counts, state dropped Processing Time12:00 12:05 12:10 12:15 12:10 12:15 12:20 12:07 12:13 12:08 EventTime 12:15 12:18 12:04 watermark updated to 12:14 - 10m = 12:04 for next trigger, state < 12:04 deleted data is late, but considered in counts parsedData .withWatermark("timestamp", "10 minutes") .groupBy(window("timestamp","5 minutes")) .count() system tracks max observed event time 12:08 wm = 12:04 10min 12:14 More details in my blog post
  • 42. Clean separation of concerns parsedData .withWatermark("timestamp", "10 minutes") .groupBy(window("timestamp","5 minutes")) .count() .writeStream .trigger("10 seconds") .start() Query Semantics Processing Details separated from
  • 43. Clean separation of concerns parsedData .withWatermark("timestamp", "10 minutes") .groupBy(window("timestamp","5 minutes")) .count() .writeStream .trigger("10 seconds") .start() Query Semantics How to group data by time? (same for batch & streaming) Processing Details
  • 44. Clean separation of concerns parsedData .withWatermark("timestamp", "10 minutes") .groupBy(window("timestamp","5 minutes")) .count() .writeStream .trigger("10 seconds") .start() Query Semantics How to group data by time? (same for batch & streaming) Processing Details How late can data be?
  • 45. Clean separation of concerns parsedData .withWatermark("timestamp", "10 minutes") .groupBy(window("timestamp","5 minutes")) .count() .writeStream .trigger("10 seconds") .start() Query Semantics How to group data by time? (same for batch & streaming) Processing Details How late can data be? How often to emit updates?
  • 46. Arbitrary Stateful Operations [Spark 2.2] mapGroupsWithState allows any user-defined stateful function to a user-defined state Direct support for per-key timeouts in event-time or processing-time Supports Scala and Java 46 ds.groupByKey(_.id) .mapGroupsWithState (timeoutConf) (mappingWithStateFunc) def mappingWithStateFunc( key: K, values: Iterator[V], state: GroupState[S]): U = { // update or remove state // set timeouts // return mapped value }
  • 47. Other interesting operations Streaming Deduplication Watermarks to limit state Stream-batch Joins Stream-stream Joins Can use mapGroupsWithState Direct support oming soon! val batchData = spark.read .format("parquet") .load("/additional-data") parsedData.join(batchData, "device") parsedData.dropDuplicates("eventId")
  • 49. Metric Processing @ Dashboards Analyze trends in usage as they occur Alerts Notify engineers of critical issues Ad-hoc Analysis Diagnose issues when they occur ETL Clean, normalize and store historical data Events generated by user actions (logins, clicks, spark job updates)
  • 50. Metric Processing @ Dashboards Alerts Ad-hoc Analysis ETL Difficult with only streaming frameworks Limited retention in streaming storage Inefficient for ad-hoc queries Hard for novice users (limited or no SQL support)
  • 52. Read from rawLogs = spark.readStream .format("kafka") .option("kafka.bootstrap.servers", ...) .option("subscribe", "rawLogs") .load() augmentedLogs = rawLogs .withColumn("msg", from_json($"value".cast("string"), schema)) .select("timestamp", "msg.*") .join(table("customers"), ["customer_id"]) DataFrames can be reused for multiple streams Can build libraries of useful DataFrames and share code between applications JSON ETL
  • 53. Write to augmented .repartition(1) .writeStream .format("parquet") .option("path", "/data/metrics") .trigger("1 minute") .start() Store augmented stream as efficient columnar data for later processing Latency: ~1 minute Buffer data and write one large file every minute for efficient reads ETL
  • 54. Dashboards logins = spark.readStream.parquet("/data/metrics") .where("metric = 'login'") .groupBy(window("timestamp", "1 minute")) .count() display(logins) // Visualize in Databricks notebooks Always up-to-date visualizations of important business trends Latency: ~1 minute to hours (configurable) Dashboards
  • 55. Filter and write to filteredLogs = augmentedLogs .where("eventType = 'clusterHeartbeat'") .selectExpr("to_json(struct("*")) as value") filteredLogs.writeStream .format("kafka") .option("kafka.bootstrap.servers", ...) .option("topic", "clusterHeartbeats") .start() Forward filtered and augmented events back to Kafka Latency: ~100ms average Filter to_json() to convert columns back into json string, and then save as different Kafka topic
  • 56. Simple Alerts sparkErrors .as[ClusterHeartBeat] .filter(_.load > 99) .writeStream .foreach(new PagerdutySink(credentials)) E.g. Alert when Spark cluster load > threshold Latency: ~100 ms Alerts Notify PagerDuty
  • 57. Complex Alerts sparkErrors .as[ClusterHeartBeat] .groupBy(_.id) .flatMapGroupsWithState(Update, ProcessingTimeTimeout("1 minute")) { (id: Int, events: Iterator[ClusterHeartBeat], state: GroupState[ClusterState]) => ... // check if cluster non-responsive for a while } E.g. Monitor health of Spark clusters using custom stateful logic Latency: ~10 seconds Alerts React if no heartbeat from cluster for 1 min
  • 58. Ad-hoc Analysis SELECT * FROM parquet.`/data/metrics` WHERE level IN ('WARN', 'ERROR') AND customer = "…" AND timestamp < now() – INTERVAL 1 HOUR Trouble shoot problems as they occur with latest information Latency: ~1 minute Ad-hoc Analysis will read latest data when query executed
  • 59. Metric Processing @ = Metrics Filter ETL Dashboards Ad-hoc Analysis Alerts14+ billion records / hour with 10 nodes meet diverse latency requirements as efficiently as possible
  • 60. More Info Structured Streaming Programming Guide http://spark.apache.org/docs/latest/structured-streaming-programming-guide.html Databricks blog posts for more focused discussions https://databricks.com/blog/2016/07/28/structured-streaming-in-apache-spark.html https://databricks.com/blog/2017/01/19/real-time-streaming-etl-structured-streaming-apache-spark-2-1.html https://databricks.com/blog/2017/02/23/working-complex-data-formats-structured-streaming-apache-spark-2-1.html https://databricks.com/blog/2017/04/26/processing-data-in-apache-kafka-with-structured-streaming-in-apache-spark-2-2.html https://databricks.com/blog/2017/05/08/event-time-aggregation-watermarking-apache-sparks-structured-streaming.html and more to come, stay tuned!!
  • 61. UNIFIED ANALYTICS PLATFORM Try Apache Spark in Databricks! • Collaborative cloud environment • Free version (community edition) DATABRICKS RUNTIME 3.0 • Apache Spark - optimized for the cloud • Caching and optimization layer - DBIO • Enterprise security - DBES Try for free today databricks.com