SlideShare a Scribd company logo
1 of 79
Download to read offline
What no one tells you about
writing a streaming app
Mark Grover (@mark_grover) - Software Engineer, Cloudera
Ted Malaska (@TedMalaska) - Group Technical Architect, Blizzard
tiny.cloudera.com/streaming-app
Book
• Book signing, 6:30pm
– Cloudera booth
• hadooparchitecturebook.com
• @hadooparchbook
• github.com/hadooparchitecturebook
• slideshare.com/hadooparchbook
Agenda
• 5 things no one tells you about writing a
streaming app
#5 - Monitoring and
managing jobs
Batch systems
Batch
processing
HDFS/S3 HDFS/S3
Managing and monitoring batch
systems
Batch
processing
HDFS/S3 HDFS/S3
● Use Cron/Oozie/Azkaban/Luigi for orchestration
● Validation logic in the job (e.g. if input dir is empty)
● Logs aggregated at end of the job
○ Track times against previous runs, etc.
● “Automatic” orchestration (micro-batching)
● Long running driver process
Streaming systems systems
Stream
processingContinuous stream
Storage (e.g. HBase,
Cassandra, Solr, Kafka, etc.)
• YARN
– Doesn’t aggregate logs until job finishes
• Spark
– Checkpoints can’t survive app or Spark upgrades
– Need to clear checkpoint directory during upgrade
Not originally built for streaming
1. Management
a. Where to run the driver?
b. How to restart the driver automatically if it fails?
c. How to pause the job?
2. Monitoring
a. How to prevent backlog i.e. make sure processing time
< batch interval?
b. How to keep tabs on health of the long running driver
process, etc.?
Big questions remain unanswered
Disclaimer
• Most discussion that follows corresponds to
YARN but can be applied to other cluster
managers as well.
Recap: (YARN) Client mode
Recap: (YARN) Cluster mode
Recap: (YARN) Client mode
Client dies,
driver dies i.e.
job dies
Recap: (YARN) Cluster mode
Still OK if the
client dies.
• Run on YARN Cluster mode
– Driver will continue running when the client machine
goes down
1a. Where to run the driver?
• Set up automatic restart
s
In spark configuration (e.g. spark-defaults.conf):
spark.yarn.maxAppAttempts=2
spark.yarn.am.attemptFailuresValidityInterval=1h
1b. How to restart driver?
● If running on Mesos, use Marathon:
● See “Graceful Shutdown” later for YARN, etc.
1c. How to pause a job?
Image source: https://raw.githubusercontent.com/mhausenblas/elsa/master/doc/elsa-marathon-deploy.png
Suspend
button!
2. Monitoring - Spark Streaming UI
Records
processed
Min rate
(records/s
ec)
25th
percentile
50th
percentile
(Median)
75th
percentile
Max rate
(records/s
ec)
Processing time,
scheduling delay, total
delay, etc.
Base image from: http://i.imgur.com/1ooDGhm.png
2. Monitoring - But I want more!
• Spark has a configurable metrics system
– Based on Dropwizard Metrics Library
• Use Graphite/Grafana to dashboard metrics like:
– Number of records processed
– Memory/GC usage on driver and executors
– Total delay
– Utilization of the cluster
Summary - Managing and
monitoring
• Afterthought
• But it’s possible
• In Structured Streaming, use
StreamingQueryListener (starting Apache
Spark 2.1)
• Spark Streaming on YARN by Marcin Kuthan
http://mkuthan.github.io/blog/2016/09/30/spark-stre
aming-on-yarn/
● Marathon
https://mesosphere.github.io/marathon/
References
#4 - Prevent data loss
Prevent data loss
• Ok, my driver is automatically restarting
• Can I lose data in between driver restarts?
No data loss!
As long as you do things the right way
How to do things the right way
Stream
processing1. File
2. Receiver based source (Flume, Kafka)
3. Kafka direct stream
Storage (e.g. HBase,
Cassandra, Solr, Kafka, etc.)
1. File sources
Stream
processing
Storage (e.g. HBase,
Cassandra, Solr, Kafka, etc.)HDFS/S3
FileStream
1. File sources
Stream
processing
Storage (e.g. HBase,
Cassandra, Solr, Kafka, etc.)HDFS/S3
FileStream
● Use checkpointing!
Checkpointing
// new context
val ssc = new StreamingContext(...)
...
// set checkpoint directory
ssc.checkpoint(checkpointDirectory)
What is checkpointing
• Metadata checkpointing (Configuration,
Incomplete batches, etc.)
– Recover from driver failures
• Data checkpointing
– Trim lineage when doing stateful transformations (e.g.
updateStateByKey)
• Spark streaming checkpoints both data and
metadata
Checkpointing gotchas
• Checkpoints don’t work across app or Spark
upgrades
• Clear out (or change) checkpointing directory
across upgrades
Ted’s Rant
Spark with
while loop
Storage (e.g. HBase,
Cassandra, Solr, Kafka, etc.)HDFS/S3
FileStream
Landing
In
Process
Finished
2. Receiver based sources
Image source: http://static.oschina.net/uploads/space/2015/0618/110032_9Fvp_1410765.png
Receiver based sources
• Enable checkpointing, AND
• Enable Write Ahead Log (WAL)
– Set the following property to true
spark.streaming.receiver.writeAheadLog.en
able
– Default is false!
Why do we need a WAL?
• Data on Receiver is stored in executor memory
• Without WAL, a failure in the middle of operation
can lead to data loss
• With WAL, data is written to durable storage
(e.g. HDFS, S3) before being acknowledged
back to source.
WAL gotchas!
• Makes a copy of all data on disk
• Use StorageLevel.MEMORY_AND_DISK_SER storage
level for your DStream
– No need to replicate the data in memory across Spark
executors
• For S3
spark.streaming.receiver.writeAheadLog.clo
seFileAfterWrite to true (similar for driver)
But what about Kafka?
• Kafka already stores a copy of the data
• Why store another copy in WAL?
● Use “direct” connector
● No need for a WAL with the direct connector!
3. Spark Streaming with Kafka
Stream
processingContinuous stream
Storage (e.g. HBase,
Cassandra, Solr, Kafka, etc.)
Kafka with WAL
Image source:
https://databricks.com/wp-content/uploads/2015/03/Screen-Shot-2015-03-29-at-10.11.42-PM.png
Kafka direct connector
(without WAL)
Image source:
https://databricks.com/wp-content/uploads/2015/03/Screen-Shot-2015-03-29-at-10.14.11-PM.png
Why no WAL?
• No receiver process creating blocks
• Data is stored in Kafka, can be directly
recovered from there
Direct connector gotchas
• Need to track offsets for driver recovery
• Checkpoints?
– No! Not recoverable across upgrades
• Track them yourself
– In ZooKeeper, HDFS, or a database
• For accuracy
– Processing needs to be idempotent, OR
– Update offsets in the same transaction when updating results
Summary - prevent data loss
• Different for different sources
• Preventable if done right
• In Structured Streaming, state is stored in
memory (backed by HDFS/S3 WAL), starting
Spark 2.1
References
• Improved fault-tolerance and zero data loss
https://databricks.com/blog/2015/01/15/improved-driver
-fault-tolerance-and-zero-data-loss-in-spark-streaming.
html
● Tracking offsets when using direct connector
http://blog.cloudera.com/blog/2015/03/exactly-once-sp
ark-streaming-from-apache-kafka/
#3 - Do I really need to use
Spark Streaming?
When do you use streaming?
When do you use streaming?
ALWAYS!
Do I need to use Spark Streaming?
Think about your goals
Types of goals
• Atomic Enrichment
• Notifications
• Joining
• Partitioning
• Ingestion
• Counting
• Incremental machine learning
• Progressive Analysis
Types of goals
• Atomic enrichment
– No cache/context needed
Types of goals
• Notifications
– NRT
– Atomic
– With Context
– With Global Summary
51
#2 - Partitioned cache data
Data is partitioned
based on field(s) and
then cached
52
#3 - External fetch
Data fetched from
external system
Types of goals
• Joining
– Big Joins
– Broadcast Joins
– Streaming Joins
Types of goals
• !!Ted to add a diagram!!
• Partitioning
– Fan out
– Fan in
– Shuffling for key isolation
Types of goals
• Ingestion
– File Systems or Block Stores
– No SQLs
– Lucene
– Kafka
Types of goals
• Counting
– Streaming counting
– Count on query
– Aggregation on query
Types of goals
• Machine Learning
Types of goals
• Progressive Analysis
– Reading the stream
– SQL on the stream
Summary - Do I need to use
streaming?
● Spark Streaming is great for
○ Accurate counts
○ Windowing aggregations
○ Progressive Analysis
○ Continuous Machine Learning
#2 - Exactly once semantics
Exactly once semantics
• No duplicate records
• Perfect Counting
In the days of Lambda
Spark Streaming State
DStream
DStream
DStream
Single Pass
Source Receiver RDD
Source Receiver RDD
RDD
Filter Count
Push inserts and
Updates to Kudu
Source Receiver
RDD
partitions
RDD
Parition
RDD
Single Pass
Filter Count
Pre-first
Batch
First
Batch
Second
Batch
Stateful RDD 1
Push inserts and
Updates to Kudu
Stateful RDD 2
Stateful RDD 1
Initial
Stateful RDD 1
Normal Spark
KuduInputFormat
Spark
Streaming
Normal
Spark
State within
Things can go wrong in many
places
Source Sink Dist. log Consumer
Spark
Streaming
OpenTSDB
KairosDB
Things can go wrong in many
places
Source Sink Dist. log Consumer
Spark
Streaming
OpenTSDB
KairosDB
Possible
Double
Possible
Resubmission
Need to manage
offsets
State needs to be
in sync with
batches
Always puts, no
increments
Always put,
must be resilient
Restarts needs to
get full state
No one else
writing to stream
aggregated fields
Seq Numbers
can help here
Consider large divergence
in event time retrieval
Summary - Exactly once semantics
● Seq number tied to sources
● Puts for external storage system
● Consider large divergence in event time
retrieval
○ Increase divergence window
○ Retrieve state from external storage system *
○ Ignore
○ Process off line
○ Source aggregation (pre-distributed log)
#1 - Graceful shutting down
your streaming app
Graceful shutdown
Can we define graceful?
• Offsets known
• State stored externally
• Stopping at the right place (i.e. batch finished)
How to be graceful?
• Thread hooks
– Check for an external flag every N seconds
/**
* Stop the execution of the streams, with option of ensuring all received data
* has been processed.
*
* @param stopSparkContext if true, stops the associated SparkContext. The underlying SparkContext
* will be stopped regardless of whether this StreamingContext has been
* started.
* @param stopGracefully if true, stops gracefully by waiting for the processing of all
* received data to be completed
*/
def stop(stopSparkContext: Boolean, stopGracefully: Boolean): Unit = {
Under the hood of grace
receiverTracker.stop(processAllReceivedData) //default is to wait 10 second, grace waits until done
jobGenerator.stop(processAllReceivedData) // Will use spark.streaming.gracefulStopTimeout
jobExecutor.shutdown()
val terminated = if (processAllReceivedData) {
jobExecutor.awaitTermination(1, TimeUnit.HOURS) // just a very large period of time
} else {
jobExecutor.awaitTermination(2, TimeUnit.SECONDS)
}
if (!terminated) {
jobExecutor.shutdownNow()
}
How to be graceful?
• cmd line
– $SPARK_HOME_DIR/bin/spark-submit --master $MASTER_REST_URL --kill $DRIVER_ID
– spark.streaming.stopGracefullyOnShutdown=true
private def stopOnShutdown(): Unit = {
val stopGracefully = conf.getBoolean("spark.streaming.stopGracefullyOnShutdown", false)
logInfo(s"Invoking stop(stopGracefully=$stopGracefully) from shutdown hook")
// Do not stop SparkContext, let its own shutdown hook stop it
stop(stopSparkContext = false, stopGracefully = stopGracefully)
}
How to be graceful?
• By marker file
– Touch a file when starting the app on HDFS
– Remove the file when you want to stop
– Separate thread in Spark app, calls
streamingContext.stop(stopSparkContext =
true, stopGracefully = true)
● Externally in ZK, HDFS, HBase, Database
● Recover on restart
Storing offsets
Summary - Graceful shutdown
● Use provided command line, or
● Background thread with a marker file
Conclusion
Conclusion
• #5 – How to monitor and manage jobs?
• #4 – How to prevent data loss?
• #3 – Do I need to use Spark Streaming?
• #2 – How to achieve exactly/effectively once
semantics?
• #1 – How to gracefully shutdown your app?
Book signing
• Cloudera booth @ 6:30pm
Thank You.
tiny.cloudera.com/streaming-app

More Related Content

What's hot

Performance Monitoring: Understanding Your Scylla Cluster
Performance Monitoring: Understanding Your Scylla ClusterPerformance Monitoring: Understanding Your Scylla Cluster
Performance Monitoring: Understanding Your Scylla ClusterScyllaDB
 
Common Strategies for Improving Performance on Your Delta Lakehouse
Common Strategies for Improving Performance on Your Delta LakehouseCommon Strategies for Improving Performance on Your Delta Lakehouse
Common Strategies for Improving Performance on Your Delta LakehouseDatabricks
 
Apache Hudi: The Path Forward
Apache Hudi: The Path ForwardApache Hudi: The Path Forward
Apache Hudi: The Path ForwardAlluxio, Inc.
 
Apache Kudu: Technical Deep Dive


Apache Kudu: Technical Deep Dive

Apache Kudu: Technical Deep Dive


Apache Kudu: Technical Deep Dive

Cloudera, Inc.
 
Facebook Messages & HBase
Facebook Messages & HBaseFacebook Messages & HBase
Facebook Messages & HBase强 王
 
Large Scale Lakehouse Implementation Using Structured Streaming
Large Scale Lakehouse Implementation Using Structured StreamingLarge Scale Lakehouse Implementation Using Structured Streaming
Large Scale Lakehouse Implementation Using Structured StreamingDatabricks
 
Apache Spark Performance Troubleshooting at Scale, Challenges, Tools, and Met...
Apache Spark Performance Troubleshooting at Scale, Challenges, Tools, and Met...Apache Spark Performance Troubleshooting at Scale, Challenges, Tools, and Met...
Apache Spark Performance Troubleshooting at Scale, Challenges, Tools, and Met...Databricks
 
Building an open data platform with apache iceberg
Building an open data platform with apache icebergBuilding an open data platform with apache iceberg
Building an open data platform with apache icebergAlluxio, Inc.
 
The Flux Capacitor of Kafka Streams and ksqlDB (Matthias J. Sax, Confluent) K...
The Flux Capacitor of Kafka Streams and ksqlDB (Matthias J. Sax, Confluent) K...The Flux Capacitor of Kafka Streams and ksqlDB (Matthias J. Sax, Confluent) K...
The Flux Capacitor of Kafka Streams and ksqlDB (Matthias J. Sax, Confluent) K...HostedbyConfluent
 
Moving Beyond Lambda Architectures with Apache Kudu
Moving Beyond Lambda Architectures with Apache KuduMoving Beyond Lambda Architectures with Apache Kudu
Moving Beyond Lambda Architectures with Apache KuduCloudera, Inc.
 
Simplify CDC Pipeline with Spark Streaming SQL and Delta Lake
Simplify CDC Pipeline with Spark Streaming SQL and Delta LakeSimplify CDC Pipeline with Spark Streaming SQL and Delta Lake
Simplify CDC Pipeline with Spark Streaming SQL and Delta LakeDatabricks
 
Delta Lake: Optimizing Merge
Delta Lake: Optimizing MergeDelta Lake: Optimizing Merge
Delta Lake: Optimizing MergeDatabricks
 
Power of the Log: LSM & Append Only Data Structures
Power of the Log: LSM & Append Only Data StructuresPower of the Log: LSM & Append Only Data Structures
Power of the Log: LSM & Append Only Data Structuresconfluent
 
Building large scale transactional data lake using apache hudi
Building large scale transactional data lake using apache hudiBuilding large scale transactional data lake using apache hudi
Building large scale transactional data lake using apache hudiBill Liu
 
Tuning Apache Kafka Connectors for Flink.pptx
Tuning Apache Kafka Connectors for Flink.pptxTuning Apache Kafka Connectors for Flink.pptx
Tuning Apache Kafka Connectors for Flink.pptxFlink Forward
 
Etl is Dead; Long Live Streams
Etl is Dead; Long Live StreamsEtl is Dead; Long Live Streams
Etl is Dead; Long Live Streamsconfluent
 
Accelerating Data Ingestion with Databricks Autoloader
Accelerating Data Ingestion with Databricks AutoloaderAccelerating Data Ingestion with Databricks Autoloader
Accelerating Data Ingestion with Databricks AutoloaderDatabricks
 
Cosco: An Efficient Facebook-Scale Shuffle Service
Cosco: An Efficient Facebook-Scale Shuffle ServiceCosco: An Efficient Facebook-Scale Shuffle Service
Cosco: An Efficient Facebook-Scale Shuffle ServiceDatabricks
 
Flexible and Real-Time Stream Processing with Apache Flink
Flexible and Real-Time Stream Processing with Apache FlinkFlexible and Real-Time Stream Processing with Apache Flink
Flexible and Real-Time Stream Processing with Apache FlinkDataWorks Summit
 

What's hot (20)

Performance Monitoring: Understanding Your Scylla Cluster
Performance Monitoring: Understanding Your Scylla ClusterPerformance Monitoring: Understanding Your Scylla Cluster
Performance Monitoring: Understanding Your Scylla Cluster
 
The Impala Cookbook
The Impala CookbookThe Impala Cookbook
The Impala Cookbook
 
Common Strategies for Improving Performance on Your Delta Lakehouse
Common Strategies for Improving Performance on Your Delta LakehouseCommon Strategies for Improving Performance on Your Delta Lakehouse
Common Strategies for Improving Performance on Your Delta Lakehouse
 
Apache Hudi: The Path Forward
Apache Hudi: The Path ForwardApache Hudi: The Path Forward
Apache Hudi: The Path Forward
 
Apache Kudu: Technical Deep Dive


Apache Kudu: Technical Deep Dive

Apache Kudu: Technical Deep Dive


Apache Kudu: Technical Deep Dive


 
Facebook Messages & HBase
Facebook Messages & HBaseFacebook Messages & HBase
Facebook Messages & HBase
 
Large Scale Lakehouse Implementation Using Structured Streaming
Large Scale Lakehouse Implementation Using Structured StreamingLarge Scale Lakehouse Implementation Using Structured Streaming
Large Scale Lakehouse Implementation Using Structured Streaming
 
Apache Spark Performance Troubleshooting at Scale, Challenges, Tools, and Met...
Apache Spark Performance Troubleshooting at Scale, Challenges, Tools, and Met...Apache Spark Performance Troubleshooting at Scale, Challenges, Tools, and Met...
Apache Spark Performance Troubleshooting at Scale, Challenges, Tools, and Met...
 
Building an open data platform with apache iceberg
Building an open data platform with apache icebergBuilding an open data platform with apache iceberg
Building an open data platform with apache iceberg
 
The Flux Capacitor of Kafka Streams and ksqlDB (Matthias J. Sax, Confluent) K...
The Flux Capacitor of Kafka Streams and ksqlDB (Matthias J. Sax, Confluent) K...The Flux Capacitor of Kafka Streams and ksqlDB (Matthias J. Sax, Confluent) K...
The Flux Capacitor of Kafka Streams and ksqlDB (Matthias J. Sax, Confluent) K...
 
Moving Beyond Lambda Architectures with Apache Kudu
Moving Beyond Lambda Architectures with Apache KuduMoving Beyond Lambda Architectures with Apache Kudu
Moving Beyond Lambda Architectures with Apache Kudu
 
Simplify CDC Pipeline with Spark Streaming SQL and Delta Lake
Simplify CDC Pipeline with Spark Streaming SQL and Delta LakeSimplify CDC Pipeline with Spark Streaming SQL and Delta Lake
Simplify CDC Pipeline with Spark Streaming SQL and Delta Lake
 
Delta Lake: Optimizing Merge
Delta Lake: Optimizing MergeDelta Lake: Optimizing Merge
Delta Lake: Optimizing Merge
 
Power of the Log: LSM & Append Only Data Structures
Power of the Log: LSM & Append Only Data StructuresPower of the Log: LSM & Append Only Data Structures
Power of the Log: LSM & Append Only Data Structures
 
Building large scale transactional data lake using apache hudi
Building large scale transactional data lake using apache hudiBuilding large scale transactional data lake using apache hudi
Building large scale transactional data lake using apache hudi
 
Tuning Apache Kafka Connectors for Flink.pptx
Tuning Apache Kafka Connectors for Flink.pptxTuning Apache Kafka Connectors for Flink.pptx
Tuning Apache Kafka Connectors for Flink.pptx
 
Etl is Dead; Long Live Streams
Etl is Dead; Long Live StreamsEtl is Dead; Long Live Streams
Etl is Dead; Long Live Streams
 
Accelerating Data Ingestion with Databricks Autoloader
Accelerating Data Ingestion with Databricks AutoloaderAccelerating Data Ingestion with Databricks Autoloader
Accelerating Data Ingestion with Databricks Autoloader
 
Cosco: An Efficient Facebook-Scale Shuffle Service
Cosco: An Efficient Facebook-Scale Shuffle ServiceCosco: An Efficient Facebook-Scale Shuffle Service
Cosco: An Efficient Facebook-Scale Shuffle Service
 
Flexible and Real-Time Stream Processing with Apache Flink
Flexible and Real-Time Stream Processing with Apache FlinkFlexible and Real-Time Stream Processing with Apache Flink
Flexible and Real-Time Stream Processing with Apache Flink
 

Viewers also liked

Building a Dataset Search Engine with Spark and Elasticsearch: Spark Summit E...
Building a Dataset Search Engine with Spark and Elasticsearch: Spark Summit E...Building a Dataset Search Engine with Spark and Elasticsearch: Spark Summit E...
Building a Dataset Search Engine with Spark and Elasticsearch: Spark Summit E...Spark Summit
 
Building Realtime Data Pipelines with Kafka Connect and Spark Streaming: Spar...
Building Realtime Data Pipelines with Kafka Connect and Spark Streaming: Spar...Building Realtime Data Pipelines with Kafka Connect and Spark Streaming: Spar...
Building Realtime Data Pipelines with Kafka Connect and Spark Streaming: Spar...Spark Summit
 
Lessons Learned from Dockerizing Spark Workloads: Spark Summit East talk by T...
Lessons Learned from Dockerizing Spark Workloads: Spark Summit East talk by T...Lessons Learned from Dockerizing Spark Workloads: Spark Summit East talk by T...
Lessons Learned from Dockerizing Spark Workloads: Spark Summit East talk by T...Spark Summit
 
Insights Without Tradeoffs Using Structured Streaming keynote by Michael Armb...
Insights Without Tradeoffs Using Structured Streaming keynote by Michael Armb...Insights Without Tradeoffs Using Structured Streaming keynote by Michael Armb...
Insights Without Tradeoffs Using Structured Streaming keynote by Michael Armb...Spark Summit
 
Monitoring the Dynamic Resource Usage of Scala and Python Spark Jobs in Yarn:...
Monitoring the Dynamic Resource Usage of Scala and Python Spark Jobs in Yarn:...Monitoring the Dynamic Resource Usage of Scala and Python Spark Jobs in Yarn:...
Monitoring the Dynamic Resource Usage of Scala and Python Spark Jobs in Yarn:...Spark Summit
 
A New “Sparkitecture” for Modernizing your Data Warehouse: Spark Summit East ...
A New “Sparkitecture” for Modernizing your Data Warehouse: Spark Summit East ...A New “Sparkitecture” for Modernizing your Data Warehouse: Spark Summit East ...
A New “Sparkitecture” for Modernizing your Data Warehouse: Spark Summit East ...Spark Summit
 
Tuning and Monitoring Deep Learning on Apache Spark
Tuning and Monitoring Deep Learning on Apache SparkTuning and Monitoring Deep Learning on Apache Spark
Tuning and Monitoring Deep Learning on Apache SparkDatabricks
 
Going Real-Time: Creating Frequently-Updating Datasets for Personalization: S...
Going Real-Time: Creating Frequently-Updating Datasets for Personalization: S...Going Real-Time: Creating Frequently-Updating Datasets for Personalization: S...
Going Real-Time: Creating Frequently-Updating Datasets for Personalization: S...Spark Summit
 
Spark SQL: Another 16x Faster After Tungsten: Spark Summit East talk by Brad ...
Spark SQL: Another 16x Faster After Tungsten: Spark Summit East talk by Brad ...Spark SQL: Another 16x Faster After Tungsten: Spark Summit East talk by Brad ...
Spark SQL: Another 16x Faster After Tungsten: Spark Summit East talk by Brad ...Spark Summit
 
Netflix's Recommendation ML Pipeline Using Apache Spark: Spark Summit East ta...
Netflix's Recommendation ML Pipeline Using Apache Spark: Spark Summit East ta...Netflix's Recommendation ML Pipeline Using Apache Spark: Spark Summit East ta...
Netflix's Recommendation ML Pipeline Using Apache Spark: Spark Summit East ta...Spark Summit
 

Viewers also liked (10)

Building a Dataset Search Engine with Spark and Elasticsearch: Spark Summit E...
Building a Dataset Search Engine with Spark and Elasticsearch: Spark Summit E...Building a Dataset Search Engine with Spark and Elasticsearch: Spark Summit E...
Building a Dataset Search Engine with Spark and Elasticsearch: Spark Summit E...
 
Building Realtime Data Pipelines with Kafka Connect and Spark Streaming: Spar...
Building Realtime Data Pipelines with Kafka Connect and Spark Streaming: Spar...Building Realtime Data Pipelines with Kafka Connect and Spark Streaming: Spar...
Building Realtime Data Pipelines with Kafka Connect and Spark Streaming: Spar...
 
Lessons Learned from Dockerizing Spark Workloads: Spark Summit East talk by T...
Lessons Learned from Dockerizing Spark Workloads: Spark Summit East talk by T...Lessons Learned from Dockerizing Spark Workloads: Spark Summit East talk by T...
Lessons Learned from Dockerizing Spark Workloads: Spark Summit East talk by T...
 
Insights Without Tradeoffs Using Structured Streaming keynote by Michael Armb...
Insights Without Tradeoffs Using Structured Streaming keynote by Michael Armb...Insights Without Tradeoffs Using Structured Streaming keynote by Michael Armb...
Insights Without Tradeoffs Using Structured Streaming keynote by Michael Armb...
 
Monitoring the Dynamic Resource Usage of Scala and Python Spark Jobs in Yarn:...
Monitoring the Dynamic Resource Usage of Scala and Python Spark Jobs in Yarn:...Monitoring the Dynamic Resource Usage of Scala and Python Spark Jobs in Yarn:...
Monitoring the Dynamic Resource Usage of Scala and Python Spark Jobs in Yarn:...
 
A New “Sparkitecture” for Modernizing your Data Warehouse: Spark Summit East ...
A New “Sparkitecture” for Modernizing your Data Warehouse: Spark Summit East ...A New “Sparkitecture” for Modernizing your Data Warehouse: Spark Summit East ...
A New “Sparkitecture” for Modernizing your Data Warehouse: Spark Summit East ...
 
Tuning and Monitoring Deep Learning on Apache Spark
Tuning and Monitoring Deep Learning on Apache SparkTuning and Monitoring Deep Learning on Apache Spark
Tuning and Monitoring Deep Learning on Apache Spark
 
Going Real-Time: Creating Frequently-Updating Datasets for Personalization: S...
Going Real-Time: Creating Frequently-Updating Datasets for Personalization: S...Going Real-Time: Creating Frequently-Updating Datasets for Personalization: S...
Going Real-Time: Creating Frequently-Updating Datasets for Personalization: S...
 
Spark SQL: Another 16x Faster After Tungsten: Spark Summit East talk by Brad ...
Spark SQL: Another 16x Faster After Tungsten: Spark Summit East talk by Brad ...Spark SQL: Another 16x Faster After Tungsten: Spark Summit East talk by Brad ...
Spark SQL: Another 16x Faster After Tungsten: Spark Summit East talk by Brad ...
 
Netflix's Recommendation ML Pipeline Using Apache Spark: Spark Summit East ta...
Netflix's Recommendation ML Pipeline Using Apache Spark: Spark Summit East ta...Netflix's Recommendation ML Pipeline Using Apache Spark: Spark Summit East ta...
Netflix's Recommendation ML Pipeline Using Apache Spark: Spark Summit East ta...
 

Similar to What No One Tells You About Writing a Streaming App: Spark Summit East talk by Mark Grover and Ted Malaska

Top 5 mistakes when writing Streaming applications
Top 5 mistakes when writing Streaming applicationsTop 5 mistakes when writing Streaming applications
Top 5 mistakes when writing Streaming applicationshadooparchbook
 
Headaches and Breakthroughs in Building Continuous Applications
Headaches and Breakthroughs in Building Continuous ApplicationsHeadaches and Breakthroughs in Building Continuous Applications
Headaches and Breakthroughs in Building Continuous ApplicationsDatabricks
 
Spark + AI Summit 2019: Headaches and Breakthroughs in Building Continuous Ap...
Spark + AI Summit 2019: Headaches and Breakthroughs in Building Continuous Ap...Spark + AI Summit 2019: Headaches and Breakthroughs in Building Continuous Ap...
Spark + AI Summit 2019: Headaches and Breakthroughs in Building Continuous Ap...Landon Robinson
 
Introduction to Apache Spark :: Lagos Scala Meetup session 2
Introduction to Apache Spark :: Lagos Scala Meetup session 2 Introduction to Apache Spark :: Lagos Scala Meetup session 2
Introduction to Apache Spark :: Lagos Scala Meetup session 2 Olalekan Fuad Elesin
 
Serverless Machine Learning on Modern Hardware Using Apache Spark with Patric...
Serverless Machine Learning on Modern Hardware Using Apache Spark with Patric...Serverless Machine Learning on Modern Hardware Using Apache Spark with Patric...
Serverless Machine Learning on Modern Hardware Using Apache Spark with Patric...Databricks
 
Spark Streaming @ Scale (Clicktale)
Spark Streaming @ Scale (Clicktale)Spark Streaming @ Scale (Clicktale)
Spark Streaming @ Scale (Clicktale)Yuval Itzchakov
 
Taboola Road To Scale With Apache Spark
Taboola Road To Scale With Apache SparkTaboola Road To Scale With Apache Spark
Taboola Road To Scale With Apache Sparktsliwowicz
 
Real time Analytics with Apache Kafka and Apache Spark
Real time Analytics with Apache Kafka and Apache SparkReal time Analytics with Apache Kafka and Apache Spark
Real time Analytics with Apache Kafka and Apache SparkRahul Jain
 
Lessons Learned: Using Spark and Microservices
Lessons Learned: Using Spark and MicroservicesLessons Learned: Using Spark and Microservices
Lessons Learned: Using Spark and MicroservicesAlexis Seigneurin
 
Highlights and Challenges from Running Spark on Mesos in Production by Morri ...
Highlights and Challenges from Running Spark on Mesos in Production by Morri ...Highlights and Challenges from Running Spark on Mesos in Production by Morri ...
Highlights and Challenges from Running Spark on Mesos in Production by Morri ...Spark Summit
 
Global Big Data Conference Sept 2014 AWS Kinesis Spark Streaming Approximatio...
Global Big Data Conference Sept 2014 AWS Kinesis Spark Streaming Approximatio...Global Big Data Conference Sept 2014 AWS Kinesis Spark Streaming Approximatio...
Global Big Data Conference Sept 2014 AWS Kinesis Spark Streaming Approximatio...Chris Fregly
 
Intro to Apache Spark by CTO of Twingo
Intro to Apache Spark by CTO of TwingoIntro to Apache Spark by CTO of Twingo
Intro to Apache Spark by CTO of TwingoMapR Technologies
 
Apache Airflow (incubating) NL HUG Meetup 2016-07-19
Apache Airflow (incubating) NL HUG Meetup 2016-07-19Apache Airflow (incubating) NL HUG Meetup 2016-07-19
Apache Airflow (incubating) NL HUG Meetup 2016-07-19Bolke de Bruin
 
Extending Spark Streaming to Support Complex Event Processing
Extending Spark Streaming to Support Complex Event ProcessingExtending Spark Streaming to Support Complex Event Processing
Extending Spark Streaming to Support Complex Event ProcessingOh Chan Kwon
 
Apache Spark - A High Level overview
Apache Spark - A High Level overviewApache Spark - A High Level overview
Apache Spark - A High Level overviewKaran Alang
 
Ingesting hdfs intosolrusingsparktrimmed
Ingesting hdfs intosolrusingsparktrimmedIngesting hdfs intosolrusingsparktrimmed
Ingesting hdfs intosolrusingsparktrimmedwhoschek
 

Similar to What No One Tells You About Writing a Streaming App: Spark Summit East talk by Mark Grover and Ted Malaska (20)

Top 5 mistakes when writing Streaming applications
Top 5 mistakes when writing Streaming applicationsTop 5 mistakes when writing Streaming applications
Top 5 mistakes when writing Streaming applications
 
Headaches and Breakthroughs in Building Continuous Applications
Headaches and Breakthroughs in Building Continuous ApplicationsHeadaches and Breakthroughs in Building Continuous Applications
Headaches and Breakthroughs in Building Continuous Applications
 
Spark + AI Summit 2019: Headaches and Breakthroughs in Building Continuous Ap...
Spark + AI Summit 2019: Headaches and Breakthroughs in Building Continuous Ap...Spark + AI Summit 2019: Headaches and Breakthroughs in Building Continuous Ap...
Spark + AI Summit 2019: Headaches and Breakthroughs in Building Continuous Ap...
 
Fault tolerance
Fault toleranceFault tolerance
Fault tolerance
 
Introduction to Apache Spark :: Lagos Scala Meetup session 2
Introduction to Apache Spark :: Lagos Scala Meetup session 2 Introduction to Apache Spark :: Lagos Scala Meetup session 2
Introduction to Apache Spark :: Lagos Scala Meetup session 2
 
Serverless Machine Learning on Modern Hardware Using Apache Spark with Patric...
Serverless Machine Learning on Modern Hardware Using Apache Spark with Patric...Serverless Machine Learning on Modern Hardware Using Apache Spark with Patric...
Serverless Machine Learning on Modern Hardware Using Apache Spark with Patric...
 
Spark Streaming @ Scale (Clicktale)
Spark Streaming @ Scale (Clicktale)Spark Streaming @ Scale (Clicktale)
Spark Streaming @ Scale (Clicktale)
 
Taboola Road To Scale With Apache Spark
Taboola Road To Scale With Apache SparkTaboola Road To Scale With Apache Spark
Taboola Road To Scale With Apache Spark
 
Apache Spark Components
Apache Spark ComponentsApache Spark Components
Apache Spark Components
 
Real time Analytics with Apache Kafka and Apache Spark
Real time Analytics with Apache Kafka and Apache SparkReal time Analytics with Apache Kafka and Apache Spark
Real time Analytics with Apache Kafka and Apache Spark
 
Lessons Learned: Using Spark and Microservices
Lessons Learned: Using Spark and MicroservicesLessons Learned: Using Spark and Microservices
Lessons Learned: Using Spark and Microservices
 
Highlights and Challenges from Running Spark on Mesos in Production by Morri ...
Highlights and Challenges from Running Spark on Mesos in Production by Morri ...Highlights and Challenges from Running Spark on Mesos in Production by Morri ...
Highlights and Challenges from Running Spark on Mesos in Production by Morri ...
 
Global Big Data Conference Sept 2014 AWS Kinesis Spark Streaming Approximatio...
Global Big Data Conference Sept 2014 AWS Kinesis Spark Streaming Approximatio...Global Big Data Conference Sept 2014 AWS Kinesis Spark Streaming Approximatio...
Global Big Data Conference Sept 2014 AWS Kinesis Spark Streaming Approximatio...
 
Intro to Apache Spark by CTO of Twingo
Intro to Apache Spark by CTO of TwingoIntro to Apache Spark by CTO of Twingo
Intro to Apache Spark by CTO of Twingo
 
Apache Airflow (incubating) NL HUG Meetup 2016-07-19
Apache Airflow (incubating) NL HUG Meetup 2016-07-19Apache Airflow (incubating) NL HUG Meetup 2016-07-19
Apache Airflow (incubating) NL HUG Meetup 2016-07-19
 
Spark cep
Spark cepSpark cep
Spark cep
 
Extending Spark Streaming to Support Complex Event Processing
Extending Spark Streaming to Support Complex Event ProcessingExtending Spark Streaming to Support Complex Event Processing
Extending Spark Streaming to Support Complex Event Processing
 
Apache Spark - A High Level overview
Apache Spark - A High Level overviewApache Spark - A High Level overview
Apache Spark - A High Level overview
 
Kudu austin oct 2015.pptx
Kudu austin oct 2015.pptxKudu austin oct 2015.pptx
Kudu austin oct 2015.pptx
 
Ingesting hdfs intosolrusingsparktrimmed
Ingesting hdfs intosolrusingsparktrimmedIngesting hdfs intosolrusingsparktrimmed
Ingesting hdfs intosolrusingsparktrimmed
 

More from Spark Summit

FPGA-Based Acceleration Architecture for Spark SQL Qi Xie and Quanfu Wang
FPGA-Based Acceleration Architecture for Spark SQL Qi Xie and Quanfu Wang FPGA-Based Acceleration Architecture for Spark SQL Qi Xie and Quanfu Wang
FPGA-Based Acceleration Architecture for Spark SQL Qi Xie and Quanfu Wang Spark Summit
 
VEGAS: The Missing Matplotlib for Scala/Apache Spark with DB Tsai and Roger M...
VEGAS: The Missing Matplotlib for Scala/Apache Spark with DB Tsai and Roger M...VEGAS: The Missing Matplotlib for Scala/Apache Spark with DB Tsai and Roger M...
VEGAS: The Missing Matplotlib for Scala/Apache Spark with DB Tsai and Roger M...Spark Summit
 
Apache Spark Structured Streaming Helps Smart Manufacturing with Xiaochang Wu
Apache Spark Structured Streaming Helps Smart Manufacturing with  Xiaochang WuApache Spark Structured Streaming Helps Smart Manufacturing with  Xiaochang Wu
Apache Spark Structured Streaming Helps Smart Manufacturing with Xiaochang WuSpark Summit
 
Improving Traffic Prediction Using Weather Data with Ramya Raghavendra
Improving Traffic Prediction Using Weather Data  with Ramya RaghavendraImproving Traffic Prediction Using Weather Data  with Ramya Raghavendra
Improving Traffic Prediction Using Weather Data with Ramya RaghavendraSpark Summit
 
A Tale of Two Graph Frameworks on Spark: GraphFrames and Tinkerpop OLAP Artem...
A Tale of Two Graph Frameworks on Spark: GraphFrames and Tinkerpop OLAP Artem...A Tale of Two Graph Frameworks on Spark: GraphFrames and Tinkerpop OLAP Artem...
A Tale of Two Graph Frameworks on Spark: GraphFrames and Tinkerpop OLAP Artem...Spark Summit
 
No More Cumbersomeness: Automatic Predictive Modeling on Apache Spark Marcin ...
No More Cumbersomeness: Automatic Predictive Modeling on Apache Spark Marcin ...No More Cumbersomeness: Automatic Predictive Modeling on Apache Spark Marcin ...
No More Cumbersomeness: Automatic Predictive Modeling on Apache Spark Marcin ...Spark Summit
 
Apache Spark and Tensorflow as a Service with Jim Dowling
Apache Spark and Tensorflow as a Service with Jim DowlingApache Spark and Tensorflow as a Service with Jim Dowling
Apache Spark and Tensorflow as a Service with Jim DowlingSpark Summit
 
Apache Spark and Tensorflow as a Service with Jim Dowling
Apache Spark and Tensorflow as a Service with Jim DowlingApache Spark and Tensorflow as a Service with Jim Dowling
Apache Spark and Tensorflow as a Service with Jim DowlingSpark Summit
 
MMLSpark: Lessons from Building a SparkML-Compatible Machine Learning Library...
MMLSpark: Lessons from Building a SparkML-Compatible Machine Learning Library...MMLSpark: Lessons from Building a SparkML-Compatible Machine Learning Library...
MMLSpark: Lessons from Building a SparkML-Compatible Machine Learning Library...Spark Summit
 
Next CERN Accelerator Logging Service with Jakub Wozniak
Next CERN Accelerator Logging Service with Jakub WozniakNext CERN Accelerator Logging Service with Jakub Wozniak
Next CERN Accelerator Logging Service with Jakub WozniakSpark Summit
 
Powering a Startup with Apache Spark with Kevin Kim
Powering a Startup with Apache Spark with Kevin KimPowering a Startup with Apache Spark with Kevin Kim
Powering a Startup with Apache Spark with Kevin KimSpark Summit
 
Improving Traffic Prediction Using Weather Datawith Ramya Raghavendra
Improving Traffic Prediction Using Weather Datawith Ramya RaghavendraImproving Traffic Prediction Using Weather Datawith Ramya Raghavendra
Improving Traffic Prediction Using Weather Datawith Ramya RaghavendraSpark Summit
 
Hiding Apache Spark Complexity for Fast Prototyping of Big Data Applications—...
Hiding Apache Spark Complexity for Fast Prototyping of Big Data Applications—...Hiding Apache Spark Complexity for Fast Prototyping of Big Data Applications—...
Hiding Apache Spark Complexity for Fast Prototyping of Big Data Applications—...Spark Summit
 
How Nielsen Utilized Databricks for Large-Scale Research and Development with...
How Nielsen Utilized Databricks for Large-Scale Research and Development with...How Nielsen Utilized Databricks for Large-Scale Research and Development with...
How Nielsen Utilized Databricks for Large-Scale Research and Development with...Spark Summit
 
Spline: Apache Spark Lineage not Only for the Banking Industry with Marek Nov...
Spline: Apache Spark Lineage not Only for the Banking Industry with Marek Nov...Spline: Apache Spark Lineage not Only for the Banking Industry with Marek Nov...
Spline: Apache Spark Lineage not Only for the Banking Industry with Marek Nov...Spark Summit
 
Goal Based Data Production with Sim Simeonov
Goal Based Data Production with Sim SimeonovGoal Based Data Production with Sim Simeonov
Goal Based Data Production with Sim SimeonovSpark Summit
 
Preventing Revenue Leakage and Monitoring Distributed Systems with Machine Le...
Preventing Revenue Leakage and Monitoring Distributed Systems with Machine Le...Preventing Revenue Leakage and Monitoring Distributed Systems with Machine Le...
Preventing Revenue Leakage and Monitoring Distributed Systems with Machine Le...Spark Summit
 
Getting Ready to Use Redis with Apache Spark with Dvir Volk
Getting Ready to Use Redis with Apache Spark with Dvir VolkGetting Ready to Use Redis with Apache Spark with Dvir Volk
Getting Ready to Use Redis with Apache Spark with Dvir VolkSpark Summit
 
Deduplication and Author-Disambiguation of Streaming Records via Supervised M...
Deduplication and Author-Disambiguation of Streaming Records via Supervised M...Deduplication and Author-Disambiguation of Streaming Records via Supervised M...
Deduplication and Author-Disambiguation of Streaming Records via Supervised M...Spark Summit
 
MatFast: In-Memory Distributed Matrix Computation Processing and Optimization...
MatFast: In-Memory Distributed Matrix Computation Processing and Optimization...MatFast: In-Memory Distributed Matrix Computation Processing and Optimization...
MatFast: In-Memory Distributed Matrix Computation Processing and Optimization...Spark Summit
 

More from Spark Summit (20)

FPGA-Based Acceleration Architecture for Spark SQL Qi Xie and Quanfu Wang
FPGA-Based Acceleration Architecture for Spark SQL Qi Xie and Quanfu Wang FPGA-Based Acceleration Architecture for Spark SQL Qi Xie and Quanfu Wang
FPGA-Based Acceleration Architecture for Spark SQL Qi Xie and Quanfu Wang
 
VEGAS: The Missing Matplotlib for Scala/Apache Spark with DB Tsai and Roger M...
VEGAS: The Missing Matplotlib for Scala/Apache Spark with DB Tsai and Roger M...VEGAS: The Missing Matplotlib for Scala/Apache Spark with DB Tsai and Roger M...
VEGAS: The Missing Matplotlib for Scala/Apache Spark with DB Tsai and Roger M...
 
Apache Spark Structured Streaming Helps Smart Manufacturing with Xiaochang Wu
Apache Spark Structured Streaming Helps Smart Manufacturing with  Xiaochang WuApache Spark Structured Streaming Helps Smart Manufacturing with  Xiaochang Wu
Apache Spark Structured Streaming Helps Smart Manufacturing with Xiaochang Wu
 
Improving Traffic Prediction Using Weather Data with Ramya Raghavendra
Improving Traffic Prediction Using Weather Data  with Ramya RaghavendraImproving Traffic Prediction Using Weather Data  with Ramya Raghavendra
Improving Traffic Prediction Using Weather Data with Ramya Raghavendra
 
A Tale of Two Graph Frameworks on Spark: GraphFrames and Tinkerpop OLAP Artem...
A Tale of Two Graph Frameworks on Spark: GraphFrames and Tinkerpop OLAP Artem...A Tale of Two Graph Frameworks on Spark: GraphFrames and Tinkerpop OLAP Artem...
A Tale of Two Graph Frameworks on Spark: GraphFrames and Tinkerpop OLAP Artem...
 
No More Cumbersomeness: Automatic Predictive Modeling on Apache Spark Marcin ...
No More Cumbersomeness: Automatic Predictive Modeling on Apache Spark Marcin ...No More Cumbersomeness: Automatic Predictive Modeling on Apache Spark Marcin ...
No More Cumbersomeness: Automatic Predictive Modeling on Apache Spark Marcin ...
 
Apache Spark and Tensorflow as a Service with Jim Dowling
Apache Spark and Tensorflow as a Service with Jim DowlingApache Spark and Tensorflow as a Service with Jim Dowling
Apache Spark and Tensorflow as a Service with Jim Dowling
 
Apache Spark and Tensorflow as a Service with Jim Dowling
Apache Spark and Tensorflow as a Service with Jim DowlingApache Spark and Tensorflow as a Service with Jim Dowling
Apache Spark and Tensorflow as a Service with Jim Dowling
 
MMLSpark: Lessons from Building a SparkML-Compatible Machine Learning Library...
MMLSpark: Lessons from Building a SparkML-Compatible Machine Learning Library...MMLSpark: Lessons from Building a SparkML-Compatible Machine Learning Library...
MMLSpark: Lessons from Building a SparkML-Compatible Machine Learning Library...
 
Next CERN Accelerator Logging Service with Jakub Wozniak
Next CERN Accelerator Logging Service with Jakub WozniakNext CERN Accelerator Logging Service with Jakub Wozniak
Next CERN Accelerator Logging Service with Jakub Wozniak
 
Powering a Startup with Apache Spark with Kevin Kim
Powering a Startup with Apache Spark with Kevin KimPowering a Startup with Apache Spark with Kevin Kim
Powering a Startup with Apache Spark with Kevin Kim
 
Improving Traffic Prediction Using Weather Datawith Ramya Raghavendra
Improving Traffic Prediction Using Weather Datawith Ramya RaghavendraImproving Traffic Prediction Using Weather Datawith Ramya Raghavendra
Improving Traffic Prediction Using Weather Datawith Ramya Raghavendra
 
Hiding Apache Spark Complexity for Fast Prototyping of Big Data Applications—...
Hiding Apache Spark Complexity for Fast Prototyping of Big Data Applications—...Hiding Apache Spark Complexity for Fast Prototyping of Big Data Applications—...
Hiding Apache Spark Complexity for Fast Prototyping of Big Data Applications—...
 
How Nielsen Utilized Databricks for Large-Scale Research and Development with...
How Nielsen Utilized Databricks for Large-Scale Research and Development with...How Nielsen Utilized Databricks for Large-Scale Research and Development with...
How Nielsen Utilized Databricks for Large-Scale Research and Development with...
 
Spline: Apache Spark Lineage not Only for the Banking Industry with Marek Nov...
Spline: Apache Spark Lineage not Only for the Banking Industry with Marek Nov...Spline: Apache Spark Lineage not Only for the Banking Industry with Marek Nov...
Spline: Apache Spark Lineage not Only for the Banking Industry with Marek Nov...
 
Goal Based Data Production with Sim Simeonov
Goal Based Data Production with Sim SimeonovGoal Based Data Production with Sim Simeonov
Goal Based Data Production with Sim Simeonov
 
Preventing Revenue Leakage and Monitoring Distributed Systems with Machine Le...
Preventing Revenue Leakage and Monitoring Distributed Systems with Machine Le...Preventing Revenue Leakage and Monitoring Distributed Systems with Machine Le...
Preventing Revenue Leakage and Monitoring Distributed Systems with Machine Le...
 
Getting Ready to Use Redis with Apache Spark with Dvir Volk
Getting Ready to Use Redis with Apache Spark with Dvir VolkGetting Ready to Use Redis with Apache Spark with Dvir Volk
Getting Ready to Use Redis with Apache Spark with Dvir Volk
 
Deduplication and Author-Disambiguation of Streaming Records via Supervised M...
Deduplication and Author-Disambiguation of Streaming Records via Supervised M...Deduplication and Author-Disambiguation of Streaming Records via Supervised M...
Deduplication and Author-Disambiguation of Streaming Records via Supervised M...
 
MatFast: In-Memory Distributed Matrix Computation Processing and Optimization...
MatFast: In-Memory Distributed Matrix Computation Processing and Optimization...MatFast: In-Memory Distributed Matrix Computation Processing and Optimization...
MatFast: In-Memory Distributed Matrix Computation Processing and Optimization...
 

Recently uploaded

Market Analysis in the 5 Largest Economic Countries in Southeast Asia.pdf
Market Analysis in the 5 Largest Economic Countries in Southeast Asia.pdfMarket Analysis in the 5 Largest Economic Countries in Southeast Asia.pdf
Market Analysis in the 5 Largest Economic Countries in Southeast Asia.pdfRachmat Ramadhan H
 
Determinants of health, dimensions of health, positive health and spectrum of...
Determinants of health, dimensions of health, positive health and spectrum of...Determinants of health, dimensions of health, positive health and spectrum of...
Determinants of health, dimensions of health, positive health and spectrum of...shambhavirathore45
 
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
 
VidaXL dropshipping via API with DroFx.pptx
VidaXL dropshipping via API with DroFx.pptxVidaXL dropshipping via API with DroFx.pptx
VidaXL dropshipping via API with DroFx.pptxolyaivanovalion
 
BabyOno dropshipping via API with DroFx.pptx
BabyOno dropshipping via API with DroFx.pptxBabyOno dropshipping via API with DroFx.pptx
BabyOno dropshipping via API with DroFx.pptxolyaivanovalion
 
Log Analysis using OSSEC sasoasasasas.pptx
Log Analysis using OSSEC sasoasasasas.pptxLog Analysis using OSSEC sasoasasasas.pptx
Log Analysis using OSSEC sasoasasasas.pptxJohnnyPlasten
 
Mature dropshipping via API with DroFx.pptx
Mature dropshipping via API with DroFx.pptxMature dropshipping via API with DroFx.pptx
Mature dropshipping via API with DroFx.pptxolyaivanovalion
 
꧁❤ Greater Noida Call Girls Delhi ❤꧂ 9711199171 ☎️ Hard And Sexy Vip Call
꧁❤ Greater Noida Call Girls Delhi ❤꧂ 9711199171 ☎️ Hard And Sexy Vip Call꧁❤ Greater Noida Call Girls Delhi ❤꧂ 9711199171 ☎️ Hard And Sexy Vip Call
꧁❤ Greater Noida Call Girls Delhi ❤꧂ 9711199171 ☎️ Hard And Sexy Vip Callshivangimorya083
 
(PARI) Call Girls Wanowrie ( 7001035870 ) HI-Fi Pune Escorts Service
(PARI) Call Girls Wanowrie ( 7001035870 ) HI-Fi Pune Escorts Service(PARI) Call Girls Wanowrie ( 7001035870 ) HI-Fi Pune Escorts Service
(PARI) Call Girls Wanowrie ( 7001035870 ) HI-Fi Pune Escorts Serviceranjana rawat
 
ALSO dropshipping via API with DroFx.pptx
ALSO dropshipping via API with DroFx.pptxALSO dropshipping via API with DroFx.pptx
ALSO dropshipping via API with DroFx.pptxolyaivanovalion
 
Chintamani Call Girls: 🍓 7737669865 🍓 High Profile Model Escorts | Bangalore ...
Chintamani Call Girls: 🍓 7737669865 🍓 High Profile Model Escorts | Bangalore ...Chintamani Call Girls: 🍓 7737669865 🍓 High Profile Model Escorts | Bangalore ...
Chintamani Call Girls: 🍓 7737669865 🍓 High Profile Model Escorts | Bangalore ...amitlee9823
 
Vip Model Call Girls (Delhi) Karol Bagh 9711199171✔️Body to body massage wit...
Vip Model  Call Girls (Delhi) Karol Bagh 9711199171✔️Body to body massage wit...Vip Model  Call Girls (Delhi) Karol Bagh 9711199171✔️Body to body massage wit...
Vip Model Call Girls (Delhi) Karol Bagh 9711199171✔️Body to body massage wit...shivangimorya083
 
Ravak dropshipping via API with DroFx.pptx
Ravak dropshipping via API with DroFx.pptxRavak dropshipping via API with DroFx.pptx
Ravak dropshipping via API with DroFx.pptxolyaivanovalion
 
BPAC WITH UFSBI GENERAL PRESENTATION 18_05_2017-1.pptx
BPAC WITH UFSBI GENERAL PRESENTATION 18_05_2017-1.pptxBPAC WITH UFSBI GENERAL PRESENTATION 18_05_2017-1.pptx
BPAC WITH UFSBI GENERAL PRESENTATION 18_05_2017-1.pptxMohammedJunaid861692
 
Halmar dropshipping via API with DroFx
Halmar  dropshipping  via API with DroFxHalmar  dropshipping  via API with DroFx
Halmar dropshipping via API with DroFxolyaivanovalion
 
100-Concepts-of-AI by Anupama Kate .pptx
100-Concepts-of-AI by Anupama Kate .pptx100-Concepts-of-AI by Anupama Kate .pptx
100-Concepts-of-AI by Anupama Kate .pptxAnupama Kate
 
Best VIP Call Girls Noida Sector 22 Call Me: 8448380779
Best VIP Call Girls Noida Sector 22 Call Me: 8448380779Best VIP Call Girls Noida Sector 22 Call Me: 8448380779
Best VIP Call Girls Noida Sector 22 Call Me: 8448380779Delhi Call girls
 
Accredited-Transport-Cooperatives-Jan-2021-Web.pdf
Accredited-Transport-Cooperatives-Jan-2021-Web.pdfAccredited-Transport-Cooperatives-Jan-2021-Web.pdf
Accredited-Transport-Cooperatives-Jan-2021-Web.pdfadriantubila
 
{Pooja: 9892124323 } Call Girl in Mumbai | Jas Kaur Rate 4500 Free Hotel Del...
{Pooja:  9892124323 } Call Girl in Mumbai | Jas Kaur Rate 4500 Free Hotel Del...{Pooja:  9892124323 } Call Girl in Mumbai | Jas Kaur Rate 4500 Free Hotel Del...
{Pooja: 9892124323 } Call Girl in Mumbai | Jas Kaur Rate 4500 Free Hotel Del...Pooja Nehwal
 

Recently uploaded (20)

Market Analysis in the 5 Largest Economic Countries in Southeast Asia.pdf
Market Analysis in the 5 Largest Economic Countries in Southeast Asia.pdfMarket Analysis in the 5 Largest Economic Countries in Southeast Asia.pdf
Market Analysis in the 5 Largest Economic Countries in Southeast Asia.pdf
 
Determinants of health, dimensions of health, positive health and spectrum of...
Determinants of health, dimensions of health, positive health and spectrum of...Determinants of health, dimensions of health, positive health and spectrum of...
Determinants of health, dimensions of health, positive health and spectrum of...
 
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
 
VidaXL dropshipping via API with DroFx.pptx
VidaXL dropshipping via API with DroFx.pptxVidaXL dropshipping via API with DroFx.pptx
VidaXL dropshipping via API with DroFx.pptx
 
BabyOno dropshipping via API with DroFx.pptx
BabyOno dropshipping via API with DroFx.pptxBabyOno dropshipping via API with DroFx.pptx
BabyOno dropshipping via API with DroFx.pptx
 
Log Analysis using OSSEC sasoasasasas.pptx
Log Analysis using OSSEC sasoasasasas.pptxLog Analysis using OSSEC sasoasasasas.pptx
Log Analysis using OSSEC sasoasasasas.pptx
 
Mature dropshipping via API with DroFx.pptx
Mature dropshipping via API with DroFx.pptxMature dropshipping via API with DroFx.pptx
Mature dropshipping via API with DroFx.pptx
 
꧁❤ Greater Noida Call Girls Delhi ❤꧂ 9711199171 ☎️ Hard And Sexy Vip Call
꧁❤ Greater Noida Call Girls Delhi ❤꧂ 9711199171 ☎️ Hard And Sexy Vip Call꧁❤ Greater Noida Call Girls Delhi ❤꧂ 9711199171 ☎️ Hard And Sexy Vip Call
꧁❤ Greater Noida Call Girls Delhi ❤꧂ 9711199171 ☎️ Hard And Sexy Vip Call
 
(PARI) Call Girls Wanowrie ( 7001035870 ) HI-Fi Pune Escorts Service
(PARI) Call Girls Wanowrie ( 7001035870 ) HI-Fi Pune Escorts Service(PARI) Call Girls Wanowrie ( 7001035870 ) HI-Fi Pune Escorts Service
(PARI) Call Girls Wanowrie ( 7001035870 ) HI-Fi Pune Escorts Service
 
ALSO dropshipping via API with DroFx.pptx
ALSO dropshipping via API with DroFx.pptxALSO dropshipping via API with DroFx.pptx
ALSO dropshipping via API with DroFx.pptx
 
Chintamani Call Girls: 🍓 7737669865 🍓 High Profile Model Escorts | Bangalore ...
Chintamani Call Girls: 🍓 7737669865 🍓 High Profile Model Escorts | Bangalore ...Chintamani Call Girls: 🍓 7737669865 🍓 High Profile Model Escorts | Bangalore ...
Chintamani Call Girls: 🍓 7737669865 🍓 High Profile Model Escorts | Bangalore ...
 
Vip Model Call Girls (Delhi) Karol Bagh 9711199171✔️Body to body massage wit...
Vip Model  Call Girls (Delhi) Karol Bagh 9711199171✔️Body to body massage wit...Vip Model  Call Girls (Delhi) Karol Bagh 9711199171✔️Body to body massage wit...
Vip Model Call Girls (Delhi) Karol Bagh 9711199171✔️Body to body massage wit...
 
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
 
Ravak dropshipping via API with DroFx.pptx
Ravak dropshipping via API with DroFx.pptxRavak dropshipping via API with DroFx.pptx
Ravak dropshipping via API with DroFx.pptx
 
BPAC WITH UFSBI GENERAL PRESENTATION 18_05_2017-1.pptx
BPAC WITH UFSBI GENERAL PRESENTATION 18_05_2017-1.pptxBPAC WITH UFSBI GENERAL PRESENTATION 18_05_2017-1.pptx
BPAC WITH UFSBI GENERAL PRESENTATION 18_05_2017-1.pptx
 
Halmar dropshipping via API with DroFx
Halmar  dropshipping  via API with DroFxHalmar  dropshipping  via API with DroFx
Halmar dropshipping via API with DroFx
 
100-Concepts-of-AI by Anupama Kate .pptx
100-Concepts-of-AI by Anupama Kate .pptx100-Concepts-of-AI by Anupama Kate .pptx
100-Concepts-of-AI by Anupama Kate .pptx
 
Best VIP Call Girls Noida Sector 22 Call Me: 8448380779
Best VIP Call Girls Noida Sector 22 Call Me: 8448380779Best VIP Call Girls Noida Sector 22 Call Me: 8448380779
Best VIP Call Girls Noida Sector 22 Call Me: 8448380779
 
Accredited-Transport-Cooperatives-Jan-2021-Web.pdf
Accredited-Transport-Cooperatives-Jan-2021-Web.pdfAccredited-Transport-Cooperatives-Jan-2021-Web.pdf
Accredited-Transport-Cooperatives-Jan-2021-Web.pdf
 
{Pooja: 9892124323 } Call Girl in Mumbai | Jas Kaur Rate 4500 Free Hotel Del...
{Pooja:  9892124323 } Call Girl in Mumbai | Jas Kaur Rate 4500 Free Hotel Del...{Pooja:  9892124323 } Call Girl in Mumbai | Jas Kaur Rate 4500 Free Hotel Del...
{Pooja: 9892124323 } Call Girl in Mumbai | Jas Kaur Rate 4500 Free Hotel Del...
 

What No One Tells You About Writing a Streaming App: Spark Summit East talk by Mark Grover and Ted Malaska

  • 1. What no one tells you about writing a streaming app Mark Grover (@mark_grover) - Software Engineer, Cloudera Ted Malaska (@TedMalaska) - Group Technical Architect, Blizzard tiny.cloudera.com/streaming-app
  • 2. Book • Book signing, 6:30pm – Cloudera booth • hadooparchitecturebook.com • @hadooparchbook • github.com/hadooparchitecturebook • slideshare.com/hadooparchbook
  • 3. Agenda • 5 things no one tells you about writing a streaming app
  • 4. #5 - Monitoring and managing jobs
  • 6. Managing and monitoring batch systems Batch processing HDFS/S3 HDFS/S3 ● Use Cron/Oozie/Azkaban/Luigi for orchestration ● Validation logic in the job (e.g. if input dir is empty) ● Logs aggregated at end of the job ○ Track times against previous runs, etc.
  • 7. ● “Automatic” orchestration (micro-batching) ● Long running driver process Streaming systems systems Stream processingContinuous stream Storage (e.g. HBase, Cassandra, Solr, Kafka, etc.)
  • 8. • YARN – Doesn’t aggregate logs until job finishes • Spark – Checkpoints can’t survive app or Spark upgrades – Need to clear checkpoint directory during upgrade Not originally built for streaming
  • 9. 1. Management a. Where to run the driver? b. How to restart the driver automatically if it fails? c. How to pause the job? 2. Monitoring a. How to prevent backlog i.e. make sure processing time < batch interval? b. How to keep tabs on health of the long running driver process, etc.? Big questions remain unanswered
  • 10. Disclaimer • Most discussion that follows corresponds to YARN but can be applied to other cluster managers as well.
  • 13. Recap: (YARN) Client mode Client dies, driver dies i.e. job dies
  • 14. Recap: (YARN) Cluster mode Still OK if the client dies.
  • 15. • Run on YARN Cluster mode – Driver will continue running when the client machine goes down 1a. Where to run the driver?
  • 16. • Set up automatic restart s In spark configuration (e.g. spark-defaults.conf): spark.yarn.maxAppAttempts=2 spark.yarn.am.attemptFailuresValidityInterval=1h 1b. How to restart driver?
  • 17. ● If running on Mesos, use Marathon: ● See “Graceful Shutdown” later for YARN, etc. 1c. How to pause a job? Image source: https://raw.githubusercontent.com/mhausenblas/elsa/master/doc/elsa-marathon-deploy.png Suspend button!
  • 18. 2. Monitoring - Spark Streaming UI Records processed Min rate (records/s ec) 25th percentile 50th percentile (Median) 75th percentile Max rate (records/s ec) Processing time, scheduling delay, total delay, etc. Base image from: http://i.imgur.com/1ooDGhm.png
  • 19. 2. Monitoring - But I want more! • Spark has a configurable metrics system – Based on Dropwizard Metrics Library • Use Graphite/Grafana to dashboard metrics like: – Number of records processed – Memory/GC usage on driver and executors – Total delay – Utilization of the cluster
  • 20. Summary - Managing and monitoring • Afterthought • But it’s possible • In Structured Streaming, use StreamingQueryListener (starting Apache Spark 2.1)
  • 21. • Spark Streaming on YARN by Marcin Kuthan http://mkuthan.github.io/blog/2016/09/30/spark-stre aming-on-yarn/ ● Marathon https://mesosphere.github.io/marathon/ References
  • 22. #4 - Prevent data loss
  • 23. Prevent data loss • Ok, my driver is automatically restarting • Can I lose data in between driver restarts?
  • 24. No data loss! As long as you do things the right way
  • 25. How to do things the right way Stream processing1. File 2. Receiver based source (Flume, Kafka) 3. Kafka direct stream Storage (e.g. HBase, Cassandra, Solr, Kafka, etc.)
  • 26. 1. File sources Stream processing Storage (e.g. HBase, Cassandra, Solr, Kafka, etc.)HDFS/S3 FileStream
  • 27. 1. File sources Stream processing Storage (e.g. HBase, Cassandra, Solr, Kafka, etc.)HDFS/S3 FileStream ● Use checkpointing!
  • 28. Checkpointing // new context val ssc = new StreamingContext(...) ... // set checkpoint directory ssc.checkpoint(checkpointDirectory)
  • 29. What is checkpointing • Metadata checkpointing (Configuration, Incomplete batches, etc.) – Recover from driver failures • Data checkpointing – Trim lineage when doing stateful transformations (e.g. updateStateByKey) • Spark streaming checkpoints both data and metadata
  • 30. Checkpointing gotchas • Checkpoints don’t work across app or Spark upgrades • Clear out (or change) checkpointing directory across upgrades
  • 31. Ted’s Rant Spark with while loop Storage (e.g. HBase, Cassandra, Solr, Kafka, etc.)HDFS/S3 FileStream Landing In Process Finished
  • 32. 2. Receiver based sources Image source: http://static.oschina.net/uploads/space/2015/0618/110032_9Fvp_1410765.png
  • 33. Receiver based sources • Enable checkpointing, AND • Enable Write Ahead Log (WAL) – Set the following property to true spark.streaming.receiver.writeAheadLog.en able – Default is false!
  • 34. Why do we need a WAL? • Data on Receiver is stored in executor memory • Without WAL, a failure in the middle of operation can lead to data loss • With WAL, data is written to durable storage (e.g. HDFS, S3) before being acknowledged back to source.
  • 35. WAL gotchas! • Makes a copy of all data on disk • Use StorageLevel.MEMORY_AND_DISK_SER storage level for your DStream – No need to replicate the data in memory across Spark executors • For S3 spark.streaming.receiver.writeAheadLog.clo seFileAfterWrite to true (similar for driver)
  • 36. But what about Kafka? • Kafka already stores a copy of the data • Why store another copy in WAL?
  • 37. ● Use “direct” connector ● No need for a WAL with the direct connector! 3. Spark Streaming with Kafka Stream processingContinuous stream Storage (e.g. HBase, Cassandra, Solr, Kafka, etc.)
  • 38. Kafka with WAL Image source: https://databricks.com/wp-content/uploads/2015/03/Screen-Shot-2015-03-29-at-10.11.42-PM.png
  • 39. Kafka direct connector (without WAL) Image source: https://databricks.com/wp-content/uploads/2015/03/Screen-Shot-2015-03-29-at-10.14.11-PM.png
  • 40. Why no WAL? • No receiver process creating blocks • Data is stored in Kafka, can be directly recovered from there
  • 41. Direct connector gotchas • Need to track offsets for driver recovery • Checkpoints? – No! Not recoverable across upgrades • Track them yourself – In ZooKeeper, HDFS, or a database • For accuracy – Processing needs to be idempotent, OR – Update offsets in the same transaction when updating results
  • 42. Summary - prevent data loss • Different for different sources • Preventable if done right • In Structured Streaming, state is stored in memory (backed by HDFS/S3 WAL), starting Spark 2.1
  • 43. References • Improved fault-tolerance and zero data loss https://databricks.com/blog/2015/01/15/improved-driver -fault-tolerance-and-zero-data-loss-in-spark-streaming. html ● Tracking offsets when using direct connector http://blog.cloudera.com/blog/2015/03/exactly-once-sp ark-streaming-from-apache-kafka/
  • 44. #3 - Do I really need to use Spark Streaming?
  • 45. When do you use streaming?
  • 46. When do you use streaming? ALWAYS!
  • 47. Do I need to use Spark Streaming? Think about your goals
  • 48. Types of goals • Atomic Enrichment • Notifications • Joining • Partitioning • Ingestion • Counting • Incremental machine learning • Progressive Analysis
  • 49. Types of goals • Atomic enrichment – No cache/context needed
  • 50. Types of goals • Notifications – NRT – Atomic – With Context – With Global Summary
  • 51. 51 #2 - Partitioned cache data Data is partitioned based on field(s) and then cached
  • 52. 52 #3 - External fetch Data fetched from external system
  • 53. Types of goals • Joining – Big Joins – Broadcast Joins – Streaming Joins
  • 54. Types of goals • !!Ted to add a diagram!! • Partitioning – Fan out – Fan in – Shuffling for key isolation
  • 55. Types of goals • Ingestion – File Systems or Block Stores – No SQLs – Lucene – Kafka
  • 56. Types of goals • Counting – Streaming counting – Count on query – Aggregation on query
  • 57. Types of goals • Machine Learning
  • 58. Types of goals • Progressive Analysis – Reading the stream – SQL on the stream
  • 59. Summary - Do I need to use streaming? ● Spark Streaming is great for ○ Accurate counts ○ Windowing aggregations ○ Progressive Analysis ○ Continuous Machine Learning
  • 60. #2 - Exactly once semantics
  • 61. Exactly once semantics • No duplicate records • Perfect Counting
  • 62. In the days of Lambda
  • 63. Spark Streaming State DStream DStream DStream Single Pass Source Receiver RDD Source Receiver RDD RDD Filter Count Push inserts and Updates to Kudu Source Receiver RDD partitions RDD Parition RDD Single Pass Filter Count Pre-first Batch First Batch Second Batch Stateful RDD 1 Push inserts and Updates to Kudu Stateful RDD 2 Stateful RDD 1 Initial Stateful RDD 1 Normal Spark KuduInputFormat Spark Streaming Normal Spark
  • 65. Things can go wrong in many places Source Sink Dist. log Consumer Spark Streaming OpenTSDB KairosDB
  • 66. Things can go wrong in many places Source Sink Dist. log Consumer Spark Streaming OpenTSDB KairosDB Possible Double Possible Resubmission Need to manage offsets State needs to be in sync with batches Always puts, no increments Always put, must be resilient Restarts needs to get full state No one else writing to stream aggregated fields Seq Numbers can help here Consider large divergence in event time retrieval
  • 67. Summary - Exactly once semantics ● Seq number tied to sources ● Puts for external storage system ● Consider large divergence in event time retrieval ○ Increase divergence window ○ Retrieve state from external storage system * ○ Ignore ○ Process off line ○ Source aggregation (pre-distributed log)
  • 68. #1 - Graceful shutting down your streaming app
  • 69. Graceful shutdown Can we define graceful? • Offsets known • State stored externally • Stopping at the right place (i.e. batch finished)
  • 70. How to be graceful? • Thread hooks – Check for an external flag every N seconds /** * Stop the execution of the streams, with option of ensuring all received data * has been processed. * * @param stopSparkContext if true, stops the associated SparkContext. The underlying SparkContext * will be stopped regardless of whether this StreamingContext has been * started. * @param stopGracefully if true, stops gracefully by waiting for the processing of all * received data to be completed */ def stop(stopSparkContext: Boolean, stopGracefully: Boolean): Unit = {
  • 71. Under the hood of grace receiverTracker.stop(processAllReceivedData) //default is to wait 10 second, grace waits until done jobGenerator.stop(processAllReceivedData) // Will use spark.streaming.gracefulStopTimeout jobExecutor.shutdown() val terminated = if (processAllReceivedData) { jobExecutor.awaitTermination(1, TimeUnit.HOURS) // just a very large period of time } else { jobExecutor.awaitTermination(2, TimeUnit.SECONDS) } if (!terminated) { jobExecutor.shutdownNow() }
  • 72. How to be graceful? • cmd line – $SPARK_HOME_DIR/bin/spark-submit --master $MASTER_REST_URL --kill $DRIVER_ID – spark.streaming.stopGracefullyOnShutdown=true private def stopOnShutdown(): Unit = { val stopGracefully = conf.getBoolean("spark.streaming.stopGracefullyOnShutdown", false) logInfo(s"Invoking stop(stopGracefully=$stopGracefully) from shutdown hook") // Do not stop SparkContext, let its own shutdown hook stop it stop(stopSparkContext = false, stopGracefully = stopGracefully) }
  • 73. How to be graceful? • By marker file – Touch a file when starting the app on HDFS – Remove the file when you want to stop – Separate thread in Spark app, calls streamingContext.stop(stopSparkContext = true, stopGracefully = true)
  • 74. ● Externally in ZK, HDFS, HBase, Database ● Recover on restart Storing offsets
  • 75. Summary - Graceful shutdown ● Use provided command line, or ● Background thread with a marker file
  • 77. Conclusion • #5 – How to monitor and manage jobs? • #4 – How to prevent data loss? • #3 – Do I need to use Spark Streaming? • #2 – How to achieve exactly/effectively once semantics? • #1 – How to gracefully shutdown your app?
  • 78. Book signing • Cloudera booth @ 6:30pm