SlideShare une entreprise Scribd logo
1  sur  63
Télécharger pour lire hors ligne
Productizing Structured
Streaming Jobs
Burak Yavuz
April 24, 2019 – SAIS 2019 San Francisco
Who	am	I
● Software	Engineer	– Databricks
- “We	make	your	streams	come	true”
● Apache	Spark	Committer
● MS	in	Management	Science	&	Engineering	-
Stanford	University
● BS	in	Mechanical	Engineering	- Bogazici
University,	Istanbul
Writing code is fun…
… is that all we do?
Image from: https://www.smartsheet.com/sites/default/files/IC-Software-Development-Life-Cycle.jpg
Let’s look at the operational
aspects of data pipelines
Agenda
How to
• Test
• Monitor
• Deploy
• Update
Structured Streaming Jobs
Structured Streaming
stream processing on Spark SQL engine
fast, scalable, fault-tolerant
rich, unified, high level APIs
deal with complex data and complex workloads
rich ecosystem of data sources
integrate with many storage systems
Structured Streaming @
1000s of customer streaming apps
in production on Databricks
1000+ trillions of rows processed
in production
Streaming word count
Anatomy of a Streaming Query
spark.readStream
.format("kafka")
.option("subscribe", "input")
.load()
.groupBy($"value".cast("string"))
.count()
.writeStream
.format("kafka")
.option("topic", "output")
.trigger("1 minute")
.outputMode(OutputMode.Complete())
.option("checkpointLocation", "…")
.start()
Source
• Specify one or more locations
to read data from
• Built in support for
Files/Kafka/Socket,
pluggable.
• Can include multiple sources
of different types using
union()
Anatomy of a Streaming Query
spark.readStream
.format("kafka")
.option("subscribe", "input")
.load()
.groupBy('value.cast("string") as 'key)
.agg(count("*") as 'value)
.writeStream
.format("kafka")
.option("topic", "output")
.trigger("1 minute")
.outputMode(OutputMode.Complete())
.option("checkpointLocation", "…")
.start()
Transformation
• Using DataFrames,
Datasets and/or SQL.
• Catalyst figures out how to
execute the transformation
incrementally.
• Internal processing always
exactly-once.
Anatomy of a Streaming Query
spark.readStream
.format("kafka")
.option("subscribe", "input")
.load()
.groupBy('value.cast("string") as 'key)
.agg(count("*") as 'value)
.writeStream
.format("kafka")
.option("topic", "output")
.trigger("1 minute")
.outputMode(OutputMode.Complete())
.option("checkpointLocation", "…")
.start()
Sink
• Accepts the output of each
batch.
• When supported sinks are
transactional and exactly
once (Files).
• Use foreach to execute
arbitrary code.
Anatomy of a Streaming Query
spark.readStream
.format("kafka")
.option("subscribe", "input")
.load()
.groupBy('value.cast("string") as 'key)
.agg(count("*") as 'value)
.writeStream
.format("kafka")
.option("topic", "output")
.trigger("1 minute")
.outputMode("update")
.option("checkpointLocation", "…")
.start()
Output mode – What's output
• Complete – Output the whole answer
every time
• Update – Output changed rows
• Append – Output new rows only
Trigger – When to output
• Specified as a time, eventually
supports data size
• No trigger means as fast as possible
Anatomy of a Streaming Query
spark.readStream
.format("kafka")
.option("subscribe", "input")
.load()
.groupBy('value.cast("string") as 'key)
.agg(count("*") as 'value)
.writeStream
.format("kafka")
.option("topic", "output")
.trigger("1 minute")
.outputMode("update")
.option("checkpointLocation", "…")
.start()
Checkpoint
• Tracks the progress of a
query in persistent storage
• Can be used to restart the
query if there is a failure.
Reference Architecture
Data Pipelines @ Databricks
Event Based
Reporting
Streaming
Analytics
Bronze Tables Silver Tables Gold Tables
Event Based File Sources
• Launched Structured Streaming connectors:
• s3-sqs on AWS (DBR 3.5)
• abs-aqs on Azure (DBR 5.0)
• As blobs are generated:
• Events are published to SQS/AQS
• Spark reads these events
• Then reads original files from
blob storage system Azure
Blob
Storage Event Grid
Queue Storage
AWS SQS
AWS S3
Properties of Bronze/Silver/Gold
• Bronze tables
• No data processing
• Deduplication + JSON => Parquet conversion
• Data kept around for a couple weeks in order to fix mistakes just in case
• Silver tables
• Tens/Hundreds of tables
• Directly queryable tables
• PII masking/redaction
• Gold tables
• Materialized views of silver tables
• Curated tables by the Data Science team
Why this Architecture?
• Maximize Flexibility
• Maximize Scalability
• Lower Costs
See TD’s talk:
“Designing Structured Streaming
Pipelines—How to Architect Things Right”
April 25 2:40pm – Streaming Track
Testing
Testing
spark.readStream
.format("kafka")
.option("subscribe", "input")
.load()
.groupBy('value.cast("string") as 'key)
.agg(count("*") as 'value)
.writeStream
.format("kafka")
.option("topic", "output")
.trigger("1 minute")
.outputMode("update")
.option("checkpointLocation", "…")
.start()
- How do we test this
code?
- Do we need to set up
Kafka?
- How do we verify
result correctness?
Testing
Strategy 1: Don’t care about sources and sinks. Just test your
business logic, using batch DataFrames
.groupBy('value.cast("string") as 'key)
.agg(count("*") as 'value)
Pros:
- Easy to do in
Scala/Python
Cons:
- Not all batch operations
are supported in Streaming
Testing
Strategy 2: Leverage the StreamTest test harness available in Apache
Spark
val inputData = MemoryStream[Array[Byte]]
val stream = inputData.toDS().toDF("value")
.groupBy('value.cast("string") as 'key)
.agg(count("*") as 'value)
testStream(stream, OutputMode.Update)(
AddData(inputData, "a".getBytes(), "b".getBytes()),
CheckAnswer(("a" -> 1), ("b" -> 1))
)
Testing
Strategy 2: Leverage the StreamTest test harness
available in Apache Spark
val inputData = MemoryStream[Array[Byte]]
val stream = inputData.toDS().toDF("value")
.groupBy('value.cast("string") as 'key)
.agg(count("*") as 'value)
testStream(stream, OutputMode.Update)(
AddData(inputData, "a".getBytes(), "b".getBytes()),
CheckAnswer(("a" -> 1), ("b" -> 1))
)
Source is in
memory
Schema can be set
arbitrarily to mimic
real source
Testing
Strategy 2: Leverage the StreamTest test harness
available in Apache Spark
val inputData = MemoryStream[Array[Byte]]
val stream = inputData.toDS().toDF("value")
.groupBy('value.cast("string") as 'key)
.agg(count("*") as 'value)
testStream(stream, OutputMode.Update)(
AddData(inputData, "a".getBytes(), "b".getBytes()),
CheckAnswer(("a" -> 1), ("b" -> 1))
)
Transformation
unchanged.
Testing
Strategy 2: Leverage the StreamTest test harness
available in Apache Spark
testStream(stream, OutputMode.Update)(
AddData(inputData, ...),
CheckAnswer(("a" -> 1), ("b" -> 1))
)
Starts a stream outputting
data to a memory sink
Testing
Strategy 2: Leverage the StreamTest test harness
available in Apache Spark
testStream(stream, OutputMode.Update)(
AddData(inputData, "a".getBytes(), "b".getBytes()),
CheckAnswer(("a" -> 1), ("b" -> 1))
)
Add
data to
the
source
Testing
Strategy 2: Leverage the StreamTest test harness
available in Apache Spark
testStream(stream, OutputMode.Update)(
AddData(inputData, "a".getBytes(), "b".getBytes()),
CheckAnswer(("a" -> 1), ("b" -> 1))
)
Process all data and
check result
Testing
Available actions in StreamTest:
- StartStream: Allows you to provide a trigger, checkpoint location, or SQL
configurations for your stream
- AddData: Adds data to your source
- CheckAnswer: Check the current data available in your sink
- CheckLastBatch: Check data that was written to your sink in the last
epoch/micro-batch
- StopStream: Stop your stream to mimic failures/upgrades
- ExpectFailure: Allows you to test failure scenarios on the last batch based
on input data
Testing
When things go wrong:
[info] - map with recovery *** FAILED *** (8 seconds, 79 milliseconds)
[info] == Results ==
[info] !== Correct Answer - 6 == == Spark Answer - 3 ==
[info] struct<value:int> struct<value:int>
[info] [2] [2]
[info] [3] [3]
[info] [4] [4]
[info] ![5]
[info] ![6]
[info] ![7]
Testing
When things go wrong (cont’d):
[info] == Progress ==
[info] AddData to MemoryStream[value#1]: 1,2,3
[info] StartStream(ProcessingTime(0),org.apache.spark.util.SystemClock@4
0afc81b,Map(),null)
[info] CheckAnswer: [2],[3],[4]
[info] StopStream
[info] StartStream(ProcessingTime(0),org.apache.spark.util.SystemClock@1
074e4ef,Map(),null)
[info] => CheckAnswer: [2],[3],[4],[5],[6],[7]
[info]
Testing
When things go wrong (cont’d):
[info] == Stream ==
[info] Output Mode: Append
[info] Stream state: {MemoryStream[value#1]: 0}
[info] Thread state: alive
[info] Thread stack trace: java.lang.Thread.sleep(Native Method)
[info] org.apache.spark.sql.execution.streaming.MicroBatchExecution.$anonfun$runActivatedStream$1(MicroBa
tchExecution.scala:236)
[info] org.apache.spark.sql.execution.streaming.MicroBatchExecution$$Lambda$1357/1672418781.apply$mcZ$sp(
Unknown Source)
[info] org.apache.spark.sql.execution.streaming.ProcessingTimeExecutor.execute(TriggerExecutor.scala:56)
[info] org.apache.spark.sql.execution.streaming.MicroBatchExecution.runActivatedStream(MicroBatchExecutio
n.scala:180)
[info] org.apache.spark.sql.execution.streaming.StreamExecution.org$apache$spark$sql$execution$streaming$
StreamExecution$$runStream(StreamExecution.scala:345)
[info] org.apache.spark.sql.execution.streaming.StreamExecution$$anon$1.run(StreamExecution.scala:257)
[info]
[info]
[info] == Sink ==
[info] 0: [2] [3] [4]
Testing
How to use StreamTest?
a) Copy the code from the Spark repository to your project (recommended)
- Isolates you from changes in open source that may break your build
Testing
How to use StreamTest?
b) Import the spark-sql test jars
Maven:
<dependency>
<groupId>org.apache.spark</groupId>
<artifactId>spark-sql_2.11</artifactId>
<version>2.4.1</version>
<scope>test</scope>
<type>test-jar</type>
</dependency>
SBT:
"org.apache.spark" %% "spark-sql” % "2.4.0" % "test" classifier "tests"
Testing
Strategy 2: Leverage the StreamTest test harness available in Apache
Spark
Pros:
- A great test harness for free!
- Quick and cheap way to test
business logic
Cons:
- Only available in Scala
Testing
Strategy 3: Integration testing using Databricks Jobs
1. Have a replica of production in a staging account
2. Use Databricks REST APIs/Airflow/Azure Data Factory to kick off a
single-run job
3. Verify data output, data latency, job duration
Pros:
- Closest option to mirror
production
Cons:
- Hard to set up
- Expensive
Testing
Testing
What else to watch out for?
- Table schemas: Changing the schema/logic of one stream upstream
can break cascading jobs
Stay tuned for Spark Summit Europe!
- Dependency hell: The environment your local machine or Continuous
Integration service may differ from Production
Check out Databricks Container Services!
Testing
What else to watch out for?
- Stress Testing: Most times Spark isn’t the bottleneck. In fact,
throwing more money at your Spark clusters make the problem worse!
a) Don’t forget to tune your Kafka brokers (num.io.threads,
num.network.threads)
b) Most cloud services have rate limits, make sure you avoid them as
much as you can
Testing Best Practices
1. Leverage the StreamTest harness
for unit tests
- Use MemorySource and
MemorySink to test business logic
Testing Best Practices
2. Maintain a staging environment to
integration test before pushing to
production
- You can use Databricks Jobs and
Databricks Container Services to
ensure you have a replica of your
production environment
Testing Best Practices
3. Don’t forget to test data dependencies, schema changes upstream
can break downstream jobs
Testing Best Practices
4. Perform stress tests in staging
environment to have a runbook for
production. Not all problems lie in
your Spark cluster.
Monitoring
Monitoring
Get last progress of the
streaming query
Current input and processing rates
Current processed offsets
Current state metrics
Get progress asynchronously
through by registering your own
StreamingQueryListener
new StreamingQueryListener {
def onQueryStart(...)
def onQueryProgress(...)
def onQueryTermination(...)
}
streamingQuery.lastProgress()
{ ...
"inputRowsPerSecond" : 10024.225210926405,
"processedRowsPerSecond" : 10063.737001006373,
"durationMs" : { ... },
"sources" : [ ... ],
"sink" : { ... }
...
}
Monitoring
Leverage the StreamingQueryListener API
Push data to:
- Azure Monitor
- AWS CloudWatch
- Apache Kafka
{
"id" : "be3ff70b-d2e7-428f-ac68-31ee765c7744",
"runId" : "2302c661-ae0f-4a52-969f-c0d62899af06",
"name" : null,
"timestamp" : "2019-04-23T00:32:26.146Z",
"batchId" : 3,
"numInputRows" : 4316,
"inputRowsPerSecond" : 169.45425991362387,
"processedRowsPerSecond" : 158.81660288489846,
"durationMs" : {
"addBatch" : 26364,
"getBatch" : 6,
"getOffset" : 23,
"queryPlanning" : 12,
"triggerExecution" : 27176,
"walCommit" : 365
},
"stateOperators" : [ ...],
"sources" : [ ... ],
"sink" : { "description" : ... }
}
Monitoring
Even if you are running a map-only job, you can add a watermark
- This allows you to collect event time min, max, average in metrics
You can add current_timestamp() to keep track of ingress timestamps
- udf(() => new java.sql.Timestamp(System.currentTimeMillis)) to get
accurate processing timestamp
Monitoring
Start streams on your tables for monitoring and build streaming
dashboards in Databricks!
• Use display(streaming_df) to get live updating displays in Databricks
• Use foreach/foreachBatch to trigger alerts
Deploying
Data Pipelines @ Databricks
Event Based
Reporting
Streaming
Analytics
Bronze Tables Silver Tables Gold Tables
Deploying
Where to deploy this many (hundreds of) streams?
a) Each stream gets a cluster
Pros: Cons:
+ Better isolation - Costly
- More moving parts
b) Multiplex many streams on a single cluster
Pros: Cons:
+ Better cluster utilization - Driver becomes a bottleneck
+ Potential Delta Cache - Determining how many is difficult
re-use - Load balancing streams across
clusters also difficult
Deploying
What causes bottlenecks in the Driver?
1. Locks!
- JSON Serialization of offsets in streaming (Jackson)
- Scala compiler (Encoder creation)
- Hadoop Configurations (java.util.Properties)
- Whole Stage Codegen (ClassLoader.loadClass)
2. Garbage Collection
Deploying
How many streams can you run on a single driver?
- Depends on your streaming sources and sinks
Sources: Sink:
1. Delta Lake 1. Kafka
2. Event Based File Sources 2. Delta Lake
3. Kafka / Azure EventHub / Kinesis 3. Other File Formats
4. Other File Sources (JSON/CSV)
Efficiency
Deploying
How many streams can you run on a single driver?
- ~80 S3-SQS => Delta Streams at modest data rates
- ~40 Delta => Delta Streams at high data rates
After removing most locks, we got to 200 Delta => Delta streams at
modest data rates, with 40 streams per SparkSession
Updating
Updating
spark.readStream
.format("kafka")
.option("subscribe", "input")
.load()
.groupBy('value.cast("string") as 'key)
.agg(count("*") as 'value)
.writeStream
.format("kafka")
.option("topic", "output")
.trigger("1 minute")
.outputMode("update")
.option("checkpointLocation", "…")
.start()
Checkpoint
• Tracks the progress of a
query in persistent storage
• Can be used to restart the
query if there is a failure.
Updating
The Checkpoint:
- The checkpoint location is the unique identity of your stream
- Contains:
a) The id of the stream (json file named metadata)
b) Source offsets (folder named sources, contains json files)
c) Aggregation state (folder named state, contains binary files)
d) Commit files (folder named commits, contains json files)
e) Source Metadata (folder named sources)
Updating
Based on files stored in a checkpoint, what can you change?
1. Sinks
2. Input/Output schema (in the absence of stateful operations)
3. Triggers
4. Transformations
5. Spark Versions
Updating
Based on files stored in a checkpoint, what can’t you change?
1. Stateful operations: agg, flatMapGroupsWithState, dropDuplicates,
join
- Schema: key, value
- Parallelism: spark.sql.shuffle.partitions
- Can’t add or remove stateful operators
2. Output Mode (will work, but semantics of stream has changed)
3. Sources
Updating
How to workaround limitations?
• Restart stream from scratch
• Use new checkpoint location – avoid
eventual consistency on S3
• Partition source tables by date,
restart stream from a given date
Operating Pipelines Are Hard
Stay Tuned for:
Thank You
“Do you have any questions for my prepared answers?”
– Henry Kissinger

Contenu connexe

Tendances

Introduction to PySpark
Introduction to PySparkIntroduction to PySpark
Introduction to PySparkRussell Jurney
 
Deep Dive: Memory Management in Apache Spark
Deep Dive: Memory Management in Apache SparkDeep Dive: Memory Management in Apache Spark
Deep Dive: Memory Management in Apache SparkDatabricks
 
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
 
Pyspark Tutorial | Introduction to Apache Spark with Python | PySpark Trainin...
Pyspark Tutorial | Introduction to Apache Spark with Python | PySpark Trainin...Pyspark Tutorial | Introduction to Apache Spark with Python | PySpark Trainin...
Pyspark Tutorial | Introduction to Apache Spark with Python | PySpark Trainin...Edureka!
 
Amazon S3 Best Practice and Tuning for Hadoop/Spark in the Cloud
Amazon S3 Best Practice and Tuning for Hadoop/Spark in the CloudAmazon S3 Best Practice and Tuning for Hadoop/Spark in the Cloud
Amazon S3 Best Practice and Tuning for Hadoop/Spark in the CloudNoritaka Sekiyama
 
Intro to Delta Lake
Intro to Delta LakeIntro to Delta Lake
Intro to Delta LakeDatabricks
 
Spark SQL Tutorial | Spark Tutorial for Beginners | Apache Spark Training | E...
Spark SQL Tutorial | Spark Tutorial for Beginners | Apache Spark Training | E...Spark SQL Tutorial | Spark Tutorial for Beginners | Apache Spark Training | E...
Spark SQL Tutorial | Spark Tutorial for Beginners | Apache Spark Training | E...Edureka!
 
Deep Dive into the New Features of Apache Spark 3.0
Deep Dive into the New Features of Apache Spark 3.0Deep Dive into the New Features of Apache Spark 3.0
Deep Dive into the New Features of Apache Spark 3.0Databricks
 
Parquet performance tuning: the missing guide
Parquet performance tuning: the missing guideParquet performance tuning: the missing guide
Parquet performance tuning: the missing guideRyan Blue
 
Native Support of Prometheus Monitoring in Apache Spark 3.0
Native Support of Prometheus Monitoring in Apache Spark 3.0Native Support of Prometheus Monitoring in Apache Spark 3.0
Native Support of Prometheus Monitoring in Apache Spark 3.0Databricks
 
Considerations for Data Access in the Lakehouse
Considerations for Data Access in the LakehouseConsiderations for Data Access in the Lakehouse
Considerations for Data Access in the LakehouseDatabricks
 
A Thorough Comparison of Delta Lake, Iceberg and Hudi
A Thorough Comparison of Delta Lake, Iceberg and HudiA Thorough Comparison of Delta Lake, Iceberg and Hudi
A Thorough Comparison of Delta Lake, Iceberg and HudiDatabricks
 
Delta from a Data Engineer's Perspective
Delta from a Data Engineer's PerspectiveDelta from a Data Engineer's Perspective
Delta from a Data Engineer's PerspectiveDatabricks
 
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 in Depth: Core Concepts, Architecture & Internals
Apache Spark in Depth: Core Concepts, Architecture & InternalsApache Spark in Depth: Core Concepts, Architecture & Internals
Apache Spark in Depth: Core Concepts, Architecture & InternalsAnton Kirillov
 
A Deep Dive into Query Execution Engine of Spark SQL
A Deep Dive into Query Execution Engine of Spark SQLA Deep Dive into Query Execution Engine of Spark SQL
A Deep Dive into Query Execution Engine of Spark SQLDatabricks
 
Data Lakehouse Symposium | Day 4
Data Lakehouse Symposium | Day 4Data Lakehouse Symposium | Day 4
Data Lakehouse Symposium | Day 4Databricks
 
Apache Spark Core—Deep Dive—Proper Optimization
Apache Spark Core—Deep Dive—Proper OptimizationApache Spark Core—Deep Dive—Proper Optimization
Apache Spark Core—Deep Dive—Proper OptimizationDatabricks
 
ksqlDB: A Stream-Relational Database System
ksqlDB: A Stream-Relational Database SystemksqlDB: A Stream-Relational Database System
ksqlDB: A Stream-Relational Database Systemconfluent
 
Introduction to apache spark
Introduction to apache spark Introduction to apache spark
Introduction to apache spark Aakashdata
 

Tendances (20)

Introduction to PySpark
Introduction to PySparkIntroduction to PySpark
Introduction to PySpark
 
Deep Dive: Memory Management in Apache Spark
Deep Dive: Memory Management in Apache SparkDeep Dive: Memory Management in Apache Spark
Deep Dive: Memory Management in Apache Spark
 
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
 
Pyspark Tutorial | Introduction to Apache Spark with Python | PySpark Trainin...
Pyspark Tutorial | Introduction to Apache Spark with Python | PySpark Trainin...Pyspark Tutorial | Introduction to Apache Spark with Python | PySpark Trainin...
Pyspark Tutorial | Introduction to Apache Spark with Python | PySpark Trainin...
 
Amazon S3 Best Practice and Tuning for Hadoop/Spark in the Cloud
Amazon S3 Best Practice and Tuning for Hadoop/Spark in the CloudAmazon S3 Best Practice and Tuning for Hadoop/Spark in the Cloud
Amazon S3 Best Practice and Tuning for Hadoop/Spark in the Cloud
 
Intro to Delta Lake
Intro to Delta LakeIntro to Delta Lake
Intro to Delta Lake
 
Spark SQL Tutorial | Spark Tutorial for Beginners | Apache Spark Training | E...
Spark SQL Tutorial | Spark Tutorial for Beginners | Apache Spark Training | E...Spark SQL Tutorial | Spark Tutorial for Beginners | Apache Spark Training | E...
Spark SQL Tutorial | Spark Tutorial for Beginners | Apache Spark Training | E...
 
Deep Dive into the New Features of Apache Spark 3.0
Deep Dive into the New Features of Apache Spark 3.0Deep Dive into the New Features of Apache Spark 3.0
Deep Dive into the New Features of Apache Spark 3.0
 
Parquet performance tuning: the missing guide
Parquet performance tuning: the missing guideParquet performance tuning: the missing guide
Parquet performance tuning: the missing guide
 
Native Support of Prometheus Monitoring in Apache Spark 3.0
Native Support of Prometheus Monitoring in Apache Spark 3.0Native Support of Prometheus Monitoring in Apache Spark 3.0
Native Support of Prometheus Monitoring in Apache Spark 3.0
 
Considerations for Data Access in the Lakehouse
Considerations for Data Access in the LakehouseConsiderations for Data Access in the Lakehouse
Considerations for Data Access in the Lakehouse
 
A Thorough Comparison of Delta Lake, Iceberg and Hudi
A Thorough Comparison of Delta Lake, Iceberg and HudiA Thorough Comparison of Delta Lake, Iceberg and Hudi
A Thorough Comparison of Delta Lake, Iceberg and Hudi
 
Delta from a Data Engineer's Perspective
Delta from a Data Engineer's PerspectiveDelta from a Data Engineer's Perspective
Delta from a Data Engineer's Perspective
 
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 in Depth: Core Concepts, Architecture & Internals
Apache Spark in Depth: Core Concepts, Architecture & InternalsApache Spark in Depth: Core Concepts, Architecture & Internals
Apache Spark in Depth: Core Concepts, Architecture & Internals
 
A Deep Dive into Query Execution Engine of Spark SQL
A Deep Dive into Query Execution Engine of Spark SQLA Deep Dive into Query Execution Engine of Spark SQL
A Deep Dive into Query Execution Engine of Spark SQL
 
Data Lakehouse Symposium | Day 4
Data Lakehouse Symposium | Day 4Data Lakehouse Symposium | Day 4
Data Lakehouse Symposium | Day 4
 
Apache Spark Core—Deep Dive—Proper Optimization
Apache Spark Core—Deep Dive—Proper OptimizationApache Spark Core—Deep Dive—Proper Optimization
Apache Spark Core—Deep Dive—Proper Optimization
 
ksqlDB: A Stream-Relational Database System
ksqlDB: A Stream-Relational Database SystemksqlDB: A Stream-Relational Database System
ksqlDB: A Stream-Relational Database System
 
Introduction to apache spark
Introduction to apache spark Introduction to apache spark
Introduction to apache spark
 

Similaire à Productizing Structured Streaming Jobs

Writing Continuous Applications with Structured Streaming PySpark API
Writing Continuous Applications with Structured Streaming PySpark APIWriting Continuous Applications with Structured Streaming PySpark API
Writing Continuous Applications with Structured Streaming PySpark APIDatabricks
 
Writing Continuous Applications with Structured Streaming Python APIs in Apac...
Writing Continuous Applications with Structured Streaming Python APIs in Apac...Writing Continuous Applications with Structured Streaming Python APIs in Apac...
Writing Continuous Applications with Structured Streaming Python APIs in Apac...Databricks
 
Writing Continuous Applications with Structured Streaming in PySpark
Writing Continuous Applications with Structured Streaming in PySparkWriting Continuous Applications with Structured Streaming in PySpark
Writing Continuous Applications with Structured Streaming in PySparkDatabricks
 
Spark Sql for Training
Spark Sql for TrainingSpark Sql for Training
Spark Sql for TrainingBryan Yang
 
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
 
Easy, scalable, fault tolerant stream processing with structured streaming - ...
Easy, scalable, fault tolerant stream processing with structured streaming - ...Easy, scalable, fault tolerant stream processing with structured streaming - ...
Easy, scalable, fault tolerant stream processing with structured streaming - ...Databricks
 
Analytics Metrics delivery and ML Feature visualization: Evolution of Data Pl...
Analytics Metrics delivery and ML Feature visualization: Evolution of Data Pl...Analytics Metrics delivery and ML Feature visualization: Evolution of Data Pl...
Analytics Metrics delivery and ML Feature visualization: Evolution of Data Pl...Chester Chen
 
Easy, scalable, fault tolerant stream processing with structured streaming - ...
Easy, scalable, fault tolerant stream processing with structured streaming - ...Easy, scalable, fault tolerant stream processing with structured streaming - ...
Easy, scalable, fault tolerant stream processing with structured streaming - ...Databricks
 
Scylla Summit 2016: Analytics Show Time - Spark and Presto Powered by Scylla
Scylla Summit 2016: Analytics Show Time - Spark and Presto Powered by ScyllaScylla Summit 2016: Analytics Show Time - Spark and Presto Powered by Scylla
Scylla Summit 2016: Analytics Show Time - Spark and Presto Powered by ScyllaScyllaDB
 
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
 
Performance Troubleshooting Using Apache Spark Metrics
Performance Troubleshooting Using Apache Spark MetricsPerformance Troubleshooting Using Apache Spark Metrics
Performance Troubleshooting Using Apache Spark MetricsDatabricks
 
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
 
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
 
ETL with SPARK - First Spark London meetup
ETL with SPARK - First Spark London meetupETL with SPARK - First Spark London meetup
ETL with SPARK - First Spark London meetupRafal Kwasny
 
Spark streaming , Spark SQL
Spark streaming , Spark SQLSpark streaming , Spark SQL
Spark streaming , Spark SQLYousun Jeong
 
Making Structured Streaming Ready for Production
Making Structured Streaming Ready for ProductionMaking Structured Streaming Ready for Production
Making Structured Streaming Ready for ProductionDatabricks
 
Learnings Using Spark Streaming and DataFrames for Walmart Search: Spark Summ...
Learnings Using Spark Streaming and DataFrames for Walmart Search: Spark Summ...Learnings Using Spark Streaming and DataFrames for Walmart Search: Spark Summ...
Learnings Using Spark Streaming and DataFrames for Walmart Search: Spark Summ...Spark Summit
 

Similaire à Productizing Structured Streaming Jobs (20)

Writing Continuous Applications with Structured Streaming PySpark API
Writing Continuous Applications with Structured Streaming PySpark APIWriting Continuous Applications with Structured Streaming PySpark API
Writing Continuous Applications with Structured Streaming PySpark API
 
Writing Continuous Applications with Structured Streaming Python APIs in Apac...
Writing Continuous Applications with Structured Streaming Python APIs in Apac...Writing Continuous Applications with Structured Streaming Python APIs in Apac...
Writing Continuous Applications with Structured Streaming Python APIs in Apac...
 
Writing Continuous Applications with Structured Streaming in PySpark
Writing Continuous Applications with Structured Streaming in PySparkWriting Continuous Applications with Structured Streaming in PySpark
Writing Continuous Applications with Structured Streaming in PySpark
 
Spark Sql for Training
Spark Sql for TrainingSpark Sql for Training
Spark Sql for Training
 
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
 
Easy, scalable, fault tolerant stream processing with structured streaming - ...
Easy, scalable, fault tolerant stream processing with structured streaming - ...Easy, scalable, fault tolerant stream processing with structured streaming - ...
Easy, scalable, fault tolerant stream processing with structured streaming - ...
 
Analytics Metrics delivery and ML Feature visualization: Evolution of Data Pl...
Analytics Metrics delivery and ML Feature visualization: Evolution of Data Pl...Analytics Metrics delivery and ML Feature visualization: Evolution of Data Pl...
Analytics Metrics delivery and ML Feature visualization: Evolution of Data Pl...
 
Easy, scalable, fault tolerant stream processing with structured streaming - ...
Easy, scalable, fault tolerant stream processing with structured streaming - ...Easy, scalable, fault tolerant stream processing with structured streaming - ...
Easy, scalable, fault tolerant stream processing with structured streaming - ...
 
Scylla Summit 2016: Analytics Show Time - Spark and Presto Powered by Scylla
Scylla Summit 2016: Analytics Show Time - Spark and Presto Powered by ScyllaScylla Summit 2016: Analytics Show Time - Spark and Presto Powered by Scylla
Scylla Summit 2016: Analytics Show Time - Spark and Presto Powered by Scylla
 
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
 
Performance Troubleshooting Using Apache Spark Metrics
Performance Troubleshooting Using Apache Spark MetricsPerformance Troubleshooting Using Apache Spark Metrics
Performance Troubleshooting Using Apache Spark Metrics
 
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...
 
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
 
ETL with SPARK - First Spark London meetup
ETL with SPARK - First Spark London meetupETL with SPARK - First Spark London meetup
ETL with SPARK - First Spark London meetup
 
Spark streaming , Spark SQL
Spark streaming , Spark SQLSpark streaming , Spark SQL
Spark streaming , Spark SQL
 
Making Structured Streaming Ready for Production
Making Structured Streaming Ready for ProductionMaking Structured Streaming Ready for Production
Making Structured Streaming Ready for Production
 
So you think you can stream.pptx
So you think you can stream.pptxSo you think you can stream.pptx
So you think you can stream.pptx
 
Learnings Using Spark Streaming and DataFrames for Walmart Search: Spark Summ...
Learnings Using Spark Streaming and DataFrames for Walmart Search: Spark Summ...Learnings Using Spark Streaming and DataFrames for Walmart Search: Spark Summ...
Learnings Using Spark Streaming and DataFrames for Walmart Search: Spark Summ...
 
Google cloud Dataflow & Apache Flink
Google cloud Dataflow & Apache FlinkGoogle cloud Dataflow & Apache Flink
Google cloud Dataflow & Apache Flink
 

Plus de 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
 
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
 
Machine Learning CI/CD for Email Attack Detection
Machine Learning CI/CD for Email Attack DetectionMachine Learning CI/CD for Email Attack Detection
Machine Learning CI/CD for Email Attack DetectionDatabricks
 

Plus de 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
 
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
 
Machine Learning CI/CD for Email Attack Detection
Machine Learning CI/CD for Email Attack DetectionMachine Learning CI/CD for Email Attack Detection
Machine Learning CI/CD for Email Attack Detection
 

Dernier

Introduction-to-Machine-Learning (1).pptx
Introduction-to-Machine-Learning (1).pptxIntroduction-to-Machine-Learning (1).pptx
Introduction-to-Machine-Learning (1).pptxfirstjob4
 
Data-Analysis for Chicago Crime Data 2023
Data-Analysis for Chicago Crime Data  2023Data-Analysis for Chicago Crime Data  2023
Data-Analysis for Chicago Crime Data 2023ymrp368
 
Halmar dropshipping via API with DroFx
Halmar  dropshipping  via API with DroFxHalmar  dropshipping  via API with DroFx
Halmar dropshipping via API with DroFxolyaivanovalion
 
Call me @ 9892124323 Cheap Rate Call Girls in Vashi with Real Photo 100% Secure
Call me @ 9892124323  Cheap Rate Call Girls in Vashi with Real Photo 100% SecureCall me @ 9892124323  Cheap Rate Call Girls in Vashi with Real Photo 100% Secure
Call me @ 9892124323 Cheap Rate Call Girls in Vashi with Real Photo 100% SecurePooja Nehwal
 
Smarteg dropshipping via API with DroFx.pptx
Smarteg dropshipping via API with DroFx.pptxSmarteg dropshipping via API with DroFx.pptx
Smarteg dropshipping via API with DroFx.pptxolyaivanovalion
 
FESE Capital Markets Fact Sheet 2024 Q1.pdf
FESE Capital Markets Fact Sheet 2024 Q1.pdfFESE Capital Markets Fact Sheet 2024 Q1.pdf
FESE Capital Markets Fact Sheet 2024 Q1.pdfMarinCaroMartnezBerg
 
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
 
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
 
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
 
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
 
Junnasandra Call Girls: 🍓 7737669865 🍓 High Profile Model Escorts | Bangalore...
Junnasandra Call Girls: 🍓 7737669865 🍓 High Profile Model Escorts | Bangalore...Junnasandra Call Girls: 🍓 7737669865 🍓 High Profile Model Escorts | Bangalore...
Junnasandra Call Girls: 🍓 7737669865 🍓 High Profile Model Escorts | Bangalore...amitlee9823
 
Edukaciniai dropshipping via API with DroFx
Edukaciniai dropshipping via API with DroFxEdukaciniai dropshipping via API with DroFx
Edukaciniai dropshipping via API with DroFxolyaivanovalion
 
BDSM⚡Call Girls in Mandawali Delhi >༒8448380779 Escort Service
BDSM⚡Call Girls in Mandawali Delhi >༒8448380779 Escort ServiceBDSM⚡Call Girls in Mandawali Delhi >༒8448380779 Escort Service
BDSM⚡Call Girls in Mandawali Delhi >༒8448380779 Escort ServiceDelhi Call girls
 
Midocean dropshipping via API with DroFx
Midocean dropshipping via API with DroFxMidocean dropshipping via API with DroFx
Midocean dropshipping via API with DroFxolyaivanovalion
 
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
 
Invezz.com - Grow your wealth with trading signals
Invezz.com - Grow your wealth with trading signalsInvezz.com - Grow your wealth with trading signals
Invezz.com - Grow your wealth with trading signalsInvezz1
 
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
 
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
 
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
 

Dernier (20)

Introduction-to-Machine-Learning (1).pptx
Introduction-to-Machine-Learning (1).pptxIntroduction-to-Machine-Learning (1).pptx
Introduction-to-Machine-Learning (1).pptx
 
Data-Analysis for Chicago Crime Data 2023
Data-Analysis for Chicago Crime Data  2023Data-Analysis for Chicago Crime Data  2023
Data-Analysis for Chicago Crime Data 2023
 
Halmar dropshipping via API with DroFx
Halmar  dropshipping  via API with DroFxHalmar  dropshipping  via API with DroFx
Halmar dropshipping via API with DroFx
 
Call me @ 9892124323 Cheap Rate Call Girls in Vashi with Real Photo 100% Secure
Call me @ 9892124323  Cheap Rate Call Girls in Vashi with Real Photo 100% SecureCall me @ 9892124323  Cheap Rate Call Girls in Vashi with Real Photo 100% Secure
Call me @ 9892124323 Cheap Rate Call Girls in Vashi with Real Photo 100% Secure
 
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
 
Smarteg dropshipping via API with DroFx.pptx
Smarteg dropshipping via API with DroFx.pptxSmarteg dropshipping via API with DroFx.pptx
Smarteg dropshipping via API with DroFx.pptx
 
FESE Capital Markets Fact Sheet 2024 Q1.pdf
FESE Capital Markets Fact Sheet 2024 Q1.pdfFESE Capital Markets Fact Sheet 2024 Q1.pdf
FESE Capital Markets Fact Sheet 2024 Q1.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...
 
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...
 
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
 
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
 
Junnasandra Call Girls: 🍓 7737669865 🍓 High Profile Model Escorts | Bangalore...
Junnasandra Call Girls: 🍓 7737669865 🍓 High Profile Model Escorts | Bangalore...Junnasandra Call Girls: 🍓 7737669865 🍓 High Profile Model Escorts | Bangalore...
Junnasandra Call Girls: 🍓 7737669865 🍓 High Profile Model Escorts | Bangalore...
 
Edukaciniai dropshipping via API with DroFx
Edukaciniai dropshipping via API with DroFxEdukaciniai dropshipping via API with DroFx
Edukaciniai dropshipping via API with DroFx
 
BDSM⚡Call Girls in Mandawali Delhi >༒8448380779 Escort Service
BDSM⚡Call Girls in Mandawali Delhi >༒8448380779 Escort ServiceBDSM⚡Call Girls in Mandawali Delhi >༒8448380779 Escort Service
BDSM⚡Call Girls in Mandawali Delhi >༒8448380779 Escort Service
 
Midocean dropshipping via API with DroFx
Midocean dropshipping via API with DroFxMidocean dropshipping via API with DroFx
Midocean dropshipping via API with DroFx
 
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
 
Invezz.com - Grow your wealth with trading signals
Invezz.com - Grow your wealth with trading signalsInvezz.com - Grow your wealth with trading signals
Invezz.com - Grow your wealth with trading signals
 
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...
 
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
 
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...
 

Productizing Structured Streaming Jobs

  • 1. Productizing Structured Streaming Jobs Burak Yavuz April 24, 2019 – SAIS 2019 San Francisco
  • 2. Who am I ● Software Engineer – Databricks - “We make your streams come true” ● Apache Spark Committer ● MS in Management Science & Engineering - Stanford University ● BS in Mechanical Engineering - Bogazici University, Istanbul
  • 3. Writing code is fun… … is that all we do?
  • 5. Let’s look at the operational aspects of data pipelines
  • 6. Agenda How to • Test • Monitor • Deploy • Update Structured Streaming Jobs
  • 7. Structured Streaming stream processing on Spark SQL engine fast, scalable, fault-tolerant rich, unified, high level APIs deal with complex data and complex workloads rich ecosystem of data sources integrate with many storage systems
  • 8. Structured Streaming @ 1000s of customer streaming apps in production on Databricks 1000+ trillions of rows processed in production
  • 10. Anatomy of a Streaming Query spark.readStream .format("kafka") .option("subscribe", "input") .load() .groupBy($"value".cast("string")) .count() .writeStream .format("kafka") .option("topic", "output") .trigger("1 minute") .outputMode(OutputMode.Complete()) .option("checkpointLocation", "…") .start() Source • Specify one or more locations to read data from • Built in support for Files/Kafka/Socket, pluggable. • Can include multiple sources of different types using union()
  • 11. Anatomy of a Streaming Query spark.readStream .format("kafka") .option("subscribe", "input") .load() .groupBy('value.cast("string") as 'key) .agg(count("*") as 'value) .writeStream .format("kafka") .option("topic", "output") .trigger("1 minute") .outputMode(OutputMode.Complete()) .option("checkpointLocation", "…") .start() Transformation • Using DataFrames, Datasets and/or SQL. • Catalyst figures out how to execute the transformation incrementally. • Internal processing always exactly-once.
  • 12. Anatomy of a Streaming Query spark.readStream .format("kafka") .option("subscribe", "input") .load() .groupBy('value.cast("string") as 'key) .agg(count("*") as 'value) .writeStream .format("kafka") .option("topic", "output") .trigger("1 minute") .outputMode(OutputMode.Complete()) .option("checkpointLocation", "…") .start() Sink • Accepts the output of each batch. • When supported sinks are transactional and exactly once (Files). • Use foreach to execute arbitrary code.
  • 13. Anatomy of a Streaming Query spark.readStream .format("kafka") .option("subscribe", "input") .load() .groupBy('value.cast("string") as 'key) .agg(count("*") as 'value) .writeStream .format("kafka") .option("topic", "output") .trigger("1 minute") .outputMode("update") .option("checkpointLocation", "…") .start() Output mode – What's output • Complete – Output the whole answer every time • Update – Output changed rows • Append – Output new rows only Trigger – When to output • Specified as a time, eventually supports data size • No trigger means as fast as possible
  • 14. Anatomy of a Streaming Query spark.readStream .format("kafka") .option("subscribe", "input") .load() .groupBy('value.cast("string") as 'key) .agg(count("*") as 'value) .writeStream .format("kafka") .option("topic", "output") .trigger("1 minute") .outputMode("update") .option("checkpointLocation", "…") .start() Checkpoint • Tracks the progress of a query in persistent storage • Can be used to restart the query if there is a failure.
  • 16. Data Pipelines @ Databricks Event Based Reporting Streaming Analytics Bronze Tables Silver Tables Gold Tables
  • 17. Event Based File Sources • Launched Structured Streaming connectors: • s3-sqs on AWS (DBR 3.5) • abs-aqs on Azure (DBR 5.0) • As blobs are generated: • Events are published to SQS/AQS • Spark reads these events • Then reads original files from blob storage system Azure Blob Storage Event Grid Queue Storage AWS SQS AWS S3
  • 18. Properties of Bronze/Silver/Gold • Bronze tables • No data processing • Deduplication + JSON => Parquet conversion • Data kept around for a couple weeks in order to fix mistakes just in case • Silver tables • Tens/Hundreds of tables • Directly queryable tables • PII masking/redaction • Gold tables • Materialized views of silver tables • Curated tables by the Data Science team
  • 19. Why this Architecture? • Maximize Flexibility • Maximize Scalability • Lower Costs
  • 20. See TD’s talk: “Designing Structured Streaming Pipelines—How to Architect Things Right” April 25 2:40pm – Streaming Track
  • 22. Testing spark.readStream .format("kafka") .option("subscribe", "input") .load() .groupBy('value.cast("string") as 'key) .agg(count("*") as 'value) .writeStream .format("kafka") .option("topic", "output") .trigger("1 minute") .outputMode("update") .option("checkpointLocation", "…") .start() - How do we test this code? - Do we need to set up Kafka? - How do we verify result correctness?
  • 23. Testing Strategy 1: Don’t care about sources and sinks. Just test your business logic, using batch DataFrames .groupBy('value.cast("string") as 'key) .agg(count("*") as 'value) Pros: - Easy to do in Scala/Python Cons: - Not all batch operations are supported in Streaming
  • 24. Testing Strategy 2: Leverage the StreamTest test harness available in Apache Spark val inputData = MemoryStream[Array[Byte]] val stream = inputData.toDS().toDF("value") .groupBy('value.cast("string") as 'key) .agg(count("*") as 'value) testStream(stream, OutputMode.Update)( AddData(inputData, "a".getBytes(), "b".getBytes()), CheckAnswer(("a" -> 1), ("b" -> 1)) )
  • 25. Testing Strategy 2: Leverage the StreamTest test harness available in Apache Spark val inputData = MemoryStream[Array[Byte]] val stream = inputData.toDS().toDF("value") .groupBy('value.cast("string") as 'key) .agg(count("*") as 'value) testStream(stream, OutputMode.Update)( AddData(inputData, "a".getBytes(), "b".getBytes()), CheckAnswer(("a" -> 1), ("b" -> 1)) ) Source is in memory Schema can be set arbitrarily to mimic real source
  • 26. Testing Strategy 2: Leverage the StreamTest test harness available in Apache Spark val inputData = MemoryStream[Array[Byte]] val stream = inputData.toDS().toDF("value") .groupBy('value.cast("string") as 'key) .agg(count("*") as 'value) testStream(stream, OutputMode.Update)( AddData(inputData, "a".getBytes(), "b".getBytes()), CheckAnswer(("a" -> 1), ("b" -> 1)) ) Transformation unchanged.
  • 27. Testing Strategy 2: Leverage the StreamTest test harness available in Apache Spark testStream(stream, OutputMode.Update)( AddData(inputData, ...), CheckAnswer(("a" -> 1), ("b" -> 1)) ) Starts a stream outputting data to a memory sink
  • 28. Testing Strategy 2: Leverage the StreamTest test harness available in Apache Spark testStream(stream, OutputMode.Update)( AddData(inputData, "a".getBytes(), "b".getBytes()), CheckAnswer(("a" -> 1), ("b" -> 1)) ) Add data to the source
  • 29. Testing Strategy 2: Leverage the StreamTest test harness available in Apache Spark testStream(stream, OutputMode.Update)( AddData(inputData, "a".getBytes(), "b".getBytes()), CheckAnswer(("a" -> 1), ("b" -> 1)) ) Process all data and check result
  • 30. Testing Available actions in StreamTest: - StartStream: Allows you to provide a trigger, checkpoint location, or SQL configurations for your stream - AddData: Adds data to your source - CheckAnswer: Check the current data available in your sink - CheckLastBatch: Check data that was written to your sink in the last epoch/micro-batch - StopStream: Stop your stream to mimic failures/upgrades - ExpectFailure: Allows you to test failure scenarios on the last batch based on input data
  • 31. Testing When things go wrong: [info] - map with recovery *** FAILED *** (8 seconds, 79 milliseconds) [info] == Results == [info] !== Correct Answer - 6 == == Spark Answer - 3 == [info] struct<value:int> struct<value:int> [info] [2] [2] [info] [3] [3] [info] [4] [4] [info] ![5] [info] ![6] [info] ![7]
  • 32. Testing When things go wrong (cont’d): [info] == Progress == [info] AddData to MemoryStream[value#1]: 1,2,3 [info] StartStream(ProcessingTime(0),org.apache.spark.util.SystemClock@4 0afc81b,Map(),null) [info] CheckAnswer: [2],[3],[4] [info] StopStream [info] StartStream(ProcessingTime(0),org.apache.spark.util.SystemClock@1 074e4ef,Map(),null) [info] => CheckAnswer: [2],[3],[4],[5],[6],[7] [info]
  • 33. Testing When things go wrong (cont’d): [info] == Stream == [info] Output Mode: Append [info] Stream state: {MemoryStream[value#1]: 0} [info] Thread state: alive [info] Thread stack trace: java.lang.Thread.sleep(Native Method) [info] org.apache.spark.sql.execution.streaming.MicroBatchExecution.$anonfun$runActivatedStream$1(MicroBa tchExecution.scala:236) [info] org.apache.spark.sql.execution.streaming.MicroBatchExecution$$Lambda$1357/1672418781.apply$mcZ$sp( Unknown Source) [info] org.apache.spark.sql.execution.streaming.ProcessingTimeExecutor.execute(TriggerExecutor.scala:56) [info] org.apache.spark.sql.execution.streaming.MicroBatchExecution.runActivatedStream(MicroBatchExecutio n.scala:180) [info] org.apache.spark.sql.execution.streaming.StreamExecution.org$apache$spark$sql$execution$streaming$ StreamExecution$$runStream(StreamExecution.scala:345) [info] org.apache.spark.sql.execution.streaming.StreamExecution$$anon$1.run(StreamExecution.scala:257) [info] [info] [info] == Sink == [info] 0: [2] [3] [4]
  • 34. Testing How to use StreamTest? a) Copy the code from the Spark repository to your project (recommended) - Isolates you from changes in open source that may break your build
  • 35. Testing How to use StreamTest? b) Import the spark-sql test jars Maven: <dependency> <groupId>org.apache.spark</groupId> <artifactId>spark-sql_2.11</artifactId> <version>2.4.1</version> <scope>test</scope> <type>test-jar</type> </dependency> SBT: "org.apache.spark" %% "spark-sql” % "2.4.0" % "test" classifier "tests"
  • 36. Testing Strategy 2: Leverage the StreamTest test harness available in Apache Spark Pros: - A great test harness for free! - Quick and cheap way to test business logic Cons: - Only available in Scala
  • 37. Testing Strategy 3: Integration testing using Databricks Jobs 1. Have a replica of production in a staging account 2. Use Databricks REST APIs/Airflow/Azure Data Factory to kick off a single-run job 3. Verify data output, data latency, job duration Pros: - Closest option to mirror production Cons: - Hard to set up - Expensive
  • 39. Testing What else to watch out for? - Table schemas: Changing the schema/logic of one stream upstream can break cascading jobs Stay tuned for Spark Summit Europe! - Dependency hell: The environment your local machine or Continuous Integration service may differ from Production Check out Databricks Container Services!
  • 40. Testing What else to watch out for? - Stress Testing: Most times Spark isn’t the bottleneck. In fact, throwing more money at your Spark clusters make the problem worse! a) Don’t forget to tune your Kafka brokers (num.io.threads, num.network.threads) b) Most cloud services have rate limits, make sure you avoid them as much as you can
  • 41. Testing Best Practices 1. Leverage the StreamTest harness for unit tests - Use MemorySource and MemorySink to test business logic
  • 42. Testing Best Practices 2. Maintain a staging environment to integration test before pushing to production - You can use Databricks Jobs and Databricks Container Services to ensure you have a replica of your production environment
  • 43. Testing Best Practices 3. Don’t forget to test data dependencies, schema changes upstream can break downstream jobs
  • 44. Testing Best Practices 4. Perform stress tests in staging environment to have a runbook for production. Not all problems lie in your Spark cluster.
  • 46. Monitoring Get last progress of the streaming query Current input and processing rates Current processed offsets Current state metrics Get progress asynchronously through by registering your own StreamingQueryListener new StreamingQueryListener { def onQueryStart(...) def onQueryProgress(...) def onQueryTermination(...) } streamingQuery.lastProgress() { ... "inputRowsPerSecond" : 10024.225210926405, "processedRowsPerSecond" : 10063.737001006373, "durationMs" : { ... }, "sources" : [ ... ], "sink" : { ... } ... }
  • 47. Monitoring Leverage the StreamingQueryListener API Push data to: - Azure Monitor - AWS CloudWatch - Apache Kafka { "id" : "be3ff70b-d2e7-428f-ac68-31ee765c7744", "runId" : "2302c661-ae0f-4a52-969f-c0d62899af06", "name" : null, "timestamp" : "2019-04-23T00:32:26.146Z", "batchId" : 3, "numInputRows" : 4316, "inputRowsPerSecond" : 169.45425991362387, "processedRowsPerSecond" : 158.81660288489846, "durationMs" : { "addBatch" : 26364, "getBatch" : 6, "getOffset" : 23, "queryPlanning" : 12, "triggerExecution" : 27176, "walCommit" : 365 }, "stateOperators" : [ ...], "sources" : [ ... ], "sink" : { "description" : ... } }
  • 48. Monitoring Even if you are running a map-only job, you can add a watermark - This allows you to collect event time min, max, average in metrics You can add current_timestamp() to keep track of ingress timestamps - udf(() => new java.sql.Timestamp(System.currentTimeMillis)) to get accurate processing timestamp
  • 49. Monitoring Start streams on your tables for monitoring and build streaming dashboards in Databricks! • Use display(streaming_df) to get live updating displays in Databricks • Use foreach/foreachBatch to trigger alerts
  • 51. Data Pipelines @ Databricks Event Based Reporting Streaming Analytics Bronze Tables Silver Tables Gold Tables
  • 52. Deploying Where to deploy this many (hundreds of) streams? a) Each stream gets a cluster Pros: Cons: + Better isolation - Costly - More moving parts b) Multiplex many streams on a single cluster Pros: Cons: + Better cluster utilization - Driver becomes a bottleneck + Potential Delta Cache - Determining how many is difficult re-use - Load balancing streams across clusters also difficult
  • 53. Deploying What causes bottlenecks in the Driver? 1. Locks! - JSON Serialization of offsets in streaming (Jackson) - Scala compiler (Encoder creation) - Hadoop Configurations (java.util.Properties) - Whole Stage Codegen (ClassLoader.loadClass) 2. Garbage Collection
  • 54. Deploying How many streams can you run on a single driver? - Depends on your streaming sources and sinks Sources: Sink: 1. Delta Lake 1. Kafka 2. Event Based File Sources 2. Delta Lake 3. Kafka / Azure EventHub / Kinesis 3. Other File Formats 4. Other File Sources (JSON/CSV) Efficiency
  • 55. Deploying How many streams can you run on a single driver? - ~80 S3-SQS => Delta Streams at modest data rates - ~40 Delta => Delta Streams at high data rates After removing most locks, we got to 200 Delta => Delta streams at modest data rates, with 40 streams per SparkSession
  • 57. Updating spark.readStream .format("kafka") .option("subscribe", "input") .load() .groupBy('value.cast("string") as 'key) .agg(count("*") as 'value) .writeStream .format("kafka") .option("topic", "output") .trigger("1 minute") .outputMode("update") .option("checkpointLocation", "…") .start() Checkpoint • Tracks the progress of a query in persistent storage • Can be used to restart the query if there is a failure.
  • 58. Updating The Checkpoint: - The checkpoint location is the unique identity of your stream - Contains: a) The id of the stream (json file named metadata) b) Source offsets (folder named sources, contains json files) c) Aggregation state (folder named state, contains binary files) d) Commit files (folder named commits, contains json files) e) Source Metadata (folder named sources)
  • 59. Updating Based on files stored in a checkpoint, what can you change? 1. Sinks 2. Input/Output schema (in the absence of stateful operations) 3. Triggers 4. Transformations 5. Spark Versions
  • 60. Updating Based on files stored in a checkpoint, what can’t you change? 1. Stateful operations: agg, flatMapGroupsWithState, dropDuplicates, join - Schema: key, value - Parallelism: spark.sql.shuffle.partitions - Can’t add or remove stateful operators 2. Output Mode (will work, but semantics of stream has changed) 3. Sources
  • 61. Updating How to workaround limitations? • Restart stream from scratch • Use new checkpoint location – avoid eventual consistency on S3 • Partition source tables by date, restart stream from a given date
  • 62. Operating Pipelines Are Hard Stay Tuned for:
  • 63. Thank You “Do you have any questions for my prepared answers?” – Henry Kissinger