easy, scalable, fault tolerant stream processing with structured streaming - spark meetup at intel...

46
Easy, Scalable, Fault-tolerant stream processing with Structured Streaming Tathagata “TD” Das @tathadas Spark Meetup @ Intel Santa Clara, 23 rd March 2017

Upload: databricks

Post on 11-Apr-2017

294 views

Category:

Engineering


0 download

TRANSCRIPT

Page 1: Easy, scalable, fault tolerant stream processing with structured streaming - spark meetup at intel in santa clara

Easy, Scalable, Fault-tolerantstream processing withStructured StreamingTathagata “TD” Das

@tathadas

Spark Meetup @ IntelSanta Clara, 23rd March 2017

Page 2: Easy, scalable, fault tolerant stream processing with structured streaming - spark meetup at intel in santa clara

About Me

Spark PMC Member

Built Spark Streaming in UC Berkeley

Currently focused on building Structured Streaming

Software engineer at Databricks

2

Page 3: Easy, scalable, fault tolerant stream processing with structured streaming - spark meetup at intel in santa clara

building robust stream processing

apps is hard

3

Page 4: Easy, scalable, fault tolerant stream processing with structured streaming - spark meetup at intel in santa clara

Complexities in stream processing

4

Complex Data

Diverse data formats (json, avro, binary, …)

Data can be dirty, late, out-of-order

Complex Systems

Diverse storage systems and formats (SQL, NoSQL, parquet, ... )

System failures

Complex Workloads

Event time processing

Combining streaming with interactive queries,

machine learning

Page 5: Easy, scalable, fault tolerant stream processing with structured streaming - spark meetup at intel in santa clara

Structured Streaming

stream processing on Spark SQL enginefast, scalable, fault-tolerant

rich, unified, high level APIs deal with complex data and complex workloads

rich ecosystem of data sourcesintegrate with many storage systems

5

Page 6: Easy, scalable, fault tolerant stream processing with structured streaming - spark meetup at intel in santa clara

you should not have to

reason about streaming

6

Page 7: Easy, scalable, fault tolerant stream processing with structured streaming - spark meetup at intel in santa clara

you should only write

simple batch-like queries

Spark should automatically streamify them

7

Page 8: Easy, scalable, fault tolerant stream processing with structured streaming - spark meetup at intel in santa clara

Treat Streams as Unbounded Tables

8

data stream unbounded input table

new data in the data stream

= new rows appended to a unbounded table

Page 9: Easy, scalable, fault tolerant stream processing with structured streaming - spark meetup at intel in santa clara

New ModelTrigger: every 1 sec

Time

Input data upto t = 3

Que

ry

Input: data from source as an append-only table

Trigger: how frequently to checkinput for new data

Query: operations on inputusual map/filter/reduce new window, session ops

t=1 t=2 t=3

data upto t = 1

data upto t = 2

Page 10: Easy, scalable, fault tolerant stream processing with structured streaming - spark meetup at intel in santa clara

New Model

result up to t = 1

Result

Que

ry

Time

data upto t = 1

Input data upto t = 2

result up to t = 2

data upto t = 3

result up to t = 3

Result: final operated tableupdated after every trigger

Output: what part of result to write to storage after every trigger

Complete output: write full result table every time

Output[complete mode]

write all rows in result table to storage

t=1 t=2 t=3

Page 11: Easy, scalable, fault tolerant stream processing with structured streaming - spark meetup at intel in santa clara

New Modelt=1 t=2 t=3

Result

Que

ry

Time

Input data upto t = 3

result up to t = 3

Output[append mode] write new rows since last trigger to storage

Result: final operated tableupdated after every trigger

Output: what part of result to write to storage after every trigger

Complete output: write full result table every time

Append output: write only new rows that got added to result table since previous batch

*Not all output modes are feasible with all queries

data upto t = 1

data upto t = 2

result up to t = 1

result up to t = 2

Page 12: Easy, scalable, fault tolerant stream processing with structured streaming - spark meetup at intel in santa clara

New Modelt=1 t=2 t=3

Result

Que

ry

Time

Input data upto t = 3

result up to t = 3

Output[append mode] write new rows since last trigger to storage

Conceptual model that guides how to think of a streaming query as a simple table query

Engine does not need to keep the full input table in memory once it has streamified it

data upto t = 1

data upto t = 2

result up to t = 1

result up to t = 2

Page 13: Easy, scalable, fault tolerant stream processing with structured streaming - spark meetup at intel in santa clara

static data =bounded table

streaming data =unbounded table

API - Dataset/DataFrame

Single API !

Page 14: Easy, scalable, fault tolerant stream processing with structured streaming - spark meetup at intel in santa clara

Batch Queries with DataFrames

input = spark.read.format("json").load("source-path")

result = input.select("device", "signal").where("signal > 15")

result.write.format("parquet").save("dest-path")

Read from Json file

Select some devices

Write to parquet file

Page 15: Easy, scalable, fault tolerant stream processing with structured streaming - spark meetup at intel in santa clara

Streaming Queries with DataFrames

input = spark.readStream.format("json").load("source-path")

result = input.select("device", "signal").where("signal > 15")

result.writeStream.format("parquet").start("dest-path")

Read from Json file streamReplace read with readStream

Select some devicesCode does not change

Write to Parquet file streamReplace save() with start()

Page 16: Easy, scalable, fault tolerant stream processing with structured streaming - spark meetup at intel in santa clara

DataFrames,Datasets, SQL

input = spark.readStream.format("json").load("source-path")

result = input.select("device", "signal").where("signal > 15")

result.writeStream.format("parquet").start("dest-path")

Logical Plan

Streaming Source

Projectdevice, signal

Filtersignal > 15

Streaming Sink

Spark automatically streamifies!

Spark SQL converts batch-like query to a series of incremental execution plans operating on new batches of data

Series of IncrementalExecution Plans

proc

ess

new

file

s

t = 1 t = 2 t = 3

proc

ess

new

file

s

proc

ess

new

file

s

Page 17: Easy, scalable, fault tolerant stream processing with structured streaming - spark meetup at intel in santa clara

Fault-tolerance with Checkpointing

Checkpointing - metadata (e.g. offsets) of current batch stored in a write ahead log in HDFS/S3

Query can be restarted from the log

Streaming sources can replay the exact data range in case of failure

Streaming sinks can dedup reprocessed data when writing, idempotent by design

end-to-end exactly-once guarantees

proc

ess

new

file

s

t = 1 t = 2 t = 3

proc

ess

new

file

s

proc

ess

new

file

s

write ahead

log

Page 18: Easy, scalable, fault tolerant stream processing with structured streaming - spark meetup at intel in santa clara

ComplexStreaming ETL

18

Page 19: Easy, scalable, fault tolerant stream processing with structured streaming - spark meetup at intel in santa clara

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

19

filedump

seconds hourstable

10101010

Page 20: Easy, scalable, fault tolerant stream processing with structured streaming - spark meetup at intel in santa clara

Traditional ETL

Hours of delay before taking decisions on latest data

Unacceptable when time is of essence[intrusion detection, anomaly detection, etc.]

20

filedump

seconds hourstable

10101010

Page 21: Easy, scalable, fault tolerant stream processing with structured streaming - spark meetup at intel in santa clara

Streaming ETL w/ Structured Streaming

Structured Streaming enables raw data to be available as structured data as soon as possible

21

table

seconds10101010

Page 22: Easy, scalable, fault tolerant stream processing with structured streaming - spark meetup at intel in santa clara

Streaming ETL w/ Structured Streaming

22

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("subscribe", "topic") .option("kafka.boostrap.servers",...) .load()

val parsedData = rawData.selectExpr("cast (value as string) as json")) .select(from_json("json").as("data")) .select("data.*")

val query = parsedData.writeStream.option("checkpointLocation", "/checkpoint") .partitionBy("date") .format("parquet") .start("/parquetTable/")

Page 23: Easy, scalable, fault tolerant stream processing with structured streaming - spark meetup at intel in santa clara

Reading from Kafka [Spark 2.1]

23

Support Kafka 0.10.0.1Specify options to configure

How?kafka.boostrap.servers => broker1

What?subscribe => topic1,topic2,topic3 // fixed list of topicssubscribePattern => topic* // dynamic list of topicsassign => {"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()

Page 24: Easy, scalable, fault tolerant stream processing with structured streaming - spark meetup at intel in santa clara

Reading from Kafka

24

val rawData = spark.readStream.format("kafka").option("subscribe", "topic").option("kafka.boostrap.servers",...).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

Page 25: Easy, scalable, fault tolerant stream processing with structured streaming - spark meetup at intel in santa clara

Transforming Data

Cast binary value to stringName it column json

25

val parsedData = rawData.selectExpr("cast (value as string) as json").select(from_json("json").as("data")).select("data.*")

Page 26: Easy, scalable, fault tolerant stream processing with structured streaming - spark meetup at intel in santa clara

Transforming Data

Cast binary value to stringName it column json

Parse json string and expand into nested columns, name it data

26

val parsedData = rawData.selectExpr("cast (value as string) as json").select(from_json("json").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"

Page 27: Easy, scalable, fault tolerant stream processing with structured streaming - spark meetup at intel in santa clara

Transforming Data

Cast binary value to stringName it column json

Parse json string and expand into nested columns, name it data

Flatten the nested columns

27

val parsedData = rawData.selectExpr("cast (value as string) as json").select(from_json("json").as("data")).select("data.*")

data (nested)

timestamp device …

1486087873 devA …

1486086721 devX …

timestamp device …

1486087873 devA …

1486086721 devX …

select("data.*")

(not nested)

Page 28: Easy, scalable, fault tolerant stream processing with structured streaming - spark meetup at intel in santa clara

Transforming Data

Cast binary value to stringName it column json

Parse json string and expand into nested columns, name it data

Flatten the nested columns

28

val parsedData = rawData.selectExpr("cast (value as string) as json").select(from_json("json").as("data")).select("data.*")

powerful built-in APIs to perform complex data

transformationsfrom_json, to_json, explode, ...

100s of functions

(see our blog post)

Page 29: Easy, scalable, fault tolerant stream processing with structured streaming - spark meetup at intel in santa clara

Writing to Parquet table

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

29

val query = parsedData.writeStream.option("checkpointLocation", ...).partitionBy("date").format("parquet").start("/parquetTable")

Page 30: Easy, scalable, fault tolerant stream processing with structured streaming - spark meetup at intel in santa clara

Checkpointing

Enable checkpointing by setting the checkpoint location to save offset logs

start actually starts a continuous running StreamingQuery in the Spark cluster

30

val query = parsedData.writeStream.option("checkpointLocation", ...).format("parquet").partitionBy("date").start("/parquetTable/")

Page 31: Easy, scalable, fault tolerant stream processing with structured streaming - spark meetup at intel in santa clara

Streaming Query

query is a handle to the continuously running StreamingQuery

Used to monitor and manage the execution

31

val query = parsedData.writeStream.option("checkpointLocation", ...).format("parquet").partitionBy("date").start("/parquetTable/")

proc

ess

new

dat

a

t = 1 t = 2 t = 3

proc

ess

new

dat

a

proc

ess

new

dat

a

StreamingQuery

Page 32: Easy, scalable, fault tolerant stream processing with structured streaming - spark meetup at intel in santa clara

Data Consistency on Ad-hoc Queries

Data available for complex, ad-hoc analytics within seconds

Parquet table is updated atomically, ensures prefix integrityEven if distributed, ad-hoc queries will see either all updates from streaming query or none, read more in our bloghttps://databricks.com/blog/2016/07/28/structured-streaming-in-apache-spark.html

32

seconds!complex, ad-hoc

queries on latest data

Page 33: Easy, scalable, fault tolerant stream processing with structured streaming - spark meetup at intel in santa clara

Advanced Streaming Analytics

33

Page 34: Easy, scalable, fault tolerant stream processing with structured streaming - spark meetup at intel in santa clara

Event time Aggregations

Many use cases require aggregate statistics by event timeE.g. what's the #errors in each system in the 1 hour windows?

Many challengesExtracting event time from data, handling late, out-of-order data

DStream APIs were insufficient for event-time stuff

34

Page 35: Easy, scalable, fault tolerant stream processing with structured streaming - spark meetup at intel in santa clara

Event time Aggregations

Windowing is just another type of grouping in Struct. Streaming

number of records every hour

Support UDAFs!

35

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

Page 36: Easy, scalable, fault tolerant stream processing with structured streaming - spark meetup at intel in santa clara

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!

36

proc

ess

new

dat

a

t = 1

sink

src

t = 2

proc

ess

new

dat

a

sink

src

t = 3

proc

ess

new

dat

a

sink

src

state state

write ahead

log

state updates are written to log for checkpointing

state

Page 37: Easy, scalable, fault tolerant stream processing with structured streaming - spark meetup at intel in santa clara

Watermarking and Late Data

Watermark [Spark 2.1] - threshold on how late an event is expected to be in event time

Trails behind max seen event time

Trailing gap is configurable

37

event time

max event time

watermark data older than

watermark not expected

12:30 PM

12:20 PM

trailing gapof 10 mins

Page 38: Easy, scalable, fault tolerant stream processing with structured streaming - spark meetup at intel in santa clara

Watermarking and Late Data

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

38

max event time

event time

watermark

late dataallowed to aggregate

data too late,

dropped

Page 39: Easy, scalable, fault tolerant stream processing with structured streaming - spark meetup at intel in santa clara

Watermarking and Late Data

39

max event time

event time

watermark

allowed latenessof 10 mins

parsedData.withWatermark("timestamp", "10 minutes").groupBy(window("timestamp","5 minutes")).count()

late dataallowed to aggregate

data too late,

dropped

Page 40: Easy, scalable, fault tolerant stream processing with structured streaming - spark meetup at intel in santa clara

Watermarking to Limit State [Spark 2.1]

40

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

Even

t Tim

e12:15

12:18

12:04

watermark updated to 12:14 - 10m = 12:04for 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

10 m

in

12:14

more details in online programming guide

Page 41: Easy, scalable, fault tolerant stream processing with structured streaming - spark meetup at intel in santa clara

Arbitrary Stateful Operations [Spark 2.2]

mapGroupsWithStateallows any user-definedstateful ops to a user-defined state

supports timeouts

fault-tolerant, exactly-once

supports Scala and Java41

dataset.groupByKey(groupingFunc).mapGroupsWithState(mappingFunc)

def mappingFunc(key: K, values: Iterator[V], state: KeyedState[S]): U = {

// update or remove state// set timeouts// return mapped value

}

Page 42: Easy, scalable, fault tolerant stream processing with structured streaming - spark meetup at intel in santa clara

Many more updates!StreamingQueryListener [Spark 2.1]Receive of regular progress heartbeats for health and perf monitoringAutomatic in Databricks!!

Streaming Deduplication [Spark 2.2]Automatically eliminate duplicate data from Kafka/Kinesis/etc.

More Kafka Integration [Spark 2.2]Run batch queries on Kafka, and write to Kafka from batch/streaming queries

Kinesis SourceRead from Amazon Kinesis

42

Page 43: Easy, scalable, fault tolerant stream processing with structured streaming - spark meetup at intel in santa clara

Future Directions

Stability, stability, stabilityNeeded to remove the Experimental tag

More supported operationsStream-stream joins, …

Stable source and sink APIsConnect to your streams and stores

More sources and sinks JDBC, …

43

Page 44: Easy, scalable, fault tolerant stream processing with structured streaming - spark meetup at intel in santa clara

More InfoStructured Streaming Programming Guide

http://spark.apache.org/docs/latest/structured-streaming-programming-guide.html

Databricks blog posts for more focused discussionshttps://databricks.com/blog/2016/07/28/continuous-applications-evolving-streaming-in-apache-spark-2-0.html

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

and more to come, stay tuned!!

44

Page 45: Easy, scalable, fault tolerant stream processing with structured streaming - spark meetup at intel in santa clara

Comparison with Other Engines

45Read our blog to understand this table

Page 46: Easy, scalable, fault tolerant stream processing with structured streaming - spark meetup at intel in santa clara

SPARK SUMMIT 2017DATA SCIENCE AND ENGINEERING AT SCALE

JUNE 5 – 7 | MOSCONE CENTER | SAN FRANCISCO

ORGANIZED BY spark-summit.org/2017