SlideShare a Scribd company logo
1 of 32
Download to read offline
Solving sessionization
problem with Apache
Spark batch and
streaming processing
Bartosz Konieczny
@waitingforcode1
About me
Bartosz Konieczny
#dataEngineer #ApacheSparkEnthusiast #AWSuser
#waitingforcode.com #becomedataengineer.com
#@waitingforcode #github.com/bartosz25
#canalplus #Paris
2
3
Sessions
"user activity followed by a
closing action or a period of
inactivity"
4
5
© https://pixabay.com/users/maxmann-665103/ from https://pixabay.com
Batch architecture
6
data producer
sync consumer input logs
(DFS)
input logs
(streaming broker)
orchestrator
sessions
generator
<triggers>
previous
window raw
sessions
(DFS)
output
sessions
(DFS)
Streaming architecture
7
data producer
sessions
generator
output
sessions
(DFS)
metadata state
<uses>
checkpoint location
input logs
(streaming broker)
Batch
implementation
The code
val previousSessions = loadPreviousWindowSessions(sparkSession,
previousSessionsDir)
val sessionsInWindow = sparkSession.read.schema(Visit.Schema)
.json(inputDir)
val joinedData = previousSessions.join(sessionsInWindow,
sessionsInWindow("user_id") === previousSessions("userId"), "fullouter")
.groupByKey(log => SessionGeneration.resolveGroupByKey(log))
.flatMapGroups(SessionGeneration.generate(TimeUnit.MINUTES.toMillis(5),
windowUpperBound)).cache()
joinedData.filter("isActive = true").write.mode(SaveMode.Overwrite).json(outputDir)
joinedData.filter(state => !state.isActive)
.flatMap(state => state.toSessionOutputState)
.coalesce(50).write.mode(SaveMode.Overwrite)
.option("compression", "gzip")
.json(outputDir)
9
Full outer join
val previousSessions = loadPreviousWindowSessions(sparkSession,
previousSessionsDir)
val sessionsInWindow = sparkSession.read.schema(Visit.Schema)
.json(inputDir)
val joinedData = previousSessions.join(sessionsInWindow,
sessionsInWindow("user_id") === previousSessions("userId"), "fullouter")
.groupByKey(log => SessionGeneration.resolveGroupByKey(log))
.flatMapGroups(SessionGeneration.generate(TimeUnit.MINUTES.toMillis(5),
windowUpperBound))
joinedData.filter("isActive = true").write.mode(SaveMode.Overwrite).json(outputDir)
joinedData.filter(state => !state.isActive)
.flatMap(state => state.toSessionOutputState)
.coalesce(50).write.mode(SaveMode.Overwrite)
.option("compression", "gzip")
.json(outputDir)
10
processing logic
previous
window
active
sessions
new input
logs
full outer join
Watermark simulation
val previousSessions = loadPreviousWindowSessions(sparkSession,
previousSessionsDir)
val sessionsInWindow = sparkSession.read.schema(Visit.Schema)
.json(inputDir)
val joinedData = previousSessions.join(sessionsInWindow,
sessionsInWindow("user_id") === previousSessions("userId"), "fullouter")
.groupByKey(log => SessionGeneration.resolveGroupByKey(log))
.flatMapGroups(SessionGeneration.generate(TimeUnit.MINUTES.toMillis(5),
windowUpperBound))
joinedData.filter("isActive = true").write.mode(SaveMode.Overwrite).json(outputDir)
joinedData.filter(state => !state.isActive)
.flatMap(state => state.toSessionOutputState)
.coalesce(50).write.mode(SaveMode.Overwrite)
.option("compression", "gzip")
.json(outputDir)
case class SessionIntermediaryState(userId:
Long, … expirationTimeMillisUtc: Long,
isActive: Boolean)
11
Save modes
val previousSessions = loadPreviousWindowSessions(sparkSession,
previousSessionsDir)
val sessionsInWindow = sparkSession.read.schema(Visit.Schema)
.json(inputDir)
val joinedData = previousSessions.join(sessionsInWindow,
sessionsInWindow("user_id") === previousSessions("userId"), "fullouter")
.groupByKey(log => SessionGeneration.resolveGroupByKey(log))
.flatMapGroups(SessionGeneration.generate(TimeUnit.MINUTES.toMillis(5),
windowUpperBound))
joinedData.filter("isActive = true").write.mode(SaveMode.Overwrite).json(outputDir)
joinedData.filter(state => !state.isActive)
.flatMap(state => state.toSessionOutputState)
.coalesce(50).write.mode(SaveMode.Overwrite)
.option("compression", "gzip")
.json(outputDir)
SaveMode.Append ⇒
duplicates & invalid results
(e.g. multiplied revenue!)
SaveMode.ErrorIfExists ⇒
failures & maintenance
burden
SaveMode.Ignore ⇒ no
data & old data present in
case of reprocessing
SaveMode.Overwrite ⇒
always fresh data & easy
maintenance
12
Streaming
implementation
The code
val writeQuery = query.writeStream.outputMode(OutputMode.Update())
.option("checkpointLocation", s"s3://my-checkpoint-bucket")
.foreachBatch((dataset: Dataset[SessionIntermediaryState], batchId: Long) => {
BatchWriter.writeDataset(dataset, s"${outputDir}/${batchId}")
})
val dataFrame = sparkSession.readStream
.format("kafka")
.option("kafka.bootstrap.servers", kafkaConfiguration.broker).option(...) .load()
val query = dataFrame.selectExpr("CAST(value AS STRING)")
.select(functions.from_json($"value", Visit.Schema).as("data"))
.select($"data.*").withWatermark("event_time", "3 minutes")
.groupByKey(row => row.getAs[Long]("user_id"))
.mapGroupsWithState(GroupStateTimeout.EventTimeTimeout())
(mapStreamingLogsToSessions(sessionTimeout))
watermark - late events & state
expiration
stateful processing - sessions
generation
checkpoint - fault-tolerance
14
Checkpoint - fault-tolerance
load state
for t0
query
load offsets
to process &
write them
for t1
query
process data
write
processed
offsets
write state
checkpoint location
state store offset log commit log
val writeQuery = query.writeStream.outputMode(OutputMode.Update())
.option("checkpointLocation", s"s3://sessionization-demo/checkpoint")
.foreachBatch((dataset: Dataset[SessionIntermediaryState], batchId: Long) => {
BatchWriter.writeDataset(dataset, s"${outputDir}/${batchId}")
})
.start()
15
Checkpoint - fault-tolerance
load state
for t1
query
load offsets
to process &
write them
for t1
query
process data
confirm
processed
offsets &
next
watermark
commit state
t2
partition-based
checkpoint location
state store offset log commit log
16
Stateful processing
update
remove
get
getput,remove
write update
finalize file
make snapshot
recover state
def mapStreamingLogsToSessions(timeoutDurationMs: Long)(key: Long, logs: Iterator[Row],
currentState: GroupState[SessionIntermediaryState]): SessionIntermediaryState = {
if (currentState.hasTimedOut) {
val expiredState = currentState.get.expire
currentState.remove()
expiredState
} else {
val newState = currentState.getOption.map(state => state.updateWithNewLogs(logs, timeoutDurationMs))
.getOrElse(SessionIntermediaryState.createNew(logs, timeoutDurationMs))
currentState.update(newState)
currentState.setTimeoutTimestamp(currentState.getCurrentWatermarkMs() + timeoutDurationMs)
currentState.get
}
}
17
Stateful processing
update
remove
get
getput,remove
- write update
- finalize file
- make snapshot
recover state
18
.mapGroupsWithState(...)
state store
TreeMap[Long,
ConcurrentHashMap[UnsafeRow,
UnsafeRow]
]
in-memory storage for the most
recent versions
1.delta
2.delta
3.snapshot
checkpoint
location
Watermark
val sessionTimeout = TimeUnit.MINUTES.toMillis(5)
val query = dataFrame.selectExpr("CAST(value AS STRING)")
.select(functions.from_json($"value", Visit.Schema).as("data"))
.select($"data.*")
.withWatermark("event_time", "3 minutes")
.groupByKey(row => row.getAs[Long]("user_id"))
.mapGroupsWithState(GroupStateTimeout.EventTimeTimeout())
(Mapping.mapStreamingLogsToSessions(sessionTimeout))
19
Watermark - late events
on-time
event
late
event
20
.mapGroupsWithState(...)
Watermark - expired state
State representation [simplified]
{value, TTL configuration}
Algorithm:
1. Update all states with new data → eventually extend TTL
2. Retrieve TTL configuration for the query → here: watermark
3. Retrieve all states that expired → no new data in this query & TTL expired
4. Call mapGroupsWithState on it with hasTimedOut param = true & no new data
(Iterator.empty)
// full implementation: org.apache.spark.sql.execution.streaming.FlatMapGroupsWithStateExec.InputProcessor
21
Data reprocessing
Batch
reschedule your job
© https://pics.me.me/just-one-click-and-the-zoo-is-mine-8769663.png
Streaming
State store
1. Restored state is the most recent snapshot
2. Restored state is not the most recent snapshot but a snapshot exists
3. Restored state is not the most recent snapshot and a snapshot doesn't exist
27
1.delta 3.snapshot2.delta
1.delta 3.snapshot2.delta 4.delta
1.delta 3.delta2.delta 4.delta
State store configuration
spark.sql.streaming.stateStore:
→ .minDeltasForSnapshot
→ .maintenanceInterval
28
spark.sql.streaming:
→ .maxBatchesToRetainInMemory
Checkpoint configuration
spark.sql.streaming.minBatchesToRetain
29
Few takeaways
● yet another TDD acronym - Trade-Off Driven Development
○ simplicity for latency
○ simplicity for accuracy
○ scaling for latency
● AWS
○ Kinesis - short retention period = reprocessing boundary, connector
○ S3 - trade reliability for performance
○ EMR - transient cluster
○ Redshift - COPY
● Apache Spark
○ watermarks everywhere - batch simulation
○ state store configuration
○ restore mechanism
○ overwrite idempotent mode
30
Resources
● https://github.com/bartosz25/sessionization-de
mo
● https://www.waitingforcode.com/tags/spark-ai-s
ummit-europe-2019-articles
31
Thank you!Bartosz Konieczny
@waitingforcode / github.com/bartosz25 / waitingforcode.com
Canal+
@canaltechteam

More Related Content

What's hot

Exactly-Once Financial Data Processing at Scale with Flink and Pinot
Exactly-Once Financial Data Processing at Scale with Flink and PinotExactly-Once Financial Data Processing at Scale with Flink and Pinot
Exactly-Once Financial Data Processing at Scale with Flink and PinotFlink Forward
 
PostgreSQL Performance Tuning
PostgreSQL Performance TuningPostgreSQL Performance Tuning
PostgreSQL Performance Tuningelliando dias
 
Linux tuning to improve PostgreSQL performance
Linux tuning to improve PostgreSQL performanceLinux tuning to improve PostgreSQL performance
Linux tuning to improve PostgreSQL performancePostgreSQL-Consulting
 
Velocity 2017 Performance analysis superpowers with Linux eBPF
Velocity 2017 Performance analysis superpowers with Linux eBPFVelocity 2017 Performance analysis superpowers with Linux eBPF
Velocity 2017 Performance analysis superpowers with Linux eBPFBrendan Gregg
 
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
 
eBPF Trace from Kernel to Userspace
eBPF Trace from Kernel to UserspaceeBPF Trace from Kernel to Userspace
eBPF Trace from Kernel to UserspaceSUSE Labs Taipei
 
Linux Performance Analysis: New Tools and Old Secrets
Linux Performance Analysis: New Tools and Old SecretsLinux Performance Analysis: New Tools and Old Secrets
Linux Performance Analysis: New Tools and Old SecretsBrendan Gregg
 
MySQL Group Replication - HandsOn Tutorial
MySQL Group Replication - HandsOn TutorialMySQL Group Replication - HandsOn Tutorial
MySQL Group Replication - HandsOn TutorialKenny Gryp
 
카프카, 산전수전 노하우
카프카, 산전수전 노하우카프카, 산전수전 노하우
카프카, 산전수전 노하우if kakao
 
JVM Mechanics: When Does the JVM JIT & Deoptimize?
JVM Mechanics: When Does the JVM JIT & Deoptimize?JVM Mechanics: When Does the JVM JIT & Deoptimize?
JVM Mechanics: When Does the JVM JIT & Deoptimize?Doug Hawkins
 
How to Performance-Tune Apache Spark Applications in Large Clusters
How to Performance-Tune Apache Spark Applications in Large ClustersHow to Performance-Tune Apache Spark Applications in Large Clusters
How to Performance-Tune Apache Spark Applications in Large ClustersDatabricks
 
How to upgrade like a boss to MySQL 8.0 - PLE19
How to upgrade like a boss to MySQL 8.0 -  PLE19How to upgrade like a boss to MySQL 8.0 -  PLE19
How to upgrade like a boss to MySQL 8.0 - PLE19Alkin Tezuysal
 
MySQL Performance for DevOps
MySQL Performance for DevOpsMySQL Performance for DevOps
MySQL Performance for DevOpsSveta Smirnova
 
Wars of MySQL Cluster ( InnoDB Cluster VS Galera )
Wars of MySQL Cluster ( InnoDB Cluster VS Galera ) Wars of MySQL Cluster ( InnoDB Cluster VS Galera )
Wars of MySQL Cluster ( InnoDB Cluster VS Galera ) Mydbops
 
Apache Flink in the Cloud-Native Era
Apache Flink in the Cloud-Native EraApache Flink in the Cloud-Native Era
Apache Flink in the Cloud-Native EraFlink Forward
 
Cruise Control: Effortless management of Kafka clusters
Cruise Control: Effortless management of Kafka clustersCruise Control: Effortless management of Kafka clusters
Cruise Control: Effortless management of Kafka clustersPrateek Maheshwari
 
Supporting Apache HBase : Troubleshooting and Supportability Improvements
Supporting Apache HBase : Troubleshooting and Supportability ImprovementsSupporting Apache HBase : Troubleshooting and Supportability Improvements
Supporting Apache HBase : Troubleshooting and Supportability ImprovementsDataWorks Summit
 
Four Things to Know About Reliable Spark Streaming with Typesafe and Databricks
Four Things to Know About Reliable Spark Streaming with Typesafe and DatabricksFour Things to Know About Reliable Spark Streaming with Typesafe and Databricks
Four Things to Know About Reliable Spark Streaming with Typesafe and DatabricksLegacy Typesafe (now Lightbend)
 

What's hot (20)

Exactly-Once Financial Data Processing at Scale with Flink and Pinot
Exactly-Once Financial Data Processing at Scale with Flink and PinotExactly-Once Financial Data Processing at Scale with Flink and Pinot
Exactly-Once Financial Data Processing at Scale with Flink and Pinot
 
PostgreSQL Performance Tuning
PostgreSQL Performance TuningPostgreSQL Performance Tuning
PostgreSQL Performance Tuning
 
Linux tuning to improve PostgreSQL performance
Linux tuning to improve PostgreSQL performanceLinux tuning to improve PostgreSQL performance
Linux tuning to improve PostgreSQL performance
 
The basics of fluentd
The basics of fluentdThe basics of fluentd
The basics of fluentd
 
Velocity 2017 Performance analysis superpowers with Linux eBPF
Velocity 2017 Performance analysis superpowers with Linux eBPFVelocity 2017 Performance analysis superpowers with Linux eBPF
Velocity 2017 Performance analysis superpowers with Linux eBPF
 
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
 
eBPF Trace from Kernel to Userspace
eBPF Trace from Kernel to UserspaceeBPF Trace from Kernel to Userspace
eBPF Trace from Kernel to Userspace
 
Linux Performance Analysis: New Tools and Old Secrets
Linux Performance Analysis: New Tools and Old SecretsLinux Performance Analysis: New Tools and Old Secrets
Linux Performance Analysis: New Tools and Old Secrets
 
MySQL Group Replication - HandsOn Tutorial
MySQL Group Replication - HandsOn TutorialMySQL Group Replication - HandsOn Tutorial
MySQL Group Replication - HandsOn Tutorial
 
카프카, 산전수전 노하우
카프카, 산전수전 노하우카프카, 산전수전 노하우
카프카, 산전수전 노하우
 
JVM Mechanics: When Does the JVM JIT & Deoptimize?
JVM Mechanics: When Does the JVM JIT & Deoptimize?JVM Mechanics: When Does the JVM JIT & Deoptimize?
JVM Mechanics: When Does the JVM JIT & Deoptimize?
 
How to Performance-Tune Apache Spark Applications in Large Clusters
How to Performance-Tune Apache Spark Applications in Large ClustersHow to Performance-Tune Apache Spark Applications in Large Clusters
How to Performance-Tune Apache Spark Applications in Large Clusters
 
How to upgrade like a boss to MySQL 8.0 - PLE19
How to upgrade like a boss to MySQL 8.0 -  PLE19How to upgrade like a boss to MySQL 8.0 -  PLE19
How to upgrade like a boss to MySQL 8.0 - PLE19
 
MySQL Performance for DevOps
MySQL Performance for DevOpsMySQL Performance for DevOps
MySQL Performance for DevOps
 
Wars of MySQL Cluster ( InnoDB Cluster VS Galera )
Wars of MySQL Cluster ( InnoDB Cluster VS Galera ) Wars of MySQL Cluster ( InnoDB Cluster VS Galera )
Wars of MySQL Cluster ( InnoDB Cluster VS Galera )
 
Apache Flink in the Cloud-Native Era
Apache Flink in the Cloud-Native EraApache Flink in the Cloud-Native Era
Apache Flink in the Cloud-Native Era
 
MyRocks Deep Dive
MyRocks Deep DiveMyRocks Deep Dive
MyRocks Deep Dive
 
Cruise Control: Effortless management of Kafka clusters
Cruise Control: Effortless management of Kafka clustersCruise Control: Effortless management of Kafka clusters
Cruise Control: Effortless management of Kafka clusters
 
Supporting Apache HBase : Troubleshooting and Supportability Improvements
Supporting Apache HBase : Troubleshooting and Supportability ImprovementsSupporting Apache HBase : Troubleshooting and Supportability Improvements
Supporting Apache HBase : Troubleshooting and Supportability Improvements
 
Four Things to Know About Reliable Spark Streaming with Typesafe and Databricks
Four Things to Know About Reliable Spark Streaming with Typesafe and DatabricksFour Things to Know About Reliable Spark Streaming with Typesafe and Databricks
Four Things to Know About Reliable Spark Streaming with Typesafe and Databricks
 

Similar to Using Apache Spark to Solve Sessionization Problem in Batch and Streaming

Flink 0.10 @ Bay Area Meetup (October 2015)
Flink 0.10 @ Bay Area Meetup (October 2015)Flink 0.10 @ Bay Area Meetup (October 2015)
Flink 0.10 @ Bay Area Meetup (October 2015)Stephan Ewen
 
[JEEConf-2017] RxJava as a key component in mature Big Data product
[JEEConf-2017] RxJava as a key component in mature Big Data product[JEEConf-2017] RxJava as a key component in mature Big Data product
[JEEConf-2017] RxJava as a key component in mature Big Data productIgor Lozynskyi
 
Kick your database_to_the_curb_reston_08_27_19
Kick your database_to_the_curb_reston_08_27_19Kick your database_to_the_curb_reston_08_27_19
Kick your database_to_the_curb_reston_08_27_19confluent
 
Online Meetup: Why should container system / platform builders care about con...
Online Meetup: Why should container system / platform builders care about con...Online Meetup: Why should container system / platform builders care about con...
Online Meetup: Why should container system / platform builders care about con...Docker, Inc.
 
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
 
GDG Jakarta Meetup - Streaming Analytics With Apache Beam
GDG Jakarta Meetup - Streaming Analytics With Apache BeamGDG Jakarta Meetup - Streaming Analytics With Apache Beam
GDG Jakarta Meetup - Streaming Analytics With Apache BeamImre Nagi
 
Spark streaming with kafka
Spark streaming with kafkaSpark streaming with kafka
Spark streaming with kafkaDori Waldman
 
Spark stream - Kafka
Spark stream - Kafka Spark stream - Kafka
Spark stream - Kafka Dori Waldman
 
A new kind of BPM with Activiti
A new kind of BPM with ActivitiA new kind of BPM with Activiti
A new kind of BPM with ActivitiAlfresco Software
 
Getting Started with Real-Time Analytics
Getting Started with Real-Time AnalyticsGetting Started with Real-Time Analytics
Getting Started with Real-Time AnalyticsAmazon Web Services
 
Reactive programming every day
Reactive programming every dayReactive programming every day
Reactive programming every dayVadym Khondar
 
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
 
Continuous Application with Structured Streaming 2.0
Continuous Application with Structured Streaming 2.0Continuous Application with Structured Streaming 2.0
Continuous Application with Structured Streaming 2.0Anyscale
 
こわくないよ❤️ Playframeworkソースコードリーディング入門
こわくないよ❤️ Playframeworkソースコードリーディング入門こわくないよ❤️ Playframeworkソースコードリーディング入門
こわくないよ❤️ Playframeworkソースコードリーディング入門tanacasino
 
Introducing the WSO2 Complex Event Processor
Introducing the WSO2 Complex Event ProcessorIntroducing the WSO2 Complex Event Processor
Introducing the WSO2 Complex Event ProcessorWSO2
 
Scalable Angular 2 Application Architecture
Scalable Angular 2 Application ArchitectureScalable Angular 2 Application Architecture
Scalable Angular 2 Application ArchitectureFDConf
 
VPN Access Runbook
VPN Access RunbookVPN Access Runbook
VPN Access RunbookTaha Shakeel
 
Event Sourcing - what could go wrong - Devoxx BE
Event Sourcing - what could go wrong - Devoxx BEEvent Sourcing - what could go wrong - Devoxx BE
Event Sourcing - what could go wrong - Devoxx BEAndrzej Ludwikowski
 

Similar to Using Apache Spark to Solve Sessionization Problem in Batch and Streaming (20)

Flink 0.10 @ Bay Area Meetup (October 2015)
Flink 0.10 @ Bay Area Meetup (October 2015)Flink 0.10 @ Bay Area Meetup (October 2015)
Flink 0.10 @ Bay Area Meetup (October 2015)
 
[JEEConf-2017] RxJava as a key component in mature Big Data product
[JEEConf-2017] RxJava as a key component in mature Big Data product[JEEConf-2017] RxJava as a key component in mature Big Data product
[JEEConf-2017] RxJava as a key component in mature Big Data product
 
Kick your database_to_the_curb_reston_08_27_19
Kick your database_to_the_curb_reston_08_27_19Kick your database_to_the_curb_reston_08_27_19
Kick your database_to_the_curb_reston_08_27_19
 
Online Meetup: Why should container system / platform builders care about con...
Online Meetup: Why should container system / platform builders care about con...Online Meetup: Why should container system / platform builders care about con...
Online Meetup: Why should container system / platform builders care about con...
 
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...
 
Meetup spark structured streaming
Meetup spark structured streamingMeetup spark structured streaming
Meetup spark structured streaming
 
GDG Jakarta Meetup - Streaming Analytics With Apache Beam
GDG Jakarta Meetup - Streaming Analytics With Apache BeamGDG Jakarta Meetup - Streaming Analytics With Apache Beam
GDG Jakarta Meetup - Streaming Analytics With Apache Beam
 
Spark streaming with kafka
Spark streaming with kafkaSpark streaming with kafka
Spark streaming with kafka
 
Spark stream - Kafka
Spark stream - Kafka Spark stream - Kafka
Spark stream - Kafka
 
A new kind of BPM with Activiti
A new kind of BPM with ActivitiA new kind of BPM with Activiti
A new kind of BPM with Activiti
 
Getting Started with Real-Time Analytics
Getting Started with Real-Time AnalyticsGetting Started with Real-Time Analytics
Getting Started with Real-Time Analytics
 
Reactive programming every day
Reactive programming every dayReactive programming every day
Reactive programming every day
 
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...
 
Continuous Application with Structured Streaming 2.0
Continuous Application with Structured Streaming 2.0Continuous Application with Structured Streaming 2.0
Continuous Application with Structured Streaming 2.0
 
JavaZone 2014 - goto java;
JavaZone 2014 - goto java;JavaZone 2014 - goto java;
JavaZone 2014 - goto java;
 
こわくないよ❤️ Playframeworkソースコードリーディング入門
こわくないよ❤️ Playframeworkソースコードリーディング入門こわくないよ❤️ Playframeworkソースコードリーディング入門
こわくないよ❤️ Playframeworkソースコードリーディング入門
 
Introducing the WSO2 Complex Event Processor
Introducing the WSO2 Complex Event ProcessorIntroducing the WSO2 Complex Event Processor
Introducing the WSO2 Complex Event Processor
 
Scalable Angular 2 Application Architecture
Scalable Angular 2 Application ArchitectureScalable Angular 2 Application Architecture
Scalable Angular 2 Application Architecture
 
VPN Access Runbook
VPN Access RunbookVPN Access Runbook
VPN Access Runbook
 
Event Sourcing - what could go wrong - Devoxx BE
Event Sourcing - what could go wrong - Devoxx BEEvent Sourcing - what could go wrong - Devoxx BE
Event Sourcing - what could go wrong - Devoxx BE
 

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

VIP Model Call Girls Hinjewadi ( Pune ) Call ON 8005736733 Starting From 5K t...
VIP Model Call Girls Hinjewadi ( Pune ) Call ON 8005736733 Starting From 5K t...VIP Model Call Girls Hinjewadi ( Pune ) Call ON 8005736733 Starting From 5K t...
VIP Model Call Girls Hinjewadi ( Pune ) Call ON 8005736733 Starting From 5K t...SUHANI PANDEY
 
Call Girls In Hsr Layout ☎ 7737669865 🥵 Book Your One night Stand
Call Girls In Hsr Layout ☎ 7737669865 🥵 Book Your One night StandCall Girls In Hsr Layout ☎ 7737669865 🥵 Book Your One night Stand
Call Girls In Hsr Layout ☎ 7737669865 🥵 Book Your One night Standamitlee9823
 
April 2024 - Crypto Market Report's Analysis
April 2024 - Crypto Market Report's AnalysisApril 2024 - Crypto Market Report's Analysis
April 2024 - Crypto Market Report's Analysismanisha194592
 
Thane Call Girls 7091864438 Call Girls in Thane Escort service book now -
Thane Call Girls 7091864438 Call Girls in Thane Escort service book now -Thane Call Girls 7091864438 Call Girls in Thane Escort service book now -
Thane Call Girls 7091864438 Call Girls in Thane Escort service book now -Pooja Nehwal
 
Call Girls Hsr Layout Just Call 👗 7737669865 👗 Top Class Call Girl Service Ba...
Call Girls Hsr Layout Just Call 👗 7737669865 👗 Top Class Call Girl Service Ba...Call Girls Hsr Layout Just Call 👗 7737669865 👗 Top Class Call Girl Service Ba...
Call Girls Hsr Layout Just Call 👗 7737669865 👗 Top Class Call Girl Service Ba...amitlee9823
 
Jual Obat Aborsi Surabaya ( Asli No.1 ) 085657271886 Obat Penggugur Kandungan...
Jual Obat Aborsi Surabaya ( Asli No.1 ) 085657271886 Obat Penggugur Kandungan...Jual Obat Aborsi Surabaya ( Asli No.1 ) 085657271886 Obat Penggugur Kandungan...
Jual Obat Aborsi Surabaya ( Asli No.1 ) 085657271886 Obat Penggugur Kandungan...ZurliaSoop
 
Call Girls In Bellandur ☎ 7737669865 🥵 Book Your One night Stand
Call Girls In Bellandur ☎ 7737669865 🥵 Book Your One night StandCall Girls In Bellandur ☎ 7737669865 🥵 Book Your One night Stand
Call Girls In Bellandur ☎ 7737669865 🥵 Book Your One night Standamitlee9823
 
Call Girls Begur Just Call 👗 7737669865 👗 Top Class Call Girl Service Bangalore
Call Girls Begur Just Call 👗 7737669865 👗 Top Class Call Girl Service BangaloreCall Girls Begur Just Call 👗 7737669865 👗 Top Class Call Girl Service Bangalore
Call Girls Begur Just Call 👗 7737669865 👗 Top Class Call Girl Service Bangaloreamitlee9823
 
Discover Why Less is More in B2B Research
Discover Why Less is More in B2B ResearchDiscover Why Less is More in B2B Research
Discover Why Less is More in B2B Researchmichael115558
 
Call Girls Bannerghatta Road Just Call 👗 7737669865 👗 Top Class Call Girl Ser...
Call Girls Bannerghatta Road Just Call 👗 7737669865 👗 Top Class Call Girl Ser...Call Girls Bannerghatta Road Just Call 👗 7737669865 👗 Top Class Call Girl Ser...
Call Girls Bannerghatta Road Just Call 👗 7737669865 👗 Top Class Call Girl Ser...amitlee9823
 
Call Girls In Doddaballapur Road ☎ 7737669865 🥵 Book Your One night Stand
Call Girls In Doddaballapur Road ☎ 7737669865 🥵 Book Your One night StandCall Girls In Doddaballapur Road ☎ 7737669865 🥵 Book Your One night Stand
Call Girls In Doddaballapur Road ☎ 7737669865 🥵 Book Your One night Standamitlee9823
 
Probability Grade 10 Third Quarter Lessons
Probability Grade 10 Third Quarter LessonsProbability Grade 10 Third Quarter Lessons
Probability Grade 10 Third Quarter LessonsJoseMangaJr1
 
Cheap Rate Call girls Sarita Vihar Delhi 9205541914 shot 1500 night
Cheap Rate Call girls Sarita Vihar Delhi 9205541914 shot 1500 nightCheap Rate Call girls Sarita Vihar Delhi 9205541914 shot 1500 night
Cheap Rate Call girls Sarita Vihar Delhi 9205541914 shot 1500 nightDelhi Call girls
 
Week-01-2.ppt BBB human Computer interaction
Week-01-2.ppt BBB human Computer interactionWeek-01-2.ppt BBB human Computer interaction
Week-01-2.ppt BBB human Computer interactionfulawalesam
 
Call Girls Jalahalli Just Call 👗 7737669865 👗 Top Class Call Girl Service Ban...
Call Girls Jalahalli Just Call 👗 7737669865 👗 Top Class Call Girl Service Ban...Call Girls Jalahalli Just Call 👗 7737669865 👗 Top Class Call Girl Service Ban...
Call Girls Jalahalli Just Call 👗 7737669865 👗 Top Class Call Girl Service Ban...amitlee9823
 
Escorts Service Kumaraswamy Layout ☎ 7737669865☎ Book Your One night Stand (B...
Escorts Service Kumaraswamy Layout ☎ 7737669865☎ Book Your One night Stand (B...Escorts Service Kumaraswamy Layout ☎ 7737669865☎ Book Your One night Stand (B...
Escorts Service Kumaraswamy Layout ☎ 7737669865☎ Book Your One night Stand (B...amitlee9823
 

Recently uploaded (20)

VIP Model Call Girls Hinjewadi ( Pune ) Call ON 8005736733 Starting From 5K t...
VIP Model Call Girls Hinjewadi ( Pune ) Call ON 8005736733 Starting From 5K t...VIP Model Call Girls Hinjewadi ( Pune ) Call ON 8005736733 Starting From 5K t...
VIP Model Call Girls Hinjewadi ( Pune ) Call ON 8005736733 Starting From 5K t...
 
Call Girls In Hsr Layout ☎ 7737669865 🥵 Book Your One night Stand
Call Girls In Hsr Layout ☎ 7737669865 🥵 Book Your One night StandCall Girls In Hsr Layout ☎ 7737669865 🥵 Book Your One night Stand
Call Girls In Hsr Layout ☎ 7737669865 🥵 Book Your One night Stand
 
April 2024 - Crypto Market Report's Analysis
April 2024 - Crypto Market Report's AnalysisApril 2024 - Crypto Market Report's Analysis
April 2024 - Crypto Market Report's Analysis
 
Thane Call Girls 7091864438 Call Girls in Thane Escort service book now -
Thane Call Girls 7091864438 Call Girls in Thane Escort service book now -Thane Call Girls 7091864438 Call Girls in Thane Escort service book now -
Thane Call Girls 7091864438 Call Girls in Thane Escort service book now -
 
Call Girls Hsr Layout Just Call 👗 7737669865 👗 Top Class Call Girl Service Ba...
Call Girls Hsr Layout Just Call 👗 7737669865 👗 Top Class Call Girl Service Ba...Call Girls Hsr Layout Just Call 👗 7737669865 👗 Top Class Call Girl Service Ba...
Call Girls Hsr Layout Just Call 👗 7737669865 👗 Top Class Call Girl Service Ba...
 
Jual Obat Aborsi Surabaya ( Asli No.1 ) 085657271886 Obat Penggugur Kandungan...
Jual Obat Aborsi Surabaya ( Asli No.1 ) 085657271886 Obat Penggugur Kandungan...Jual Obat Aborsi Surabaya ( Asli No.1 ) 085657271886 Obat Penggugur Kandungan...
Jual Obat Aborsi Surabaya ( Asli No.1 ) 085657271886 Obat Penggugur Kandungan...
 
Call Girls In Bellandur ☎ 7737669865 🥵 Book Your One night Stand
Call Girls In Bellandur ☎ 7737669865 🥵 Book Your One night StandCall Girls In Bellandur ☎ 7737669865 🥵 Book Your One night Stand
Call Girls In Bellandur ☎ 7737669865 🥵 Book Your One night Stand
 
Predicting Loan Approval: A Data Science Project
Predicting Loan Approval: A Data Science ProjectPredicting Loan Approval: A Data Science Project
Predicting Loan Approval: A Data Science Project
 
CHEAP Call Girls in Saket (-DELHI )🔝 9953056974🔝(=)/CALL GIRLS SERVICE
CHEAP Call Girls in Saket (-DELHI )🔝 9953056974🔝(=)/CALL GIRLS SERVICECHEAP Call Girls in Saket (-DELHI )🔝 9953056974🔝(=)/CALL GIRLS SERVICE
CHEAP Call Girls in Saket (-DELHI )🔝 9953056974🔝(=)/CALL GIRLS SERVICE
 
Abortion pills in Doha Qatar (+966572737505 ! Get Cytotec
Abortion pills in Doha Qatar (+966572737505 ! Get CytotecAbortion pills in Doha Qatar (+966572737505 ! Get Cytotec
Abortion pills in Doha Qatar (+966572737505 ! Get Cytotec
 
Call Girls Begur Just Call 👗 7737669865 👗 Top Class Call Girl Service Bangalore
Call Girls Begur Just Call 👗 7737669865 👗 Top Class Call Girl Service BangaloreCall Girls Begur Just Call 👗 7737669865 👗 Top Class Call Girl Service Bangalore
Call Girls Begur Just Call 👗 7737669865 👗 Top Class Call Girl Service Bangalore
 
Discover Why Less is More in B2B Research
Discover Why Less is More in B2B ResearchDiscover Why Less is More in B2B Research
Discover Why Less is More in B2B Research
 
Anomaly detection and data imputation within time series
Anomaly detection and data imputation within time seriesAnomaly detection and data imputation within time series
Anomaly detection and data imputation within time series
 
Call Girls Bannerghatta Road Just Call 👗 7737669865 👗 Top Class Call Girl Ser...
Call Girls Bannerghatta Road Just Call 👗 7737669865 👗 Top Class Call Girl Ser...Call Girls Bannerghatta Road Just Call 👗 7737669865 👗 Top Class Call Girl Ser...
Call Girls Bannerghatta Road Just Call 👗 7737669865 👗 Top Class Call Girl Ser...
 
Call Girls In Doddaballapur Road ☎ 7737669865 🥵 Book Your One night Stand
Call Girls In Doddaballapur Road ☎ 7737669865 🥵 Book Your One night StandCall Girls In Doddaballapur Road ☎ 7737669865 🥵 Book Your One night Stand
Call Girls In Doddaballapur Road ☎ 7737669865 🥵 Book Your One night Stand
 
Probability Grade 10 Third Quarter Lessons
Probability Grade 10 Third Quarter LessonsProbability Grade 10 Third Quarter Lessons
Probability Grade 10 Third Quarter Lessons
 
Cheap Rate Call girls Sarita Vihar Delhi 9205541914 shot 1500 night
Cheap Rate Call girls Sarita Vihar Delhi 9205541914 shot 1500 nightCheap Rate Call girls Sarita Vihar Delhi 9205541914 shot 1500 night
Cheap Rate Call girls Sarita Vihar Delhi 9205541914 shot 1500 night
 
Week-01-2.ppt BBB human Computer interaction
Week-01-2.ppt BBB human Computer interactionWeek-01-2.ppt BBB human Computer interaction
Week-01-2.ppt BBB human Computer interaction
 
Call Girls Jalahalli Just Call 👗 7737669865 👗 Top Class Call Girl Service Ban...
Call Girls Jalahalli Just Call 👗 7737669865 👗 Top Class Call Girl Service Ban...Call Girls Jalahalli Just Call 👗 7737669865 👗 Top Class Call Girl Service Ban...
Call Girls Jalahalli Just Call 👗 7737669865 👗 Top Class Call Girl Service Ban...
 
Escorts Service Kumaraswamy Layout ☎ 7737669865☎ Book Your One night Stand (B...
Escorts Service Kumaraswamy Layout ☎ 7737669865☎ Book Your One night Stand (B...Escorts Service Kumaraswamy Layout ☎ 7737669865☎ Book Your One night Stand (B...
Escorts Service Kumaraswamy Layout ☎ 7737669865☎ Book Your One night Stand (B...
 

Using Apache Spark to Solve Sessionization Problem in Batch and Streaming

  • 1. Solving sessionization problem with Apache Spark batch and streaming processing Bartosz Konieczny @waitingforcode1
  • 2. About me Bartosz Konieczny #dataEngineer #ApacheSparkEnthusiast #AWSuser #waitingforcode.com #becomedataengineer.com #@waitingforcode #github.com/bartosz25 #canalplus #Paris 2
  • 3. 3
  • 4. Sessions "user activity followed by a closing action or a period of inactivity" 4
  • 6. Batch architecture 6 data producer sync consumer input logs (DFS) input logs (streaming broker) orchestrator sessions generator <triggers> previous window raw sessions (DFS) output sessions (DFS)
  • 7. Streaming architecture 7 data producer sessions generator output sessions (DFS) metadata state <uses> checkpoint location input logs (streaming broker)
  • 9. The code val previousSessions = loadPreviousWindowSessions(sparkSession, previousSessionsDir) val sessionsInWindow = sparkSession.read.schema(Visit.Schema) .json(inputDir) val joinedData = previousSessions.join(sessionsInWindow, sessionsInWindow("user_id") === previousSessions("userId"), "fullouter") .groupByKey(log => SessionGeneration.resolveGroupByKey(log)) .flatMapGroups(SessionGeneration.generate(TimeUnit.MINUTES.toMillis(5), windowUpperBound)).cache() joinedData.filter("isActive = true").write.mode(SaveMode.Overwrite).json(outputDir) joinedData.filter(state => !state.isActive) .flatMap(state => state.toSessionOutputState) .coalesce(50).write.mode(SaveMode.Overwrite) .option("compression", "gzip") .json(outputDir) 9
  • 10. Full outer join val previousSessions = loadPreviousWindowSessions(sparkSession, previousSessionsDir) val sessionsInWindow = sparkSession.read.schema(Visit.Schema) .json(inputDir) val joinedData = previousSessions.join(sessionsInWindow, sessionsInWindow("user_id") === previousSessions("userId"), "fullouter") .groupByKey(log => SessionGeneration.resolveGroupByKey(log)) .flatMapGroups(SessionGeneration.generate(TimeUnit.MINUTES.toMillis(5), windowUpperBound)) joinedData.filter("isActive = true").write.mode(SaveMode.Overwrite).json(outputDir) joinedData.filter(state => !state.isActive) .flatMap(state => state.toSessionOutputState) .coalesce(50).write.mode(SaveMode.Overwrite) .option("compression", "gzip") .json(outputDir) 10 processing logic previous window active sessions new input logs full outer join
  • 11. Watermark simulation val previousSessions = loadPreviousWindowSessions(sparkSession, previousSessionsDir) val sessionsInWindow = sparkSession.read.schema(Visit.Schema) .json(inputDir) val joinedData = previousSessions.join(sessionsInWindow, sessionsInWindow("user_id") === previousSessions("userId"), "fullouter") .groupByKey(log => SessionGeneration.resolveGroupByKey(log)) .flatMapGroups(SessionGeneration.generate(TimeUnit.MINUTES.toMillis(5), windowUpperBound)) joinedData.filter("isActive = true").write.mode(SaveMode.Overwrite).json(outputDir) joinedData.filter(state => !state.isActive) .flatMap(state => state.toSessionOutputState) .coalesce(50).write.mode(SaveMode.Overwrite) .option("compression", "gzip") .json(outputDir) case class SessionIntermediaryState(userId: Long, … expirationTimeMillisUtc: Long, isActive: Boolean) 11
  • 12. Save modes val previousSessions = loadPreviousWindowSessions(sparkSession, previousSessionsDir) val sessionsInWindow = sparkSession.read.schema(Visit.Schema) .json(inputDir) val joinedData = previousSessions.join(sessionsInWindow, sessionsInWindow("user_id") === previousSessions("userId"), "fullouter") .groupByKey(log => SessionGeneration.resolveGroupByKey(log)) .flatMapGroups(SessionGeneration.generate(TimeUnit.MINUTES.toMillis(5), windowUpperBound)) joinedData.filter("isActive = true").write.mode(SaveMode.Overwrite).json(outputDir) joinedData.filter(state => !state.isActive) .flatMap(state => state.toSessionOutputState) .coalesce(50).write.mode(SaveMode.Overwrite) .option("compression", "gzip") .json(outputDir) SaveMode.Append ⇒ duplicates & invalid results (e.g. multiplied revenue!) SaveMode.ErrorIfExists ⇒ failures & maintenance burden SaveMode.Ignore ⇒ no data & old data present in case of reprocessing SaveMode.Overwrite ⇒ always fresh data & easy maintenance 12
  • 14. The code val writeQuery = query.writeStream.outputMode(OutputMode.Update()) .option("checkpointLocation", s"s3://my-checkpoint-bucket") .foreachBatch((dataset: Dataset[SessionIntermediaryState], batchId: Long) => { BatchWriter.writeDataset(dataset, s"${outputDir}/${batchId}") }) val dataFrame = sparkSession.readStream .format("kafka") .option("kafka.bootstrap.servers", kafkaConfiguration.broker).option(...) .load() val query = dataFrame.selectExpr("CAST(value AS STRING)") .select(functions.from_json($"value", Visit.Schema).as("data")) .select($"data.*").withWatermark("event_time", "3 minutes") .groupByKey(row => row.getAs[Long]("user_id")) .mapGroupsWithState(GroupStateTimeout.EventTimeTimeout()) (mapStreamingLogsToSessions(sessionTimeout)) watermark - late events & state expiration stateful processing - sessions generation checkpoint - fault-tolerance 14
  • 15. Checkpoint - fault-tolerance load state for t0 query load offsets to process & write them for t1 query process data write processed offsets write state checkpoint location state store offset log commit log val writeQuery = query.writeStream.outputMode(OutputMode.Update()) .option("checkpointLocation", s"s3://sessionization-demo/checkpoint") .foreachBatch((dataset: Dataset[SessionIntermediaryState], batchId: Long) => { BatchWriter.writeDataset(dataset, s"${outputDir}/${batchId}") }) .start() 15
  • 16. Checkpoint - fault-tolerance load state for t1 query load offsets to process & write them for t1 query process data confirm processed offsets & next watermark commit state t2 partition-based checkpoint location state store offset log commit log 16
  • 17. Stateful processing update remove get getput,remove write update finalize file make snapshot recover state def mapStreamingLogsToSessions(timeoutDurationMs: Long)(key: Long, logs: Iterator[Row], currentState: GroupState[SessionIntermediaryState]): SessionIntermediaryState = { if (currentState.hasTimedOut) { val expiredState = currentState.get.expire currentState.remove() expiredState } else { val newState = currentState.getOption.map(state => state.updateWithNewLogs(logs, timeoutDurationMs)) .getOrElse(SessionIntermediaryState.createNew(logs, timeoutDurationMs)) currentState.update(newState) currentState.setTimeoutTimestamp(currentState.getCurrentWatermarkMs() + timeoutDurationMs) currentState.get } } 17
  • 18. Stateful processing update remove get getput,remove - write update - finalize file - make snapshot recover state 18 .mapGroupsWithState(...) state store TreeMap[Long, ConcurrentHashMap[UnsafeRow, UnsafeRow] ] in-memory storage for the most recent versions 1.delta 2.delta 3.snapshot checkpoint location
  • 19. Watermark val sessionTimeout = TimeUnit.MINUTES.toMillis(5) val query = dataFrame.selectExpr("CAST(value AS STRING)") .select(functions.from_json($"value", Visit.Schema).as("data")) .select($"data.*") .withWatermark("event_time", "3 minutes") .groupByKey(row => row.getAs[Long]("user_id")) .mapGroupsWithState(GroupStateTimeout.EventTimeTimeout()) (Mapping.mapStreamingLogsToSessions(sessionTimeout)) 19
  • 20. Watermark - late events on-time event late event 20 .mapGroupsWithState(...)
  • 21. Watermark - expired state State representation [simplified] {value, TTL configuration} Algorithm: 1. Update all states with new data → eventually extend TTL 2. Retrieve TTL configuration for the query → here: watermark 3. Retrieve all states that expired → no new data in this query & TTL expired 4. Call mapGroupsWithState on it with hasTimedOut param = true & no new data (Iterator.empty) // full implementation: org.apache.spark.sql.execution.streaming.FlatMapGroupsWithStateExec.InputProcessor 21
  • 23. Batch
  • 24. reschedule your job © https://pics.me.me/just-one-click-and-the-zoo-is-mine-8769663.png
  • 26.
  • 27. State store 1. Restored state is the most recent snapshot 2. Restored state is not the most recent snapshot but a snapshot exists 3. Restored state is not the most recent snapshot and a snapshot doesn't exist 27 1.delta 3.snapshot2.delta 1.delta 3.snapshot2.delta 4.delta 1.delta 3.delta2.delta 4.delta
  • 28. State store configuration spark.sql.streaming.stateStore: → .minDeltasForSnapshot → .maintenanceInterval 28 spark.sql.streaming: → .maxBatchesToRetainInMemory
  • 30. Few takeaways ● yet another TDD acronym - Trade-Off Driven Development ○ simplicity for latency ○ simplicity for accuracy ○ scaling for latency ● AWS ○ Kinesis - short retention period = reprocessing boundary, connector ○ S3 - trade reliability for performance ○ EMR - transient cluster ○ Redshift - COPY ● Apache Spark ○ watermarks everywhere - batch simulation ○ state store configuration ○ restore mechanism ○ overwrite idempotent mode 30
  • 32. Thank you!Bartosz Konieczny @waitingforcode / github.com/bartosz25 / waitingforcode.com Canal+ @canaltechteam