SlideShare a Scribd company logo
1 of 88
Download to read offline
Deep Dive:
Query Execution of Spark SQL
Maryann Xue, Xingbo Jiang, Kris Mok
Apr. 2019
1
About Us
Software Engineers
• Maryann Xue
PMC of Apache Calcite & Apache Phoenix @maryannxue
• Xingbo Jiang
Apache Spark Committer @jiangxb1987
• Kris Mok
OpenJDK Committer @rednaxelafx
2
DATABRICKS WORKSPACE
Databricks Delta ML Frameworks
DATABRICKS CLOUD SERVICE
DATABRICKS RUNTIME
Reliable & Scalable Simple & Integrated
Databricks Unified Analytics Platform
APIs
Jobs
Models
Notebooks
Dashboards End to end ML lifecycle
Databricks Customers Across Industries
Financial Services Healthcare & Pharma Media & Entertainment Technology
Public Sector Retail & CPG Consumer Services Energy & Industrial IoTMarketing & AdTech
Data & Analytics Services
Apache Spark 3.x
5
Catalyst Optimization & Tungsten Execution
SparkSession / DataFrame / Dataset APIs
SQL
Spark ML
Spark
Streaming
Spark
Graph
3rd-party
Libraries
Spark CoreData Source Connectors
Apache Spark 3.x
6
Catalyst Optimization & Tungsten Execution
SparkSession / DataFrame / Dataset APIs
SQL
Spark ML
Spark
Streaming
Spark
Graph
3rd-party
Libraries
Spark CoreData Source Connectors
Spark SQL Engine
7
Analysis -> Logical Optimization -> Physical Planning -> Code Generation -> Execution
Runtime
Spark SQL Engine - Front End
8
Analysis -> Logical Optimization -> Physical Planning -> Code Generation -> Execution
Reference: A Deep Dive into Spark SQL’s Catalyst Optimizer,
Yin Huai, Spark Summit 2017
Runtime
Spark SQL Engine - Back End
9
Analysis -> Logical Optimization -> Physical Planning -> Code Generation -> Execution
Runtime
Agenda
10
Agenda
11
Physical
Planning
• Transform logical operators into physical operators
• Choose between different physical alternatives
- e.g., broadcast-hash-join vs. sort-merge-join
• Includes physical traits of the execution engine
- e.g., partitioning & ordering.
• Some ops may be mapped into multiple physical nodes
- e.g., partial agg —> shuffle —> final agg
Physical Planning
1
A Physical Plan Example
1
Scan A
Scan B
Filter
BroadcastExchange
BroadcastHashJoin
HashAggregate
ShuffleExchange
HashAggregate
SELECT a1, sum(b1)FROM A
JOIN B ON A.key = B.key
WHERE b1 < 1000 GROUP BY a1
Scan A
Filter
Join
Aggregate
Scan B
• Scalar subquery
Broadcast exchange:
- Executed as separate jobs
• Partition-local ops:
- Executed in the same stage
• Shuffle:
- The stage boundary
- A sync barrier across all nodes
Scheduling a Physical Plan
1
Job 2 Stage 1
Stage 2
Scan A
BroadcastHashJoin
HashAggregate
ShuffleExchange
HashAggregate
Job 1 Stage 1 Scan B
Filter
BroadcastExchange
Agenda
15
Code
Generation
Execution, Old: Volcano Iterator Model
• Volcano iterator model
- All ops implement the same interface, e.g., next()
- next() on final op -> pull input from child op by calling child.next() -> goes on
and on, ending up with a propagation of next() calls
• Pros: Good abstraction; Easy to implement
• Cons: Virtual function calls —> less efficient
1
Scan Filter Project Result Iterator iterate
next()next()next()
Execution, New: Whole-Stage Code Generation
• Inspired by Thomas Neumann’s paper
• Fuse a string of operators (oftentimes
the entire stage) into one WSCG op
that runs the generated code.
• A general-purpose execution engine
just like Volcano model but without
Volcano’s performance downsides:
- No virtual function calls
- Data in CPU registers
- Loop unrolling & SIMD
Scan
Filter
Project
Aggregate
long count = 0;
for (item in sales) {
if (price < 100) {
count += 1;
}
}
Execution Models: Old vs. New
• Volcano iterator model: Pull model; Driven by the final operator
1
Scan Filter Project Result Iterator iterate
next()next()next()
• WSCG model: Push model; Driven by the head/source operator
Scan Filter Project Result Iterator iterate
next()
A Physical Plan Example - WSCG
Job 2
Stage 1
Stage 2
WSCG
Scan A
BroadcastHashJoin
WSCG
HashAggregate
ShuffleExchange
HashAggregate
Job 1
Stage 1
WSCG
Scan B
Filter
BroadcastExchange
Implementation
• The top node WholeStageCodegenExec implements the iterator
interface to interop with other code-gen or non-code-gen
physical ops.
• All underlying operators implement a code-generation interface:
doProduce() & doConsume()
• Dump the generated code: df.queryExecution.debug.codegen
WSCG
Pipeline 2Pipeline 1
Single dependency
• A WSCG node contains a linear list of physical operators that
support code generation.
• No multi dependency between enclosed ops.
• A WSCG node may consist of one or more pipelines.
Op1 Op2 Op3 Op4 Op5
A Single Pipeline in WSCG
• A string of non-blocking operators form a pipeline in WSCG
• The head/source:
- Implement doProduce() - the driving loop producing source data.
• The rest:
- doProduce() - fall through to head of the pipeline.
- Implement doConsume() for its own processing logic.
Op1 Op2 Op3 WSCG
consume consume consume
produceproduceproduce produce
Generate
Code
A Single Pipeline Example
while (table.hasNext()) {
InternalRow row = table.next();
if (shouldStop()) return;
}
Scan
Filter
Project
Generated for RowIterator
WholeStageCodegen
SELECT sid FROM emps WHERE age < 36
START:
produce
produce
produce
produce
A Single Pipeline Example
while (table.hasNext()) {
InternalRow row = table.next();
if (row.getInt(2) < 36) {
}
if (shouldStop()) return;
}
Scan
Filter
Project
Generated for RowIterator
WholeStageCodegen
SELECT sid FROM emps WHERE age < 36
START:
produce
consume
produce
produce
produce
A Single Pipeline Example
while (table.hasNext()) {
InternalRow row = table.next();
if (row.getInt(2) < 36) {
String sid = row.getString(0);
rowWriter.write(0, sid);
}
if (shouldStop()) return;
}
Scan
Filter
Project
Generated for RowIterator
WholeStageCodegen
SELECT sid FROM emps WHERE age < 36
START:
produce
consume
consume
produce
produce
produce
A Single Pipeline Example
while (table.hasNext()) {
InternalRow row = table.next();
if (row.getInt(2) < 36) {
String sid = row.getString(0);
rowWriter.write(0, sid);
ret = rowWriter.getRow();
}
if (shouldStop()) return;
}
Scan
Filter
Project
Generated for RowIterator
WholeStageCodegen
SELECT sid FROM emps WHERE age < 36
START:
produce
consume
consume
consumeproduce
produce
produce
• Head (source) operator:
- The source, w/ or w/o input RDDs
- e.g., Scan, SortMergeJoin
• Non-blocking operators:
- In the middle of the pipeline
- e.g., Filter, Project
Multiple Pipelines in WSCG
• End (sink): RowIterator
- Pulls result from the last pipeline
• Blocking operators:
- End of the previous pipeline
- Start of a new pipeline
- e.g., HashAggregate, Sort
WSCG
Pipeline 2Pipeline 1
Op1 Op2 Op3 Op4 Op5
non-blockingsource non-blockingblocking
RowIterator
sink
Blocking Operators in WSCG
• A Blocking operator, e.g., HashAggregateExec, SortExec, break
pipelines, so there may be multiple pipelines in one WSCG node.
• A Blocking operator’s doConsume():
- Implement the callback to build intermediate result.
• A Blocking operator’s doProduce():
- Consume the entire output from upstream to finish building
the intermediate result.
- Start a new loop and produce output for downstream based
on the intermediate result.
A Blocking Operator Example - HashAgg
while (table.hasNext()) {
InternalRow row = table.next();
int age = row.getInt(2);
hashMap.insertOrIncrement(sid);
}
HashAggregate
doProduce()
child.produce()
HashAggregate
Scan
WholeStageCodegen
SELECT age, count(*) FROM emps GROUP BY age
consume
START:
produce
produce
A Blocking Operator Example - HashAgg
while (table.hasNext()) {
InternalRow row = table.next();
int age = row.getInt(2);
hashMap.insertOrIncrement(sid);
}
while (hashMapIter.hasNext()) {
Entry e = hashMapIter.next();
rowWriter.write(0, e.getKey());
rowWriter.write(1, e.getValue());
ret = rowWriter.getRow();
if (shouldStop()) return;
}
HashAggregate
doProduce()
child.produce()
HashAggregate
Scan
start a new pipeline
WholeStageCodegen
SELECT age, count(*) FROM emps GROUP BY age
consume
consume
START:
produce
produce
• BHJ (broadcast-hash-join) is a
pipelined operator.
• BHJ executes the build side job first,
the same way as in non-WSCG.
• BHJ is fused together with the probe
side plan (i.e., streaming plan) in
WSCG.
WSCG: BHJ vs. SMJ
Job 2
WSCG
Scan A
BroadcastHashJoin
WSCG
HashAggregate
ShuffleExchange
HashAggregate
Job 1
WSCG
Scan B
Filter
BroadcastExchange
Job 1
• SMJ (sort-merge-join) is
NOT fused with either child
plan for WSCG. Child plans
are separate WSCG nodes.
• Thus, SMJ must be the head
operator of a WSCG node.
WSCG: BHJ vs. SMJ
WSCG
SortMergeJoin
WSCG
HashAggregate
ShuffleExchange
HashAggregate
WSCG
Scan B
Filter
WSCG
Scan A
WSCG
Sort
ShuffleExchange
WSCG
Sort
ShuffleExchange
WSCG Limitations
• Problems:
- No JIT compilation for bytecode size over 8000 bytes (*).
- Over 64KB methods NOT allowed by Java Class format.
• Solutions:
- Fallback - spark.sql.codegen.fallback; spark.sql.codegen.hugeMethodLimit
- Move blocking loops into separate methods, e.g. hash-map building in
HashAgg and sort buffer building in Sort.
- Split consume() into individual methods for each operator -
spark.sql.codegen.splitConsumeFuncByOperator
About Us
Software Engineers
• Maryann Xue
PMC of Apache Calcite & Apache Phoenix @maryannxue
• Xingbo Jiang
Apache Spark Committer @jiangxb1987
• Kris Mok
OpenJDK Committer @rednaxelafx
34
Agenda
35
RDDs
(DAGs)
A Physical Plan Example
SELECT a1, sum(b1)
FROM A JOIN B
ON A.key = B.key
WHERE b1 < 1000
GROUP BY a1
Scan A
Scan B
Filter
BroadcastExchange
BroadcastHashJoin
HashAggregate
ShuffleExchange
HashAggregate
RDD and Partitions
RDD(Resilient
Distributed Dataset)
represents an
immutable,
partitioned collection
of elements that can
be operated in
parallel.
Partition
Partition
Partition
Node1
Node2
Node3
RDD
Physical Operator
Filter
Volcano
iterator
model
while (iter.hasNext())
{
val tmpVal =
iter.next()
if (condition(tmpVal))
{
return tmpVal
}
}
output
Partition
Partition
Partition
RDD
Job 2
Stage 1
Stage 2
A Physical Plan Example - Scheduling
Scan A
BroadcastHashJoin
HashAggregate
ShuffleExchange
HashAggregate
Job 1
Stage 1
Scan B
Filter
BroadcastExchange
Stage 1
Stage Execution
TaskSet0
Scan A
BroadcastHashJoin
HashAggregate
Partition0 Partition1 Partition2 Partition3
Stage Execution
0
1
2
3
Partitions
Task0
Task1
Task3
TaskSet1
Stage Execution
0
1
2
3
Partitions
Task0
Task1
Task3
TaskSet1TaskSet2
Task0
How to run a Task
Executor(spark.executor.cores=5)
Task0
spark.task.cpus=1
Task2
Task6
Task1
Task5 Task7
Task3 Task4
Fault Tolerance
● MPP-like analytics engines(e.g., Teradata, Presto, Impala):
○ Coarser-grained recovery model
○ Retry an entire query if any machine fails
○ Short/simple queries
● Spark SQL:
○ Mid-query recovery model
○ RDDs track the series of transformations used to build
them (the lineage) to recompute lost partitions.
○ Long/complex queries [e.g., complex UDFs]
Handling Task Failures
Task Failure
● Record the failure count of
the task
● Retry the task if failure
count < maxTaskFailures
● Abort the stage and
corresponding jobs if count
>= maxTaskFailures
Fetch Failure
● Don’t count the failure into
task failure count
● Retry the stage if stage failure
< maxStageFailures
● Abort the stage and
corresponding jobs if stage
failure >= maxStageFailures
● Mark executor/host as lost
(optional)
Agenda
46
Memory
Management
Memory Consumption in Executor JVM
Challenges:
• Task run in a shared-memory environment.
• Memory resource is not enough!
Spark uses memory for:
• RDD Storage [e.g., call cache()].
• Execution memory [e.g., Shuffle
and aggregation buffers]
• User code [e.g., allocate large
arrays]
Execution Memory
• Buffer intermediate results
• Normally short lived
Execution Memory
Storage Memory
User Memory
Reserved Memory
Storage Memory
• Reuse data for future
computation
• Cached data can be
long-lived
• LRU eviction for spill
data
Execution Memory
Storage Memory
User Memory
Reserved Memory
Unified Memory Manager
• Express execution and
storage memory as one
single unified region
• Keep acquiring execution
memory and evict storage as
you need more execution
memory
Execution Memory
(1.0 - spark.memory.storageFraction) *
USABLE_MEMORY
Storage Memory
spark.memory.storageFraction *
USABLE_MEMORY
User Memory
(1.0 - spark.memory.fraction) *
(SYSTEM_MEMORY - RESERVED_MEMORY)
Reserved Memory
RESERVED_SYSTEM_MEMORY_BYTES
(300MB)
Dynamic occupancy mechanism
spark.memory.storageFraction
• If one of its space is insufficient but the other is free, then it
will borrow the other’s space.
• If both parties don’t have enough space, evict storage
memory using LRU mechanism.
One problem remains...
• The memory resource is not enough!
On-Heap Memory Off-Heap Memory
Inside JVM
Managed by GC
Outside JVM
Not managed by GC
Executor Process
Off-Heap Memory
• Enabled by spark.memory.offHeap.enabled
• Memory size controlled by
spark.memory.offHeap.size
Execution Memory
Storage Memory
Off-Heap Memory
• Pros
• Speed: Off-Heap Memory > Disk
• Not bound by GC
• Cons
• Manually manage memory allocation/release
Tuning Data Structures
In Spark applications:
• Prefer arrays of objects instead of collection classes
(e.g., HashMap)
• Avoid nested structures with a lot of small objects and
pointers when possible
• Use numeric IDs or enumeration objects instead of strings
for keys
Tuning Memory Config
spark.memory.fraction
• More execution and storage memory
• Higher risk of OOM
spark.memory.storageFraction
• Increase storage memory to cache more data
• Less execution memory may lead to tasks spill more often
Tuning Memory Config
spark.memory.offHeap.enabled
spark.memory.offHeap.size
• Off-Heap memory not bound by GC
• On-Heap + Off-Heap memory must fit in total executor
memory (spark.executor.memory)
spark.shuffle.file.buffer
spark.unsafe.sorter.spill.reader.buffer.size
• Buffer shuffle file to amortize disk I/O
• More execution memory consumption
About Us
Software Engineers
• Maryann Xue
PMC of Apache Calcite & Apache Phoenix @maryannxue
• Xingbo Jiang
Apache Spark Committer @jiangxb1987
• Kris Mok
OpenJDK Committer @rednaxelafx
58
Agenda
59
Vectorized
Reader
Vectorized Readers
Read columnar format data as-is without converting to row
format.
• Apache Parquet
• Apache ORC
• Apache Arrow
• ...
60
Vectorized Readers
Parquet vectorized reader is 9 times faster than the non-
vectorized one.
See blog post
61
Vectorized Readers
Supported built-in data sources:
• Parquet
• ORC
Arrow is used for intermediate data in PySpark.
62
Implement DataSource
DataSource v2 API provides the way to implement your own
vectorized reader.
• PartitionReaderFactory
• supportColumnarReads(...) to return true
• createColumnarReader(...) to return
PartitionReader[ColumnarBatch]
• [SPARK-25186] Stabilize Data Source V2 API
63
Delta Lake
• Full ACID transactions
• Schema management
• Scalable metadata handling
• Data versioning and time travel
• Unified batch/streaming support
• Record update and deletion
• Data expectation
Delta Lake: https://delta.io/
Documentation:
https://docs.delta.io
For details, refer to the blog
https://tinyurl.com/yxhbe2lg
Agenda
65
UDF
What’s behind foo(x) in Spark SQL?
What looks like a function call can be a lot of things:
• upper(str): Built-in function
• max(val): Aggregate function
• max(val) over …: Window function
• explode(arr): Generator
• myudf(x): User-defined function
• myudaf(x): User-defined aggregate function
• transform(arr, x -> x + 1): Higher-order function
• range(10): Table-value function
Functions in Spark SQL
Builtin Scalar
Function
Java / Scala
UDF
Python UDF (*) Aggregate /
Window
Higher-order
Function
Scope 1 Row 1 Row 1 Row Whole table 1 Row
Data Feed Scalar
expressions
Scalar
expressions
Batch of data Scalar expressions
+ aggregate buffer
Expression of
complex type
Process Same JVM Same JVM Python Worker
process
Same JVM Same JVM
Impl. Level Expression Expression Physical Operator Physical Operator Expression
Data Type Internal External External Internal Internal
(*): and all other non-Java user-defined functions
UDF execution
User Defined Functions:
• Java/Scala UDFs
• Hive UDFs
• when Hive support enabled
Also we have:
• Python/Pandas UDFs
• will talk later in PySpark execution
68
Java/Scala UDFs
• UDF: User Defined Function
• Java/Scala lambdas or method references can be used.
• UDAF: User Defined Aggregate Function
• Need to implement UserDefinedAggregateFunction.
69
UDAF
Implement UserDefinedAggregateFunction
• def initialize(...)
• def update(...)
• def merge(...)
• def evaluate(...)
• ...
70
Hive UDFs
Available when Hive support enabled.
• Register using create function command
• Use in HiveQL
71
Hive UDFs
Provides wrapper expressions for each UDF type:
• HiveSimpleUDF: UDF
• HiveGenericUDF: GenericUDF
• HiveUDAFFunction: UDAF
• HiveGenericUDTF: GenericUDTF
72
UDF execution
1. Before invoking UDFs, convert arguments from internal data
format to objects suitable for each UDF types.
• Java/Scala UDF: Java/Scala objects
• Hive UDF: ObjectInspector
2. Invoke the UDF.
3. After invocation, convert the returned values back to internal
data format.
73
Agenda
74
PySpark
PySpark
PySpark is a set of Python bindings for Spark APIs.
• RDD
• DataFrame
• other libraries based on RDDs, DataFrames.
• MLlib, Structured Streaming, ...
Also, SparkR: R bindings for Spark APIs
75
PySpark
RDD vs. DataFrame:
• RDD invokes Python functions on Python worker
• DataFrame just constructs queries, and executes it on the
JVM.
• except for Python/Pandas UDFs
76
PySpark execution
Python script drives Spark on JVM via Py4J.
Executors run Python worker.
77
Python
Driver
Executor Python Worker
Executor Python Worker
Executor Python Worker
PySpark and Pandas
Ease of interop: PySpark can convert data between PySpark
DataFrame and Pandas DataFrame.
• pdf = df.toPandas()
• df = spark.createDataFrame(pdf)
78
Note: df.toPandas() triggers the execution of the PySpark
DataFrame, similar to df.collect()
PySpark and Pandas (cont’d)
New way of interop: Koalas brings the Pandas API to Apache
Spark
79
import databricks.koalas as ks
import pandas as pd
pdf = pd.DataFrame({'x':range(3), 'y':['a','b','b'], 'z':['a','b','b']})
# Create a Koalas DataFrame from pandas DataFrame
df = ks.from_pandas(pdf)
# Rename the columns
df.columns = ['x', 'y', 'z1']
# Do some operations in place:
df['x2'] = df.x * df.x
https://github.com/databricks/koalas
Agenda
80
Python/Pandas
UDF
Python UDF and Pandas UDF
@udf('double')
def plus_one(v):
return v + 1
@pandas_udf('double', PandasUDFType.SCALAR)
def pandas_plus_one(vs):
return vs + 1
81
Python/Pandas UDF execution
82
Invoke UDFPythonRunner
PhysicalOperator
Batch of data Deserializer
SerializerBatch of data
Python UDF execution
83
Invoke UDFPythonUDFRunner
PhysicalOperator
Batch of Rows
Batch of Rows
Deserializer
Serializer
Pandas UDF execution
84
Invoke UDFArrowPythonRunner
PhysicalOperator
Batch of Columns
Batch of Columns
Deserializer
Serializer
Python/Pandas UDFs
Python UDF
• Serialize/Deserialize data with Pickle
• Fetch data in blocks, but invoke UDF row by row
Pandas UDF
• Serialize/Deserialize data with Arrow
• Fetch data in blocks, and invoke UDF block by block
85
Python/Pandas UDFs
Pandas UDF perform much better than row-at-a-time Python
UDFs.
• 3x to over 100x
See blog post
86
Further Reading
This Spark+AI Summit:
• Understanding Query Plans and Spark
Previous Spark Summits:
• A Deep Dive into Spark SQL’s Catalyst Optimizer
• Deep Dive into Project Tungsten: Bringing Spark Closer to
Bare Metal
• Improving Python and Spark Performance and
Interoperability with Apache Arrow
Thank you
Maryann Xue (maryann.xue@databricks.com)
Xingbo Jiang (xingbo.jiang@databricks.com)
Kris Mok (kris.mok@databricks.com)
88

More Related Content

What's hot

What's hot (20)

Spark shuffle introduction
Spark shuffle introductionSpark shuffle introduction
Spark shuffle introduction
 
Parquet performance tuning: the missing guide
Parquet performance tuning: the missing guideParquet performance tuning: the missing guide
Parquet performance tuning: the missing guide
 
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
 
Tame the small files problem and optimize data layout for streaming ingestion...
Tame the small files problem and optimize data layout for streaming ingestion...Tame the small files problem and optimize data layout for streaming ingestion...
Tame the small files problem and optimize data layout for streaming ingestion...
 
Enabling Vectorized Engine in Apache Spark
Enabling Vectorized Engine in Apache SparkEnabling Vectorized Engine in Apache Spark
Enabling Vectorized Engine in Apache Spark
 
Programming in Spark using PySpark
Programming in Spark using PySpark      Programming in Spark using PySpark
Programming in Spark using PySpark
 
Delta Lake: Optimizing Merge
Delta Lake: Optimizing MergeDelta Lake: Optimizing Merge
Delta Lake: Optimizing Merge
 
Optimizing Apache Spark SQL Joins
Optimizing Apache Spark SQL JoinsOptimizing Apache Spark SQL Joins
Optimizing Apache Spark SQL Joins
 
The Apache Spark File Format Ecosystem
The Apache Spark File Format EcosystemThe Apache Spark File Format Ecosystem
The Apache Spark File Format Ecosystem
 
Apache Spark Overview
Apache Spark OverviewApache Spark Overview
Apache Spark Overview
 
Everyday I'm Shuffling - Tips for Writing Better Spark Programs, Strata San J...
Everyday I'm Shuffling - Tips for Writing Better Spark Programs, Strata San J...Everyday I'm Shuffling - Tips for Writing Better Spark Programs, Strata San J...
Everyday I'm Shuffling - Tips for Writing Better Spark Programs, Strata San J...
 
Dive into PySpark
Dive into PySparkDive into PySpark
Dive into PySpark
 
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
 
Evening out the uneven: dealing with skew in Flink
Evening out the uneven: dealing with skew in FlinkEvening out the uneven: dealing with skew in Flink
Evening out the uneven: dealing with skew in Flink
 
Spark Shuffle Deep Dive (Explained In Depth) - How Shuffle Works in Spark
Spark Shuffle Deep Dive (Explained In Depth) - How Shuffle Works in SparkSpark Shuffle Deep Dive (Explained In Depth) - How Shuffle Works in Spark
Spark Shuffle Deep Dive (Explained In Depth) - How Shuffle Works in Spark
 
Understanding and Improving Code Generation
Understanding and Improving Code GenerationUnderstanding and Improving Code Generation
Understanding and Improving Code Generation
 
Productizing Structured Streaming Jobs
Productizing Structured Streaming JobsProductizing Structured Streaming Jobs
Productizing Structured Streaming Jobs
 
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
 
Processing Large Data with Apache Spark -- HasGeek
Processing Large Data with Apache Spark -- HasGeekProcessing Large Data with Apache Spark -- HasGeek
Processing Large Data with Apache Spark -- HasGeek
 
Improving SparkSQL Performance by 30%: How We Optimize Parquet Pushdown and P...
Improving SparkSQL Performance by 30%: How We Optimize Parquet Pushdown and P...Improving SparkSQL Performance by 30%: How We Optimize Parquet Pushdown and P...
Improving SparkSQL Performance by 30%: How We Optimize Parquet Pushdown and P...
 

Similar to A Deep Dive into Query Execution Engine of Spark SQL

ADCSS 2022
ADCSS 2022ADCSS 2022
ql.io: Consuming HTTP at Scale
ql.io: Consuming HTTP at Scale ql.io: Consuming HTTP at Scale
ql.io: Consuming HTTP at Scale
Subbu Allamaraju
 

Similar to A Deep Dive into Query Execution Engine of Spark SQL (20)

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
 
Presto anatomy
Presto anatomyPresto anatomy
Presto anatomy
 
Apache Spark 2.0: A Deep Dive Into Structured Streaming - by Tathagata Das
Apache Spark 2.0: A Deep Dive Into Structured Streaming - by Tathagata Das Apache Spark 2.0: A Deep Dive Into Structured Streaming - by Tathagata Das
Apache Spark 2.0: A Deep Dive Into Structured Streaming - by Tathagata Das
 
[245] presto 내부구조 파헤치기
[245] presto 내부구조 파헤치기[245] presto 내부구조 파헤치기
[245] presto 내부구조 파헤치기
 
Spark SQL Catalyst Code Optimization using Function Outlining with Kavana Bha...
Spark SQL Catalyst Code Optimization using Function Outlining with Kavana Bha...Spark SQL Catalyst Code Optimization using Function Outlining with Kavana Bha...
Spark SQL Catalyst Code Optimization using Function Outlining with Kavana Bha...
 
RxJava applied [JavaDay Kyiv 2016]
RxJava applied [JavaDay Kyiv 2016]RxJava applied [JavaDay Kyiv 2016]
RxJava applied [JavaDay Kyiv 2016]
 
Introduction to Structured Streaming
Introduction to Structured StreamingIntroduction to Structured Streaming
Introduction to Structured Streaming
 
Lambdas puzzler - Peter Lawrey
Lambdas puzzler - Peter LawreyLambdas puzzler - Peter Lawrey
Lambdas puzzler - Peter Lawrey
 
GDG Jakarta Meetup - Streaming Analytics With Apache Beam
GDG Jakarta Meetup - Streaming Analytics With Apache BeamGDG Jakarta Meetup - Streaming Analytics With Apache Beam
GDG Jakarta Meetup - Streaming Analytics With Apache Beam
 
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
 
Server side JavaScript: going all the way
Server side JavaScript: going all the wayServer side JavaScript: going all the way
Server side JavaScript: going all the way
 
Profiling & Testing with Spark
Profiling & Testing with SparkProfiling & Testing with Spark
Profiling & Testing with Spark
 
ADCSS 2022
ADCSS 2022ADCSS 2022
ADCSS 2022
 
Exploiting GPU's for Columnar DataFrrames by Kiran Lonikar
Exploiting GPU's for Columnar DataFrrames by Kiran LonikarExploiting GPU's for Columnar DataFrrames by Kiran Lonikar
Exploiting GPU's for Columnar DataFrrames by Kiran Lonikar
 
Second Level Cache in JPA Explained
Second Level Cache in JPA ExplainedSecond Level Cache in JPA Explained
Second Level Cache in JPA Explained
 
A Deep Dive into Structured Streaming in Apache Spark
A Deep Dive into Structured Streaming in Apache Spark A Deep Dive into Structured Streaming in Apache Spark
A Deep Dive into Structured Streaming in Apache Spark
 
Apache Spark 2.0: Faster, Easier, and Smarter
Apache Spark 2.0: Faster, Easier, and SmarterApache Spark 2.0: Faster, Easier, and Smarter
Apache Spark 2.0: Faster, Easier, and Smarter
 
ql.io: Consuming HTTP at Scale
ql.io: Consuming HTTP at Scale ql.io: Consuming HTTP at Scale
ql.io: Consuming HTTP at Scale
 
Spark (Structured) Streaming vs. Kafka Streams - two stream processing platfo...
Spark (Structured) Streaming vs. Kafka Streams - two stream processing platfo...Spark (Structured) Streaming vs. Kafka Streams - two stream processing platfo...
Spark (Structured) Streaming vs. Kafka Streams - two stream processing platfo...
 
Fast and Reliable Apache Spark SQL Engine
Fast and Reliable Apache Spark SQL EngineFast and Reliable Apache Spark SQL Engine
Fast and Reliable Apache Spark SQL Engine
 

More from Databricks

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
Databricks
 
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
Databricks
 
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
Databricks
 
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
Databricks
 
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
Databricks
 

More from Databricks (20)

DW Migration Webinar-March 2022.pptx
DW Migration Webinar-March 2022.pptxDW Migration Webinar-March 2022.pptx
DW Migration Webinar-March 2022.pptx
 
Data Lakehouse Symposium | Day 1 | Part 1
Data Lakehouse Symposium | Day 1 | Part 1Data Lakehouse Symposium | Day 1 | Part 1
Data Lakehouse Symposium | Day 1 | Part 1
 
Data Lakehouse Symposium | Day 1 | Part 2
Data Lakehouse Symposium | Day 1 | Part 2Data Lakehouse Symposium | Day 1 | Part 2
Data Lakehouse Symposium | Day 1 | Part 2
 
Data Lakehouse Symposium | Day 2
Data Lakehouse Symposium | Day 2Data Lakehouse Symposium | Day 2
Data Lakehouse Symposium | Day 2
 
Data Lakehouse Symposium | Day 4
Data Lakehouse Symposium | Day 4Data Lakehouse Symposium | Day 4
Data Lakehouse Symposium | Day 4
 
5 Critical Steps to Clean Your Data Swamp When Migrating Off of Hadoop
5 Critical Steps to Clean Your Data Swamp When Migrating Off of Hadoop5 Critical Steps to Clean Your Data Swamp When Migrating Off of Hadoop
5 Critical Steps to Clean Your Data Swamp When Migrating Off of Hadoop
 
Democratizing Data Quality Through a Centralized Platform
Democratizing Data Quality Through a Centralized PlatformDemocratizing Data Quality Through a Centralized Platform
Democratizing Data Quality Through a Centralized Platform
 
Learn to Use Databricks for Data Science
Learn to Use Databricks for Data ScienceLearn to Use Databricks for Data Science
Learn to Use Databricks for Data Science
 
Why APM Is Not the Same As ML Monitoring
Why APM Is Not the Same As ML MonitoringWhy APM Is Not the Same As ML Monitoring
Why APM Is Not the Same As ML Monitoring
 
The Function, the Context, and the Data—Enabling ML Ops at Stitch Fix
The Function, the Context, and the Data—Enabling ML Ops at Stitch FixThe Function, the Context, and the Data—Enabling ML Ops at Stitch Fix
The Function, the Context, and the Data—Enabling ML Ops at Stitch Fix
 
Stage Level Scheduling Improving Big Data and AI Integration
Stage Level Scheduling Improving Big Data and AI IntegrationStage Level Scheduling Improving Big Data and AI Integration
Stage Level Scheduling Improving Big Data and AI Integration
 
Simplify Data Conversion from Spark to TensorFlow and PyTorch
Simplify Data Conversion from Spark to TensorFlow and PyTorchSimplify Data Conversion from Spark to TensorFlow and PyTorch
Simplify Data Conversion from Spark to TensorFlow and PyTorch
 
Scaling your Data Pipelines with Apache Spark on Kubernetes
Scaling your Data Pipelines with Apache Spark on KubernetesScaling your Data Pipelines with Apache Spark on Kubernetes
Scaling your Data Pipelines with Apache Spark on Kubernetes
 
Scaling and Unifying SciKit Learn and Apache Spark Pipelines
Scaling and Unifying SciKit Learn and Apache Spark PipelinesScaling and Unifying SciKit Learn and Apache Spark Pipelines
Scaling and Unifying SciKit Learn and Apache Spark Pipelines
 
Sawtooth Windows for Feature Aggregations
Sawtooth Windows for Feature AggregationsSawtooth Windows for Feature Aggregations
Sawtooth Windows for Feature Aggregations
 
Redis + Apache Spark = Swiss Army Knife Meets Kitchen Sink
Redis + Apache Spark = Swiss Army Knife Meets Kitchen SinkRedis + Apache Spark = Swiss Army Knife Meets Kitchen Sink
Redis + Apache Spark = Swiss Army Knife Meets Kitchen Sink
 
Re-imagine Data Monitoring with whylogs and Spark
Re-imagine Data Monitoring with whylogs and SparkRe-imagine Data Monitoring with whylogs and Spark
Re-imagine Data Monitoring with whylogs and Spark
 
Raven: End-to-end Optimization of ML Prediction Queries
Raven: End-to-end Optimization of ML Prediction QueriesRaven: End-to-end Optimization of ML Prediction Queries
Raven: End-to-end Optimization of ML Prediction Queries
 
Processing Large Datasets for ADAS Applications using Apache Spark
Processing Large Datasets for ADAS Applications using Apache SparkProcessing Large Datasets for ADAS Applications using Apache Spark
Processing Large Datasets for ADAS Applications using Apache Spark
 
Massive Data Processing in Adobe Using Delta Lake
Massive Data Processing in Adobe Using Delta LakeMassive Data Processing in Adobe Using Delta Lake
Massive Data Processing in Adobe Using Delta Lake
 

Recently uploaded

➥🔝 7737669865 🔝▻ Dindigul Call-girls in Women Seeking Men 🔝Dindigul🔝 Escor...
➥🔝 7737669865 🔝▻ Dindigul Call-girls in Women Seeking Men  🔝Dindigul🔝   Escor...➥🔝 7737669865 🔝▻ Dindigul Call-girls in Women Seeking Men  🔝Dindigul🔝   Escor...
➥🔝 7737669865 🔝▻ Dindigul Call-girls in Women Seeking Men 🔝Dindigul🔝 Escor...
amitlee9823
 
Call Girls In Shalimar Bagh ( Delhi) 9953330565 Escorts Service
Call Girls In Shalimar Bagh ( Delhi) 9953330565 Escorts ServiceCall Girls In Shalimar Bagh ( Delhi) 9953330565 Escorts Service
Call Girls In Shalimar Bagh ( Delhi) 9953330565 Escorts Service
9953056974 Low Rate Call Girls In Saket, Delhi NCR
 
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
9953056974 Low Rate Call Girls In Saket, Delhi NCR
 
➥🔝 7737669865 🔝▻ Mathura Call-girls in Women Seeking Men 🔝Mathura🔝 Escorts...
➥🔝 7737669865 🔝▻ Mathura Call-girls in Women Seeking Men  🔝Mathura🔝   Escorts...➥🔝 7737669865 🔝▻ Mathura Call-girls in Women Seeking Men  🔝Mathura🔝   Escorts...
➥🔝 7737669865 🔝▻ Mathura Call-girls in Women Seeking Men 🔝Mathura🔝 Escorts...
amitlee9823
 
Call Girls Indiranagar Just Call 👗 9155563397 👗 Top Class Call Girl Service B...
Call Girls Indiranagar Just Call 👗 9155563397 👗 Top Class Call Girl Service B...Call Girls Indiranagar Just Call 👗 9155563397 👗 Top Class Call Girl Service B...
Call Girls Indiranagar Just Call 👗 9155563397 👗 Top Class Call Girl Service B...
only4webmaster01
 
Abortion pills in Jeddah | +966572737505 | Get Cytotec
Abortion pills in Jeddah | +966572737505 | Get CytotecAbortion pills in Jeddah | +966572737505 | Get Cytotec
Abortion pills in Jeddah | +966572737505 | Get Cytotec
Abortion pills in Riyadh +966572737505 get cytotec
 
➥🔝 7737669865 🔝▻ Thrissur Call-girls in Women Seeking Men 🔝Thrissur🔝 Escor...
➥🔝 7737669865 🔝▻ Thrissur Call-girls in Women Seeking Men  🔝Thrissur🔝   Escor...➥🔝 7737669865 🔝▻ Thrissur Call-girls in Women Seeking Men  🔝Thrissur🔝   Escor...
➥🔝 7737669865 🔝▻ Thrissur Call-girls in Women Seeking Men 🔝Thrissur🔝 Escor...
amitlee9823
 
Call Girls Bannerghatta Road Just Call 👗 7737669865 👗 Top Class Call Girl Ser...
Call Girls Bannerghatta Road Just Call 👗 7737669865 👗 Top Class Call Girl Ser...Call Girls Bannerghatta Road Just Call 👗 7737669865 👗 Top Class Call Girl Ser...
Call Girls Bannerghatta Road Just Call 👗 7737669865 👗 Top Class Call Girl Ser...
amitlee9823
 
Call Girls In Nandini Layout ☎ 7737669865 🥵 Book Your One night Stand
Call Girls In Nandini Layout ☎ 7737669865 🥵 Book Your One night StandCall Girls In Nandini Layout ☎ 7737669865 🥵 Book Your One night Stand
Call Girls In Nandini Layout ☎ 7737669865 🥵 Book Your One night Stand
amitlee9823
 
👉 Amritsar Call Girl 👉📞 6367187148 👉📞 Just📲 Call Ruhi Call Girl Phone No Amri...
👉 Amritsar Call Girl 👉📞 6367187148 👉📞 Just📲 Call Ruhi Call Girl Phone No Amri...👉 Amritsar Call Girl 👉📞 6367187148 👉📞 Just📲 Call Ruhi Call Girl Phone No Amri...
👉 Amritsar Call Girl 👉📞 6367187148 👉📞 Just📲 Call Ruhi Call Girl Phone No Amri...
karishmasinghjnh
 
➥🔝 7737669865 🔝▻ mahisagar Call-girls in Women Seeking Men 🔝mahisagar🔝 Esc...
➥🔝 7737669865 🔝▻ mahisagar Call-girls in Women Seeking Men  🔝mahisagar🔝   Esc...➥🔝 7737669865 🔝▻ mahisagar Call-girls in Women Seeking Men  🔝mahisagar🔝   Esc...
➥🔝 7737669865 🔝▻ mahisagar Call-girls in Women Seeking Men 🔝mahisagar🔝 Esc...
amitlee9823
 
➥🔝 7737669865 🔝▻ Bangalore Call-girls in Women Seeking Men 🔝Bangalore🔝 Esc...
➥🔝 7737669865 🔝▻ Bangalore Call-girls in Women Seeking Men  🔝Bangalore🔝   Esc...➥🔝 7737669865 🔝▻ Bangalore Call-girls in Women Seeking Men  🔝Bangalore🔝   Esc...
➥🔝 7737669865 🔝▻ Bangalore Call-girls in Women Seeking Men 🔝Bangalore🔝 Esc...
amitlee9823
 
Call Girls In Doddaballapur Road ☎ 7737669865 🥵 Book Your One night Stand
Call Girls In Doddaballapur Road ☎ 7737669865 🥵 Book Your One night StandCall Girls In Doddaballapur Road ☎ 7737669865 🥵 Book Your One night Stand
Call Girls In Doddaballapur Road ☎ 7737669865 🥵 Book Your One night Stand
amitlee9823
 

Recently uploaded (20)

(NEHA) Call Girls Katra Call Now 8617697112 Katra Escorts 24x7
(NEHA) Call Girls Katra Call Now 8617697112 Katra Escorts 24x7(NEHA) Call Girls Katra Call Now 8617697112 Katra Escorts 24x7
(NEHA) Call Girls Katra Call Now 8617697112 Katra Escorts 24x7
 
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
 
➥🔝 7737669865 🔝▻ Dindigul Call-girls in Women Seeking Men 🔝Dindigul🔝 Escor...
➥🔝 7737669865 🔝▻ Dindigul Call-girls in Women Seeking Men  🔝Dindigul🔝   Escor...➥🔝 7737669865 🔝▻ Dindigul Call-girls in Women Seeking Men  🔝Dindigul🔝   Escor...
➥🔝 7737669865 🔝▻ Dindigul Call-girls in Women Seeking Men 🔝Dindigul🔝 Escor...
 
Call Girls In Shalimar Bagh ( Delhi) 9953330565 Escorts Service
Call Girls In Shalimar Bagh ( Delhi) 9953330565 Escorts ServiceCall Girls In Shalimar Bagh ( Delhi) 9953330565 Escorts Service
Call Girls In Shalimar Bagh ( Delhi) 9953330565 Escorts Service
 
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
 
➥🔝 7737669865 🔝▻ Mathura Call-girls in Women Seeking Men 🔝Mathura🔝 Escorts...
➥🔝 7737669865 🔝▻ Mathura Call-girls in Women Seeking Men  🔝Mathura🔝   Escorts...➥🔝 7737669865 🔝▻ Mathura Call-girls in Women Seeking Men  🔝Mathura🔝   Escorts...
➥🔝 7737669865 🔝▻ Mathura Call-girls in Women Seeking Men 🔝Mathura🔝 Escorts...
 
Call Girls Indiranagar Just Call 👗 9155563397 👗 Top Class Call Girl Service B...
Call Girls Indiranagar Just Call 👗 9155563397 👗 Top Class Call Girl Service B...Call Girls Indiranagar Just Call 👗 9155563397 👗 Top Class Call Girl Service B...
Call Girls Indiranagar Just Call 👗 9155563397 👗 Top Class Call Girl Service B...
 
Abortion pills in Jeddah | +966572737505 | Get Cytotec
Abortion pills in Jeddah | +966572737505 | Get CytotecAbortion pills in Jeddah | +966572737505 | Get Cytotec
Abortion pills in Jeddah | +966572737505 | Get Cytotec
 
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
 
➥🔝 7737669865 🔝▻ Thrissur Call-girls in Women Seeking Men 🔝Thrissur🔝 Escor...
➥🔝 7737669865 🔝▻ Thrissur Call-girls in Women Seeking Men  🔝Thrissur🔝   Escor...➥🔝 7737669865 🔝▻ Thrissur Call-girls in Women Seeking Men  🔝Thrissur🔝   Escor...
➥🔝 7737669865 🔝▻ Thrissur Call-girls in Women Seeking Men 🔝Thrissur🔝 Escor...
 
Call Girls Bannerghatta Road Just Call 👗 7737669865 👗 Top Class Call Girl Ser...
Call Girls Bannerghatta Road Just Call 👗 7737669865 👗 Top Class Call Girl Ser...Call Girls Bannerghatta Road Just Call 👗 7737669865 👗 Top Class Call Girl Ser...
Call Girls Bannerghatta Road Just Call 👗 7737669865 👗 Top Class Call Girl Ser...
 
Call Girls In Nandini Layout ☎ 7737669865 🥵 Book Your One night Stand
Call Girls In Nandini Layout ☎ 7737669865 🥵 Book Your One night StandCall Girls In Nandini Layout ☎ 7737669865 🥵 Book Your One night Stand
Call Girls In Nandini Layout ☎ 7737669865 🥵 Book Your One night Stand
 
Detecting Credit Card Fraud: A Machine Learning Approach
Detecting Credit Card Fraud: A Machine Learning ApproachDetecting Credit Card Fraud: A Machine Learning Approach
Detecting Credit Card Fraud: A Machine Learning Approach
 
Capstone Project on IBM Data Analytics Program
Capstone Project on IBM Data Analytics ProgramCapstone Project on IBM Data Analytics Program
Capstone Project on IBM Data Analytics Program
 
👉 Amritsar Call Girl 👉📞 6367187148 👉📞 Just📲 Call Ruhi Call Girl Phone No Amri...
👉 Amritsar Call Girl 👉📞 6367187148 👉📞 Just📲 Call Ruhi Call Girl Phone No Amri...👉 Amritsar Call Girl 👉📞 6367187148 👉📞 Just📲 Call Ruhi Call Girl Phone No Amri...
👉 Amritsar Call Girl 👉📞 6367187148 👉📞 Just📲 Call Ruhi Call Girl Phone No Amri...
 
➥🔝 7737669865 🔝▻ mahisagar Call-girls in Women Seeking Men 🔝mahisagar🔝 Esc...
➥🔝 7737669865 🔝▻ mahisagar Call-girls in Women Seeking Men  🔝mahisagar🔝   Esc...➥🔝 7737669865 🔝▻ mahisagar Call-girls in Women Seeking Men  🔝mahisagar🔝   Esc...
➥🔝 7737669865 🔝▻ mahisagar Call-girls in Women Seeking Men 🔝mahisagar🔝 Esc...
 
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
 
➥🔝 7737669865 🔝▻ Bangalore Call-girls in Women Seeking Men 🔝Bangalore🔝 Esc...
➥🔝 7737669865 🔝▻ Bangalore Call-girls in Women Seeking Men  🔝Bangalore🔝   Esc...➥🔝 7737669865 🔝▻ Bangalore Call-girls in Women Seeking Men  🔝Bangalore🔝   Esc...
➥🔝 7737669865 🔝▻ Bangalore Call-girls in Women Seeking Men 🔝Bangalore🔝 Esc...
 
Call Girls In Doddaballapur Road ☎ 7737669865 🥵 Book Your One night Stand
Call Girls In Doddaballapur Road ☎ 7737669865 🥵 Book Your One night StandCall Girls In Doddaballapur Road ☎ 7737669865 🥵 Book Your One night Stand
Call Girls In Doddaballapur Road ☎ 7737669865 🥵 Book Your One night Stand
 
Midocean dropshipping via API with DroFx
Midocean dropshipping via API with DroFxMidocean dropshipping via API with DroFx
Midocean dropshipping via API with DroFx
 

A Deep Dive into Query Execution Engine of Spark SQL

  • 1. Deep Dive: Query Execution of Spark SQL Maryann Xue, Xingbo Jiang, Kris Mok Apr. 2019 1
  • 2. About Us Software Engineers • Maryann Xue PMC of Apache Calcite & Apache Phoenix @maryannxue • Xingbo Jiang Apache Spark Committer @jiangxb1987 • Kris Mok OpenJDK Committer @rednaxelafx 2
  • 3. DATABRICKS WORKSPACE Databricks Delta ML Frameworks DATABRICKS CLOUD SERVICE DATABRICKS RUNTIME Reliable & Scalable Simple & Integrated Databricks Unified Analytics Platform APIs Jobs Models Notebooks Dashboards End to end ML lifecycle
  • 4. Databricks Customers Across Industries Financial Services Healthcare & Pharma Media & Entertainment Technology Public Sector Retail & CPG Consumer Services Energy & Industrial IoTMarketing & AdTech Data & Analytics Services
  • 5. Apache Spark 3.x 5 Catalyst Optimization & Tungsten Execution SparkSession / DataFrame / Dataset APIs SQL Spark ML Spark Streaming Spark Graph 3rd-party Libraries Spark CoreData Source Connectors
  • 6. Apache Spark 3.x 6 Catalyst Optimization & Tungsten Execution SparkSession / DataFrame / Dataset APIs SQL Spark ML Spark Streaming Spark Graph 3rd-party Libraries Spark CoreData Source Connectors
  • 7. Spark SQL Engine 7 Analysis -> Logical Optimization -> Physical Planning -> Code Generation -> Execution Runtime
  • 8. Spark SQL Engine - Front End 8 Analysis -> Logical Optimization -> Physical Planning -> Code Generation -> Execution Reference: A Deep Dive into Spark SQL’s Catalyst Optimizer, Yin Huai, Spark Summit 2017 Runtime
  • 9. Spark SQL Engine - Back End 9 Analysis -> Logical Optimization -> Physical Planning -> Code Generation -> Execution Runtime
  • 12. • Transform logical operators into physical operators • Choose between different physical alternatives - e.g., broadcast-hash-join vs. sort-merge-join • Includes physical traits of the execution engine - e.g., partitioning & ordering. • Some ops may be mapped into multiple physical nodes - e.g., partial agg —> shuffle —> final agg Physical Planning 1
  • 13. A Physical Plan Example 1 Scan A Scan B Filter BroadcastExchange BroadcastHashJoin HashAggregate ShuffleExchange HashAggregate SELECT a1, sum(b1)FROM A JOIN B ON A.key = B.key WHERE b1 < 1000 GROUP BY a1 Scan A Filter Join Aggregate Scan B
  • 14. • Scalar subquery Broadcast exchange: - Executed as separate jobs • Partition-local ops: - Executed in the same stage • Shuffle: - The stage boundary - A sync barrier across all nodes Scheduling a Physical Plan 1 Job 2 Stage 1 Stage 2 Scan A BroadcastHashJoin HashAggregate ShuffleExchange HashAggregate Job 1 Stage 1 Scan B Filter BroadcastExchange
  • 16. Execution, Old: Volcano Iterator Model • Volcano iterator model - All ops implement the same interface, e.g., next() - next() on final op -> pull input from child op by calling child.next() -> goes on and on, ending up with a propagation of next() calls • Pros: Good abstraction; Easy to implement • Cons: Virtual function calls —> less efficient 1 Scan Filter Project Result Iterator iterate next()next()next()
  • 17. Execution, New: Whole-Stage Code Generation • Inspired by Thomas Neumann’s paper • Fuse a string of operators (oftentimes the entire stage) into one WSCG op that runs the generated code. • A general-purpose execution engine just like Volcano model but without Volcano’s performance downsides: - No virtual function calls - Data in CPU registers - Loop unrolling & SIMD Scan Filter Project Aggregate long count = 0; for (item in sales) { if (price < 100) { count += 1; } }
  • 18. Execution Models: Old vs. New • Volcano iterator model: Pull model; Driven by the final operator 1 Scan Filter Project Result Iterator iterate next()next()next() • WSCG model: Push model; Driven by the head/source operator Scan Filter Project Result Iterator iterate next()
  • 19. A Physical Plan Example - WSCG Job 2 Stage 1 Stage 2 WSCG Scan A BroadcastHashJoin WSCG HashAggregate ShuffleExchange HashAggregate Job 1 Stage 1 WSCG Scan B Filter BroadcastExchange
  • 20. Implementation • The top node WholeStageCodegenExec implements the iterator interface to interop with other code-gen or non-code-gen physical ops. • All underlying operators implement a code-generation interface: doProduce() & doConsume() • Dump the generated code: df.queryExecution.debug.codegen
  • 21. WSCG Pipeline 2Pipeline 1 Single dependency • A WSCG node contains a linear list of physical operators that support code generation. • No multi dependency between enclosed ops. • A WSCG node may consist of one or more pipelines. Op1 Op2 Op3 Op4 Op5
  • 22. A Single Pipeline in WSCG • A string of non-blocking operators form a pipeline in WSCG • The head/source: - Implement doProduce() - the driving loop producing source data. • The rest: - doProduce() - fall through to head of the pipeline. - Implement doConsume() for its own processing logic. Op1 Op2 Op3 WSCG consume consume consume produceproduceproduce produce Generate Code
  • 23. A Single Pipeline Example while (table.hasNext()) { InternalRow row = table.next(); if (shouldStop()) return; } Scan Filter Project Generated for RowIterator WholeStageCodegen SELECT sid FROM emps WHERE age < 36 START: produce produce produce produce
  • 24. A Single Pipeline Example while (table.hasNext()) { InternalRow row = table.next(); if (row.getInt(2) < 36) { } if (shouldStop()) return; } Scan Filter Project Generated for RowIterator WholeStageCodegen SELECT sid FROM emps WHERE age < 36 START: produce consume produce produce produce
  • 25. A Single Pipeline Example while (table.hasNext()) { InternalRow row = table.next(); if (row.getInt(2) < 36) { String sid = row.getString(0); rowWriter.write(0, sid); } if (shouldStop()) return; } Scan Filter Project Generated for RowIterator WholeStageCodegen SELECT sid FROM emps WHERE age < 36 START: produce consume consume produce produce produce
  • 26. A Single Pipeline Example while (table.hasNext()) { InternalRow row = table.next(); if (row.getInt(2) < 36) { String sid = row.getString(0); rowWriter.write(0, sid); ret = rowWriter.getRow(); } if (shouldStop()) return; } Scan Filter Project Generated for RowIterator WholeStageCodegen SELECT sid FROM emps WHERE age < 36 START: produce consume consume consumeproduce produce produce
  • 27. • Head (source) operator: - The source, w/ or w/o input RDDs - e.g., Scan, SortMergeJoin • Non-blocking operators: - In the middle of the pipeline - e.g., Filter, Project Multiple Pipelines in WSCG • End (sink): RowIterator - Pulls result from the last pipeline • Blocking operators: - End of the previous pipeline - Start of a new pipeline - e.g., HashAggregate, Sort WSCG Pipeline 2Pipeline 1 Op1 Op2 Op3 Op4 Op5 non-blockingsource non-blockingblocking RowIterator sink
  • 28. Blocking Operators in WSCG • A Blocking operator, e.g., HashAggregateExec, SortExec, break pipelines, so there may be multiple pipelines in one WSCG node. • A Blocking operator’s doConsume(): - Implement the callback to build intermediate result. • A Blocking operator’s doProduce(): - Consume the entire output from upstream to finish building the intermediate result. - Start a new loop and produce output for downstream based on the intermediate result.
  • 29. A Blocking Operator Example - HashAgg while (table.hasNext()) { InternalRow row = table.next(); int age = row.getInt(2); hashMap.insertOrIncrement(sid); } HashAggregate doProduce() child.produce() HashAggregate Scan WholeStageCodegen SELECT age, count(*) FROM emps GROUP BY age consume START: produce produce
  • 30. A Blocking Operator Example - HashAgg while (table.hasNext()) { InternalRow row = table.next(); int age = row.getInt(2); hashMap.insertOrIncrement(sid); } while (hashMapIter.hasNext()) { Entry e = hashMapIter.next(); rowWriter.write(0, e.getKey()); rowWriter.write(1, e.getValue()); ret = rowWriter.getRow(); if (shouldStop()) return; } HashAggregate doProduce() child.produce() HashAggregate Scan start a new pipeline WholeStageCodegen SELECT age, count(*) FROM emps GROUP BY age consume consume START: produce produce
  • 31. • BHJ (broadcast-hash-join) is a pipelined operator. • BHJ executes the build side job first, the same way as in non-WSCG. • BHJ is fused together with the probe side plan (i.e., streaming plan) in WSCG. WSCG: BHJ vs. SMJ Job 2 WSCG Scan A BroadcastHashJoin WSCG HashAggregate ShuffleExchange HashAggregate Job 1 WSCG Scan B Filter BroadcastExchange
  • 32. Job 1 • SMJ (sort-merge-join) is NOT fused with either child plan for WSCG. Child plans are separate WSCG nodes. • Thus, SMJ must be the head operator of a WSCG node. WSCG: BHJ vs. SMJ WSCG SortMergeJoin WSCG HashAggregate ShuffleExchange HashAggregate WSCG Scan B Filter WSCG Scan A WSCG Sort ShuffleExchange WSCG Sort ShuffleExchange
  • 33. WSCG Limitations • Problems: - No JIT compilation for bytecode size over 8000 bytes (*). - Over 64KB methods NOT allowed by Java Class format. • Solutions: - Fallback - spark.sql.codegen.fallback; spark.sql.codegen.hugeMethodLimit - Move blocking loops into separate methods, e.g. hash-map building in HashAgg and sort buffer building in Sort. - Split consume() into individual methods for each operator - spark.sql.codegen.splitConsumeFuncByOperator
  • 34. About Us Software Engineers • Maryann Xue PMC of Apache Calcite & Apache Phoenix @maryannxue • Xingbo Jiang Apache Spark Committer @jiangxb1987 • Kris Mok OpenJDK Committer @rednaxelafx 34
  • 36. A Physical Plan Example SELECT a1, sum(b1) FROM A JOIN B ON A.key = B.key WHERE b1 < 1000 GROUP BY a1 Scan A Scan B Filter BroadcastExchange BroadcastHashJoin HashAggregate ShuffleExchange HashAggregate
  • 37. RDD and Partitions RDD(Resilient Distributed Dataset) represents an immutable, partitioned collection of elements that can be operated in parallel. Partition Partition Partition Node1 Node2 Node3 RDD
  • 38. Physical Operator Filter Volcano iterator model while (iter.hasNext()) { val tmpVal = iter.next() if (condition(tmpVal)) { return tmpVal } } output Partition Partition Partition RDD
  • 39. Job 2 Stage 1 Stage 2 A Physical Plan Example - Scheduling Scan A BroadcastHashJoin HashAggregate ShuffleExchange HashAggregate Job 1 Stage 1 Scan B Filter BroadcastExchange
  • 40. Stage 1 Stage Execution TaskSet0 Scan A BroadcastHashJoin HashAggregate Partition0 Partition1 Partition2 Partition3
  • 43. How to run a Task Executor(spark.executor.cores=5) Task0 spark.task.cpus=1 Task2 Task6 Task1 Task5 Task7 Task3 Task4
  • 44. Fault Tolerance ● MPP-like analytics engines(e.g., Teradata, Presto, Impala): ○ Coarser-grained recovery model ○ Retry an entire query if any machine fails ○ Short/simple queries ● Spark SQL: ○ Mid-query recovery model ○ RDDs track the series of transformations used to build them (the lineage) to recompute lost partitions. ○ Long/complex queries [e.g., complex UDFs]
  • 45. Handling Task Failures Task Failure ● Record the failure count of the task ● Retry the task if failure count < maxTaskFailures ● Abort the stage and corresponding jobs if count >= maxTaskFailures Fetch Failure ● Don’t count the failure into task failure count ● Retry the stage if stage failure < maxStageFailures ● Abort the stage and corresponding jobs if stage failure >= maxStageFailures ● Mark executor/host as lost (optional)
  • 47. Memory Consumption in Executor JVM Challenges: • Task run in a shared-memory environment. • Memory resource is not enough! Spark uses memory for: • RDD Storage [e.g., call cache()]. • Execution memory [e.g., Shuffle and aggregation buffers] • User code [e.g., allocate large arrays]
  • 48. Execution Memory • Buffer intermediate results • Normally short lived Execution Memory Storage Memory User Memory Reserved Memory
  • 49. Storage Memory • Reuse data for future computation • Cached data can be long-lived • LRU eviction for spill data Execution Memory Storage Memory User Memory Reserved Memory
  • 50. Unified Memory Manager • Express execution and storage memory as one single unified region • Keep acquiring execution memory and evict storage as you need more execution memory Execution Memory (1.0 - spark.memory.storageFraction) * USABLE_MEMORY Storage Memory spark.memory.storageFraction * USABLE_MEMORY User Memory (1.0 - spark.memory.fraction) * (SYSTEM_MEMORY - RESERVED_MEMORY) Reserved Memory RESERVED_SYSTEM_MEMORY_BYTES (300MB)
  • 51. Dynamic occupancy mechanism spark.memory.storageFraction • If one of its space is insufficient but the other is free, then it will borrow the other’s space. • If both parties don’t have enough space, evict storage memory using LRU mechanism.
  • 52. One problem remains... • The memory resource is not enough! On-Heap Memory Off-Heap Memory Inside JVM Managed by GC Outside JVM Not managed by GC Executor Process
  • 53. Off-Heap Memory • Enabled by spark.memory.offHeap.enabled • Memory size controlled by spark.memory.offHeap.size Execution Memory Storage Memory
  • 54. Off-Heap Memory • Pros • Speed: Off-Heap Memory > Disk • Not bound by GC • Cons • Manually manage memory allocation/release
  • 55. Tuning Data Structures In Spark applications: • Prefer arrays of objects instead of collection classes (e.g., HashMap) • Avoid nested structures with a lot of small objects and pointers when possible • Use numeric IDs or enumeration objects instead of strings for keys
  • 56. Tuning Memory Config spark.memory.fraction • More execution and storage memory • Higher risk of OOM spark.memory.storageFraction • Increase storage memory to cache more data • Less execution memory may lead to tasks spill more often
  • 57. Tuning Memory Config spark.memory.offHeap.enabled spark.memory.offHeap.size • Off-Heap memory not bound by GC • On-Heap + Off-Heap memory must fit in total executor memory (spark.executor.memory) spark.shuffle.file.buffer spark.unsafe.sorter.spill.reader.buffer.size • Buffer shuffle file to amortize disk I/O • More execution memory consumption
  • 58. About Us Software Engineers • Maryann Xue PMC of Apache Calcite & Apache Phoenix @maryannxue • Xingbo Jiang Apache Spark Committer @jiangxb1987 • Kris Mok OpenJDK Committer @rednaxelafx 58
  • 60. Vectorized Readers Read columnar format data as-is without converting to row format. • Apache Parquet • Apache ORC • Apache Arrow • ... 60
  • 61. Vectorized Readers Parquet vectorized reader is 9 times faster than the non- vectorized one. See blog post 61
  • 62. Vectorized Readers Supported built-in data sources: • Parquet • ORC Arrow is used for intermediate data in PySpark. 62
  • 63. Implement DataSource DataSource v2 API provides the way to implement your own vectorized reader. • PartitionReaderFactory • supportColumnarReads(...) to return true • createColumnarReader(...) to return PartitionReader[ColumnarBatch] • [SPARK-25186] Stabilize Data Source V2 API 63
  • 64. Delta Lake • Full ACID transactions • Schema management • Scalable metadata handling • Data versioning and time travel • Unified batch/streaming support • Record update and deletion • Data expectation Delta Lake: https://delta.io/ Documentation: https://docs.delta.io For details, refer to the blog https://tinyurl.com/yxhbe2lg
  • 66. What’s behind foo(x) in Spark SQL? What looks like a function call can be a lot of things: • upper(str): Built-in function • max(val): Aggregate function • max(val) over …: Window function • explode(arr): Generator • myudf(x): User-defined function • myudaf(x): User-defined aggregate function • transform(arr, x -> x + 1): Higher-order function • range(10): Table-value function
  • 67. Functions in Spark SQL Builtin Scalar Function Java / Scala UDF Python UDF (*) Aggregate / Window Higher-order Function Scope 1 Row 1 Row 1 Row Whole table 1 Row Data Feed Scalar expressions Scalar expressions Batch of data Scalar expressions + aggregate buffer Expression of complex type Process Same JVM Same JVM Python Worker process Same JVM Same JVM Impl. Level Expression Expression Physical Operator Physical Operator Expression Data Type Internal External External Internal Internal (*): and all other non-Java user-defined functions
  • 68. UDF execution User Defined Functions: • Java/Scala UDFs • Hive UDFs • when Hive support enabled Also we have: • Python/Pandas UDFs • will talk later in PySpark execution 68
  • 69. Java/Scala UDFs • UDF: User Defined Function • Java/Scala lambdas or method references can be used. • UDAF: User Defined Aggregate Function • Need to implement UserDefinedAggregateFunction. 69
  • 70. UDAF Implement UserDefinedAggregateFunction • def initialize(...) • def update(...) • def merge(...) • def evaluate(...) • ... 70
  • 71. Hive UDFs Available when Hive support enabled. • Register using create function command • Use in HiveQL 71
  • 72. Hive UDFs Provides wrapper expressions for each UDF type: • HiveSimpleUDF: UDF • HiveGenericUDF: GenericUDF • HiveUDAFFunction: UDAF • HiveGenericUDTF: GenericUDTF 72
  • 73. UDF execution 1. Before invoking UDFs, convert arguments from internal data format to objects suitable for each UDF types. • Java/Scala UDF: Java/Scala objects • Hive UDF: ObjectInspector 2. Invoke the UDF. 3. After invocation, convert the returned values back to internal data format. 73
  • 75. PySpark PySpark is a set of Python bindings for Spark APIs. • RDD • DataFrame • other libraries based on RDDs, DataFrames. • MLlib, Structured Streaming, ... Also, SparkR: R bindings for Spark APIs 75
  • 76. PySpark RDD vs. DataFrame: • RDD invokes Python functions on Python worker • DataFrame just constructs queries, and executes it on the JVM. • except for Python/Pandas UDFs 76
  • 77. PySpark execution Python script drives Spark on JVM via Py4J. Executors run Python worker. 77 Python Driver Executor Python Worker Executor Python Worker Executor Python Worker
  • 78. PySpark and Pandas Ease of interop: PySpark can convert data between PySpark DataFrame and Pandas DataFrame. • pdf = df.toPandas() • df = spark.createDataFrame(pdf) 78 Note: df.toPandas() triggers the execution of the PySpark DataFrame, similar to df.collect()
  • 79. PySpark and Pandas (cont’d) New way of interop: Koalas brings the Pandas API to Apache Spark 79 import databricks.koalas as ks import pandas as pd pdf = pd.DataFrame({'x':range(3), 'y':['a','b','b'], 'z':['a','b','b']}) # Create a Koalas DataFrame from pandas DataFrame df = ks.from_pandas(pdf) # Rename the columns df.columns = ['x', 'y', 'z1'] # Do some operations in place: df['x2'] = df.x * df.x https://github.com/databricks/koalas
  • 81. Python UDF and Pandas UDF @udf('double') def plus_one(v): return v + 1 @pandas_udf('double', PandasUDFType.SCALAR) def pandas_plus_one(vs): return vs + 1 81
  • 82. Python/Pandas UDF execution 82 Invoke UDFPythonRunner PhysicalOperator Batch of data Deserializer SerializerBatch of data
  • 83. Python UDF execution 83 Invoke UDFPythonUDFRunner PhysicalOperator Batch of Rows Batch of Rows Deserializer Serializer
  • 84. Pandas UDF execution 84 Invoke UDFArrowPythonRunner PhysicalOperator Batch of Columns Batch of Columns Deserializer Serializer
  • 85. Python/Pandas UDFs Python UDF • Serialize/Deserialize data with Pickle • Fetch data in blocks, but invoke UDF row by row Pandas UDF • Serialize/Deserialize data with Arrow • Fetch data in blocks, and invoke UDF block by block 85
  • 86. Python/Pandas UDFs Pandas UDF perform much better than row-at-a-time Python UDFs. • 3x to over 100x See blog post 86
  • 87. Further Reading This Spark+AI Summit: • Understanding Query Plans and Spark Previous Spark Summits: • A Deep Dive into Spark SQL’s Catalyst Optimizer • Deep Dive into Project Tungsten: Bringing Spark Closer to Bare Metal • Improving Python and Spark Performance and Interoperability with Apache Arrow
  • 88. Thank you Maryann Xue (maryann.xue@databricks.com) Xingbo Jiang (xingbo.jiang@databricks.com) Kris Mok (kris.mok@databricks.com) 88