SlideShare a Scribd company logo
From Query Plan to Query
Performance:
Supercharging your Spark Queries using the Spark UI
SQL Tab
Max Thone - Resident Solutions Architect
Stefan van Wouw - Sr. Resident Solutions Architect
Agenda
Introduction to Spark SQL Tab
The Most Common Components
of the Query Plan
Supercharge your spark queries
Introduction to Spark SQL Tab
Why should you know about the SQL Tab?
▪ Shows how the Spark query is executed
▪ Can be used to reason about query execution time.
What is a Query Plan?
▪ A Spark SQL/Dataframe/Dataset query goes through Spark Catalyst Optimizer before
being executed by the JVM
▪ With “Query plan” we mean the “Selected Physical Plan”, it is the output of Catalyst
Catalyst Optimizer
From the Databricks glossary (https://databricks.com/glossary/catalyst-optimizer)
Dataframe
“action”
Query
(=physical
plan)
Spark Job
Spark Job
Spark Job
Stage
Stage
Stage
Stage
Stage
Stage
Stage
Stage
Tasks
Tasks
Hierarchy: From Spark Dataframe to Spark task
One “dataframe action” can spawn multiple queries, which can spawn multiple jobs
Query
(=physical
plan)
A simple example (1)
# dfSalesSample is some cached dataframe
dfItemSales = (dfSalesSample
.filter(f.col("item_id") >= 600000)
.groupBy("item_id")
.agg(f.sum(f.col("sales")).alias("itemSales")))
# Trigger the query
dfItemSales.write.format("noop").mode("overwrite").save()
(1) dataframe “action”
(2) Query (physical plan)
(3) Job
(4) Two Stages
(5) Nine tasks
A simple example (2)
# dfSalesSample is some cached dataframe
dfItemSales = (dfSalesSample
.filter(f.col("item_id") >= 600000)
.groupBy("item_id")
.agg(f.sum(f.col("sales")).alias("itemSales")))
# Trigger the query
dfItemSales.write.format("noop").mode("overwrite").save()
== Physical Plan ==
OverwriteByExpression org.apache.spark.sql.execution.datasources.noop.NoopTable$@dc93aa9, [AlwaysTrue()], org.apache.spark.sql.util.CaseInsensitiveStringMap@1f
+- *(2) HashAggregate(keys=[item_id#232L], functions=[finalmerge_sum(merge sum#1247L) AS sum(cast(sales#233 as bigint))#1210L], output=[item_id#232L, itemSales#1211L])
+- Exchange hashpartitioning(item_id#232L, 8), true, [id=#1268]
+- *(1) HashAggregate(keys=[item_id#232L], functions=[partial_sum(cast(sales#233 as bigint)) AS sum#1247L], output=[item_id#232L, sum#1247L])
+- *(1) Filter (isnotnull(item_id#232L) AND (item_id#232L >= 600000))
+- InMemoryTableScan [item_id#232L, sales#233], [isnotnull(item_id#232L), (item_id#232L >= 600000)]
A simple example (3)
== Physical Plan ==
OverwriteByExpression org.apache.spark.sql.execution.datasources.noop.NoopTable$@dc93aa9, [AlwaysTrue()], org.apache.spark.sql.util.CaseInsensitiveStringMap@1f
+- *(2) HashAggregate(keys=[item_id#232L], functions=[finalmerge_sum(merge sum#1247L) AS sum(cast(sales#233 as bigint))#1210L], output=[item_id#232L, itemSales#1211L])
+- Exchange hashpartitioning(item_id#232L, 8), true, [id=#1268]
+- *(1) HashAggregate(keys=[item_id#232L], functions=[partial_sum(cast(sales#233 as bigint)) AS sum#1247L], output=[item_id#232L, sum#1247L])
+- *(1) Filter (isnotnull(item_id#232L) AND (item_id#232L >= 600000))
+- InMemoryTableScan [item_id#232L, sales#233], [isnotnull(item_id#232L), (item_id#232L >= 600000)]
▪ What more possible operators exist in Physical plan?
▪ How should we interpret the “details” in the SQL plan?
▪ How can we use above knowledge to optimise our Query?
An Overview of Common Components of the
Physical Plan
The physical plan under the hood
What is the physical plan represented by in the Spark Code?
▪ The physical plan is represented by SparkPlan class
▪ SparkPlan is a recursive data structure:
▪ It represents a physical operator in the physical plan, AND the whole plan itself (1)
▪ SparkPlan is the base class, or “blueprint” for these physical operators
▪ These physical operators are “chained” together
(1) From Jacek Laskowski’s Mastering Spark SQL (https://jaceklaskowski.gitbooks.io/mastering-spark-sql/content/spark-sql-SparkPlan.html#contract
Physical operators of SparkPlan
Extending SparkPlan
Query Input
(LeafExecNode)
Output
(UnaryExecNode)
Binary
Transformation
(BinaryExecNode)
Query Input
(LeafExecNode)
Unary
Transformation
(UnaryExecNode)
Physical operators of SparkPlan
Extending SparkPlan (152 subclasses)
Query Input
(LeafExecNode)
Output
(UnaryExecNode)
Binary
Transformation
(BinaryExecNode)
Query Input
(LeafExecNode)
Unary
Transformation
(UnaryExecNode)
▪ LeafExecNode (27 subclasses)
▪ All file sources, cache read, construction of dataframes from RDDs, range
generator, and reused exchanges & subqueries.
▪ BinaryExecNode (8 subclasses)
▪ Operations with 2 dataframes as input (joins, unions, etc.)
▪ UnaryExecNode (82 subclasses)
▪ Operations with one dataframe as input. E.g. sort, aggregates, exchanges,
filters, projects, limits
▪ Other (32 traits/abstract/misc classes)
The Most Common Components of the Physical
Plan
▪ Common Narrow Transformations
▪ Distribution Requirements
(Exchange)
▪ Common Wide Transformations
▪ Aggregates
▪ Joins
▪ Ordering Requirements (Sort)
▪ Adaptive Query Execution
▪ Streaming
▪ Datasource V2 specifics
▪ Command specifics (Hive metastore
related)
▪ Dataset API specifics
▪ Caching / Reuse
▪ UDFs
Parts we will NOT cover.Parts we will cover.
Let’s start with the basics: Read/Write
Row-based Scan CSV and Write to Delta Lake
No dataframe transformations apart from read/write
spark
.read
.format("csv")
.option("header", True)
.load("/databricks-datasets/airlines")
.write
.format("delta")
.save("/tmp/airlines_delta")
Q1
Q2
1
2
3
4
Columnar Scan Delta Lake and Write to Delta Lake
High level
spark
.read
.format("delta")
.load("...path...")
.write
.format("delta")
.save("/tmp/..._delta")
Q1
Q2
Parquet is Columnar, while Spark is
row-based
Anything in this box
supports codegen
Columnar Scan Delta Lake and Write to Delta Lake
Statistics on Columnar Parquet Scan
spark
.read
.format("delta")
.load("...path...")
.write
.format("delta")
.save("/tmp/..._delta")
Q2
1
Columnar Scan Delta Lake and Write to Delta Lake
Statistics on WSCG + ColumnarToRow
spark
.read
.format("delta")
.load("...path...")
.write
.format("delta")
.save("/tmp/..._delta")
Q2
1
2
3
Common Narrow Transformations
Common Narrow Transformations
Filter / Project
spark
.read
.format("delta")
.load("...path...")
.filter(col("item_id") < 1000)
.withColumn("doubled_item_id", col("item_id")*2)
.write
.format("delta")
.save("/tmp/..._delta")
Filter → Filter
withColumn/select → Project
Common Narrow Transformations
Range / Sample / Union / Coalesce
df1 = spark.range(1000000)
df2 = spark.range(1000000)
df1
.sample(0.1)
.union(df2)
.coalesce(1)
.write
.format("delta")
.save("/tmp/..._delta")
spark.range → Range
sample → Sample
union → Union
coalesce → Coalesce
Special Case! Local Sorting
sortWithinPartitions
df.sortWithinPartitions("item_id")
sortWithinPartitions / partitionBy → Sort
(global=False)
1
Input
(item_id)
Result of
Sort
Global
result
(unsorted!
)
Partition X
33 33 33
Partition Y
34 4 4
66 8 8
4 34 34
8 66 66
Special Case! Global Sorting
orderBy
df.orderBy("item_id")
Input
(item_id)
Result of
Exchange
(example)
Result of
Sort
Global
result
(sorted!)
Partition X New
Partition X
8 4 4
33 4 8 8
Partition Y New
Partition Y
34
66 66 33 33
4 33 34 34
8 34 66 66
orderBy → Sort (global=True)
Wide Transformations
What are wide transformations?
▪ Transformations for which re-distribution of data is required
▪ e.g: joins, global sorting, and aggregations
▪ These above requirements are captured through “distribution”
requirements
Distribution requirements
Each node in the physical plan can specify how it expects data to be distributed over the Spark cluster
SparkPlan
Operator (e.g.
Filter)
requiredChildDistribution (Default: UnspecifiedDistribution)
outputPartitioning (Default: UnknownPartitioning)
Required Distribution Satisfied by (roughly)
this Partitioning of
child
Example operator
UnspecifiedDistributio
n
All Scan
AllTuples All with 1 partition only Flatmap in Pandas
OrderedDistribution RangePartitioning Sort (global)
(Hash)ClusteredDistrib
ution
HashPartitioning HashAggregate /
SortMergeJoin
BroadcastDistribution BroadcastPartitioning BroadcastHashJoin
Distribution requirements
Example for Local Sort (global=False)
Sort
(global=False)
requiredChildDistribution =
UnspecifiedDistribution
outputPartitioning = retain
child’s
Ensure the requirements Sort
(global=False)
outputPartitioning = retain
child’s
Distribution requirements
Example for Global Sort (global=True)
Sort
(global=True)
outputPartitioning =
RangePartitioning
Exchange
(rangepartition
ing)
Sort
(global=True)
requiredChildDistribution =
OrderedDistribution (ASC/DESC)
outputPartitioning = retain
child’s
Ensure the requirements
Shuffle Exchange
What are the metrics in the Shuffle exchange?
Size of shuffle bytes written
Size of serialised data read from
“local” executor
Serialised size of data read from
“remote” executors
When is it used? Before any operation that requires the same keys on same partitions (e.g. groupBy +
aggregation, and for joins (sortMergeJoin)
Broadcast Exchange
Only output rows are a metric with
broadcasts
Size of broadcasted data (in memory)
# of rows in broadcasted data
time to build the broadcast table
time to build the broadcast table
time to collect all the data
When is it used? Before any operation in which copying the same data to all nodes is required. Usually:
BroadcastHashJoin, BroadcastNestedLoopJoin
Zooming in on Aggregates
Aggregates
groupBy/agg → HashAggregate
Distribution requirement Input (item_id,
sales)
Result of
Exchange
Result of
HashAggregate 2
Partition X New Partition X
(A, 10) (A,10) (A, 13)
(B, 5) (A,3)
Partition Y New Partition Y
(A, 3) (B,1) (B, 9)
(B, 1) (B, 1)
(B, 1) (B, 1)
(B, 2) (B, 2)
df
.groupBy("item_id")
.agg(F.sum("sales"))
Aggregate implementations
df
.groupBy("item_id")
.agg(F.sum("sales"))
HashAggregateExec (Dataframe API)
- Based on HashTable structure.
- Supports codegen
- When hitting memory limits, spill to disk and start new
HashTable
- Merge all HashTables using sort based aggregation
method.
ObjectHashAggregateExec (Dataset API)
- Same as HashAggregateExec, but for JVM objects
- Does not support codegen
- Immediately falls back to sort based aggregation
method when hitting memory limits
SortAggregateExec
- sort based aggregation
Aggregates Metrics
Only in case of fallback to sorting (too many distinct
keys to keep in memory)
Partial Aggregation
Extra HashAggregate
Input (item_id,
sales)
Result of
HashAggregate 1
Result of
Exchange
Result of
HashAggregate 2
Partition X New Partition X
(A, 10) (A, 10) (A,10) (A, 13)
(B, 5) (B, 5) (A,3)
Partition Y New Partition Y
(A, 3) (A, 3) (B,5) (B, 9)
(B, 1) (B, 4) (B, 4)
(B, 1)
(B, 2)
Zooming in on Joins
Joins
# Basic aggregation + join
dfJoin = dfSalesSample.join(dfItemDim, "item_id")
Example “standard join” example (sort merge join)
▪ What kind of join algorithms exist?
▪ How does Spark choose what join algorithm to use?
▪ Where are the sorts and filters coming from?
▪ (We already know Exchanges come from
requiredChildDistribution)
Join Implementations & Requirements
Different joins have different complexities
Join Type Required Child Distribution Required
Child
Ordering
Description Complexity
(ballpark)
BroadcastHashJoinExec One Side:
BroadcastDistribution
Other: UnspecifiedDistribution
None Performs local hash join between
broadcast side and other side.
O(n)
SortMergeJoinExec Both Sides:
HashClusteredDistribution
Both Sides:
Ordered (asc)
by join keys
Compare keys of sorted data
sets and merges if match.
O(nlogn)
BroadcastNestedLoopJoinExec One Side:
BroadcastDistribution
Other: UnspecifiedDistribution
None For each row of [Left/Right]
dataset, compare all rows of
[Left/Right] data set.
O(n * m), small
m
CartesianProductExec None None Cartesian product/”cross join” +
filter
O(n* m), bigger
m
Join Strategy
How does Catalyst choose what
join?
equiJoin?
One side small
enough?
One side small
enough?
inner join?
BroadcastHashJoinExec
SortMergeJoinExec
BroadcastNestedLoopJoinExec CartesianProductExec
BroadcastNested
LoopJoinExec
Danger Zone (OOM)
No
Yes
Yes
Yes Yes
No
No No
Ordering requirements
Example for SortMergeJoinExec
SortMergeJoin
(left.id=right.id
, Inner)
outputOrdering =
[left.id, right.id] ASC
Sort ([left.id],
ASC)
SortMergeJoin
(left.id=right.id
, Inner)
requiredChildOrdering =
[left.id, right.id] (ASC)
outputOrdering = depends on
join type
Ensure the requirements
Sort ([right.id],
ASC)
Revisiting our join
# Basic aggregation + join
dfJoin = dfSalesSample.join(dfItemDim, "item_id")
Example “standard join” example (sort merge join)
equi-join? Yes
Broadcastable? No
RequiredChildDistribution -> Shuffle Exchange
RequiredChildOrdering-> Sort
} sortMergeJoin
Inner join -> Add isNotNull filter to join keys
(Logical plan, not physical plan step)
Supercharge your Spark Queries
Scenario 1: Filter + Union anti-pattern
E.g. apply different logic based on a category the data belongs to.
final_df = functools.reduce(DataFrame.union,
[
logic_cat_0(df.filter(F.col("category") == 0)),
logic_cat_1(df.filter(F.col("category") == 1)),
logic_cat_2(df.filter(F.col("category") == 2)),
logic_cat_3(df.filter(F.col("category") == 3))
]
)
…
def logic_cat_0(df: DataFrame) -> DataFrame:
return df.withColumn("output", F.col("sales") * 2)
…
Repeated
ReadsofData!
Scenario 1: Filter + Union anti-pattern FIXED
Rewrite code with CASE WHEN :)
final_df = (
df
.filter((F.col("category") >= 0) & (F.col("category") <= 3))
.withColumn("output",
F.when(F.col("category") == 0, logic_cat_0())
.when(F.col("category") == 1, logic_cat_1())
.when(F.col("category") == 2, logic_cat_2())
.otherwise(logic_cat_3())
)
)
def logic_cat_0() -> Column:
return F.col("sales") * 2
One read!
Scenario 2: Partial Aggregations
Partial aggregations do not help with high-cardinality grouping keys
transaction_dim = 100000000 # 100 million transactions
item_dim = 90000000 # 90 million itemIDs
itemDF.groupBy("itemID").agg(sum(col("sales")).alias("sales"))
Query duration: 23 seconds
This doesn’t help!
Scenario 2: Partial Aggregations FIXED
Partial aggregations do not help with high-cardinality grouping keys
transaction_dim = 100000000 # 100 million transactions
item_dim = 90000000 # 90 million itemIDs
spark.conf.set("spark.sql.aggregate.partialaggregate.skip.enabled", True)
itemDF.groupBy("itemID").agg(sum(col("sales")).alias("sales"))
Query duration: 18 seconds (22% reduction)
PR for enabling partial aggregation skipping
Scenario 3: Join Strategy
ship_ports = dfPorts.alias("p").join(
dfShips.alias("s"),
(col("s.lat") >= col("p.min_lat")) &
(col("s.lat") <= col("p.max_lat")) &
(col("s.lon") >= col("p.min_lon")) &
(col("s.lon") <= col("p.max_lon")))
Query duration: 3.5 minutes
Compare coordinates to check if a ship is in a port
slow!
Scenario 3: Join Strategy FIXED
Use a geohash to convert to equi-join
ship_ports = dfPorts.alias("p").join(
dfShips.alias("s"),
(col("s.lat") >= col("p.min_lat")) &
(col("s.lat") <= col("p.max_lat")) &
(col("s.lon") >= col("p.min_lon")) &
(col("s.lon") <= col("p.max_lon")) &
(substring(col("s.geohash"),1,2) == substring(col("p.geohash"),1,2)))
Query duration: 6 seconds
Fast!
In Summary
What we covered
The SQL Tab provides insights into how the Spark query is executed
We can use the SQL Tab to reason about query execution time.
We can answer important questions:
What part of my Spark query takes the most time?
Is my Spark query choosing the most efficient Spark operators for the task?
Want to practice / know more?
Mentally visualize what a physical plan might look like for a spark query, and then check the SQL tab if you are correct.
Check out the source code of SparkPlan
Feedback
Your feedback is important to us.
Don’t forget to rate
and review the sessions.

More Related Content

What's hot

Adaptive Query Execution: Speeding Up Spark SQL at Runtime
Adaptive Query Execution: Speeding Up Spark SQL at RuntimeAdaptive Query Execution: Speeding Up Spark SQL at Runtime
Adaptive Query Execution: Speeding Up Spark SQL at Runtime
Databricks
 
The Apache Spark File Format Ecosystem
The Apache Spark File Format EcosystemThe Apache Spark File Format Ecosystem
The Apache Spark File Format Ecosystem
Databricks
 
Lessons from the Field: Applying Best Practices to Your Apache Spark Applicat...
Lessons from the Field: Applying Best Practices to Your Apache Spark Applicat...Lessons from the Field: Applying Best Practices to Your Apache Spark Applicat...
Lessons from the Field: Applying Best Practices to Your Apache Spark Applicat...
Databricks
 
Understanding Query Plans and Spark UIs
Understanding Query Plans and Spark UIsUnderstanding Query Plans and Spark UIs
Understanding Query Plans and Spark UIs
Databricks
 
Building a SIMD Supported Vectorized Native Engine for Spark SQL
Building a SIMD Supported Vectorized Native Engine for Spark SQLBuilding a SIMD Supported Vectorized Native Engine for Spark SQL
Building a SIMD Supported Vectorized Native Engine for Spark SQL
Databricks
 
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
Bo Yang
 
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
Databricks
 
Designing ETL Pipelines with Structured Streaming and Delta Lake—How to Archi...
Designing ETL Pipelines with Structured Streaming and Delta Lake—How to Archi...Designing ETL Pipelines with Structured Streaming and Delta Lake—How to Archi...
Designing ETL Pipelines with Structured Streaming and Delta Lake—How to Archi...
Databricks
 
Fine Tuning and Enhancing Performance of Apache Spark Jobs
Fine Tuning and Enhancing Performance of Apache Spark JobsFine Tuning and Enhancing Performance of Apache Spark Jobs
Fine Tuning and Enhancing Performance of Apache Spark Jobs
Databricks
 
Accelerating Spark SQL Workloads to 50X Performance with Apache Arrow-Based F...
Accelerating Spark SQL Workloads to 50X Performance with Apache Arrow-Based F...Accelerating Spark SQL Workloads to 50X Performance with Apache Arrow-Based F...
Accelerating Spark SQL Workloads to 50X Performance with Apache Arrow-Based F...
Databricks
 
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
Databricks
 
How We Optimize Spark SQL Jobs With parallel and sync IO
How We Optimize Spark SQL Jobs With parallel and sync IOHow We Optimize Spark SQL Jobs With parallel and sync IO
How We Optimize Spark SQL Jobs With parallel and sync IO
Databricks
 
The Parquet Format and Performance Optimization Opportunities
The Parquet Format and Performance Optimization OpportunitiesThe Parquet Format and Performance Optimization Opportunities
The Parquet Format and Performance Optimization Opportunities
Databricks
 
Parquet performance tuning: the missing guide
Parquet performance tuning: the missing guideParquet performance tuning: the missing guide
Parquet performance tuning: the missing guide
Ryan Blue
 
The Rise of ZStandard: Apache Spark/Parquet/ORC/Avro
The Rise of ZStandard: Apache Spark/Parquet/ORC/AvroThe Rise of ZStandard: Apache Spark/Parquet/ORC/Avro
The Rise of ZStandard: Apache Spark/Parquet/ORC/Avro
Databricks
 
Spark shuffle introduction
Spark shuffle introductionSpark shuffle introduction
Spark shuffle introduction
colorant
 
Apache Spark Core – Practical Optimization
Apache Spark Core – Practical OptimizationApache Spark Core – Practical Optimization
Apache Spark Core – Practical Optimization
Databricks
 
Radical Speed for SQL Queries on Databricks: Photon Under the Hood
Radical Speed for SQL Queries on Databricks: Photon Under the HoodRadical Speed for SQL Queries on Databricks: Photon Under the Hood
Radical Speed for SQL Queries on Databricks: Photon Under the Hood
Databricks
 
Apache Spark Overview
Apache Spark OverviewApache Spark Overview
Apache Spark Overview
Vadim Y. Bichutskiy
 
Hive Bucketing in Apache Spark with Tejas Patil
Hive Bucketing in Apache Spark with Tejas PatilHive Bucketing in Apache Spark with Tejas Patil
Hive Bucketing in Apache Spark with Tejas Patil
Databricks
 

What's hot (20)

Adaptive Query Execution: Speeding Up Spark SQL at Runtime
Adaptive Query Execution: Speeding Up Spark SQL at RuntimeAdaptive Query Execution: Speeding Up Spark SQL at Runtime
Adaptive Query Execution: Speeding Up Spark SQL at Runtime
 
The Apache Spark File Format Ecosystem
The Apache Spark File Format EcosystemThe Apache Spark File Format Ecosystem
The Apache Spark File Format Ecosystem
 
Lessons from the Field: Applying Best Practices to Your Apache Spark Applicat...
Lessons from the Field: Applying Best Practices to Your Apache Spark Applicat...Lessons from the Field: Applying Best Practices to Your Apache Spark Applicat...
Lessons from the Field: Applying Best Practices to Your Apache Spark Applicat...
 
Understanding Query Plans and Spark UIs
Understanding Query Plans and Spark UIsUnderstanding Query Plans and Spark UIs
Understanding Query Plans and Spark UIs
 
Building a SIMD Supported Vectorized Native Engine for Spark SQL
Building a SIMD Supported Vectorized Native Engine for Spark SQLBuilding a SIMD Supported Vectorized Native Engine for Spark SQL
Building a SIMD Supported Vectorized Native Engine for Spark SQL
 
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
 
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
 
Designing ETL Pipelines with Structured Streaming and Delta Lake—How to Archi...
Designing ETL Pipelines with Structured Streaming and Delta Lake—How to Archi...Designing ETL Pipelines with Structured Streaming and Delta Lake—How to Archi...
Designing ETL Pipelines with Structured Streaming and Delta Lake—How to Archi...
 
Fine Tuning and Enhancing Performance of Apache Spark Jobs
Fine Tuning and Enhancing Performance of Apache Spark JobsFine Tuning and Enhancing Performance of Apache Spark Jobs
Fine Tuning and Enhancing Performance of Apache Spark Jobs
 
Accelerating Spark SQL Workloads to 50X Performance with Apache Arrow-Based F...
Accelerating Spark SQL Workloads to 50X Performance with Apache Arrow-Based F...Accelerating Spark SQL Workloads to 50X Performance with Apache Arrow-Based F...
Accelerating Spark SQL Workloads to 50X Performance with Apache Arrow-Based F...
 
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
 
How We Optimize Spark SQL Jobs With parallel and sync IO
How We Optimize Spark SQL Jobs With parallel and sync IOHow We Optimize Spark SQL Jobs With parallel and sync IO
How We Optimize Spark SQL Jobs With parallel and sync IO
 
The Parquet Format and Performance Optimization Opportunities
The Parquet Format and Performance Optimization OpportunitiesThe Parquet Format and Performance Optimization Opportunities
The Parquet Format and Performance Optimization Opportunities
 
Parquet performance tuning: the missing guide
Parquet performance tuning: the missing guideParquet performance tuning: the missing guide
Parquet performance tuning: the missing guide
 
The Rise of ZStandard: Apache Spark/Parquet/ORC/Avro
The Rise of ZStandard: Apache Spark/Parquet/ORC/AvroThe Rise of ZStandard: Apache Spark/Parquet/ORC/Avro
The Rise of ZStandard: Apache Spark/Parquet/ORC/Avro
 
Spark shuffle introduction
Spark shuffle introductionSpark shuffle introduction
Spark shuffle introduction
 
Apache Spark Core – Practical Optimization
Apache Spark Core – Practical OptimizationApache Spark Core – Practical Optimization
Apache Spark Core – Practical Optimization
 
Radical Speed for SQL Queries on Databricks: Photon Under the Hood
Radical Speed for SQL Queries on Databricks: Photon Under the HoodRadical Speed for SQL Queries on Databricks: Photon Under the Hood
Radical Speed for SQL Queries on Databricks: Photon Under the Hood
 
Apache Spark Overview
Apache Spark OverviewApache Spark Overview
Apache Spark Overview
 
Hive Bucketing in Apache Spark with Tejas Patil
Hive Bucketing in Apache Spark with Tejas PatilHive Bucketing in Apache Spark with Tejas Patil
Hive Bucketing in Apache Spark with Tejas Patil
 

Similar to From Query Plan to Query Performance: Supercharging your Apache Spark Queries using the Spark UI SQL Tab

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...
Databricks
 
Building a High-Performance Database with Scala, Akka, and Spark
Building a High-Performance Database with Scala, Akka, and SparkBuilding a High-Performance Database with Scala, Akka, and Spark
Building a High-Performance Database with Scala, Akka, and Spark
Evan Chan
 
Meetup spark structured streaming
Meetup spark structured streamingMeetup spark structured streaming
Meetup spark structured streaming
José Carlos García Serrano
 
Introduction to Scalding and Monoids
Introduction to Scalding and MonoidsIntroduction to Scalding and Monoids
Introduction to Scalding and Monoids
Hugo Gävert
 
Berlin buzzwords 2018
Berlin buzzwords 2018Berlin buzzwords 2018
Berlin buzzwords 2018
Matija Gobec
 
[QE 2018] Łukasz Gawron – Testing Batch and Streaming Spark Applications
[QE 2018] Łukasz Gawron – Testing Batch and Streaming Spark Applications[QE 2018] Łukasz Gawron – Testing Batch and Streaming Spark Applications
[QE 2018] Łukasz Gawron – Testing Batch and Streaming Spark Applications
Future Processing
 
Testing batch and streaming Spark applications
Testing batch and streaming Spark applicationsTesting batch and streaming Spark applications
Testing batch and streaming Spark applications
Łukasz Gawron
 
Productionizing your Streaming Jobs
Productionizing your Streaming JobsProductionizing your Streaming Jobs
Productionizing your Streaming Jobs
Databricks
 
Spark Summit EU talk by Sital Kedia
Spark Summit EU talk by Sital KediaSpark Summit EU talk by Sital Kedia
Spark Summit EU talk by Sital Kedia
Spark Summit
 
Spark Summit EU talk by Ted Malaska
Spark Summit EU talk by Ted MalaskaSpark Summit EU talk by Ted Malaska
Spark Summit EU talk by Ted Malaska
Spark Summit
 
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
 
No more struggles with Apache Spark workloads in production
No more struggles with Apache Spark workloads in productionNo more struggles with Apache Spark workloads in production
No more struggles with Apache Spark workloads in production
Chetan Khatri
 
Domain Specific Languages In Scala Duse3
Domain Specific Languages In Scala Duse3Domain Specific Languages In Scala Duse3
Domain Specific Languages In Scala Duse3
Peter Maas
 
Big Data Day LA 2016/ Hadoop/ Spark/ Kafka track - Data Provenance Support in...
Big Data Day LA 2016/ Hadoop/ Spark/ Kafka track - Data Provenance Support in...Big Data Day LA 2016/ Hadoop/ Spark/ Kafka track - Data Provenance Support in...
Big Data Day LA 2016/ Hadoop/ Spark/ Kafka track - Data Provenance Support in...
Data Con LA
 
Spark SQL Deep Dive @ Melbourne Spark Meetup
Spark SQL Deep Dive @ Melbourne Spark MeetupSpark SQL Deep Dive @ Melbourne Spark Meetup
Spark SQL Deep Dive @ Melbourne Spark Meetup
Databricks
 
Spark4
Spark4Spark4
DataEngConf SF16 - Spark SQL Workshop
DataEngConf SF16 - Spark SQL WorkshopDataEngConf SF16 - Spark SQL Workshop
DataEngConf SF16 - Spark SQL Workshop
Hakka Labs
 
Osd ctw spark
Osd ctw sparkOsd ctw spark
Osd ctw spark
Wisely chen
 
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
Rafal Kwasny
 
Spark Programming
Spark ProgrammingSpark Programming
Spark Programming
Taewook Eom
 

Similar to From Query Plan to Query Performance: Supercharging your Apache Spark Queries using the Spark UI SQL Tab (20)

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...
 
Building a High-Performance Database with Scala, Akka, and Spark
Building a High-Performance Database with Scala, Akka, and SparkBuilding a High-Performance Database with Scala, Akka, and Spark
Building a High-Performance Database with Scala, Akka, and Spark
 
Meetup spark structured streaming
Meetup spark structured streamingMeetup spark structured streaming
Meetup spark structured streaming
 
Introduction to Scalding and Monoids
Introduction to Scalding and MonoidsIntroduction to Scalding and Monoids
Introduction to Scalding and Monoids
 
Berlin buzzwords 2018
Berlin buzzwords 2018Berlin buzzwords 2018
Berlin buzzwords 2018
 
[QE 2018] Łukasz Gawron – Testing Batch and Streaming Spark Applications
[QE 2018] Łukasz Gawron – Testing Batch and Streaming Spark Applications[QE 2018] Łukasz Gawron – Testing Batch and Streaming Spark Applications
[QE 2018] Łukasz Gawron – Testing Batch and Streaming Spark Applications
 
Testing batch and streaming Spark applications
Testing batch and streaming Spark applicationsTesting batch and streaming Spark applications
Testing batch and streaming Spark applications
 
Productionizing your Streaming Jobs
Productionizing your Streaming JobsProductionizing your Streaming Jobs
Productionizing your Streaming Jobs
 
Spark Summit EU talk by Sital Kedia
Spark Summit EU talk by Sital KediaSpark Summit EU talk by Sital Kedia
Spark Summit EU talk by Sital Kedia
 
Spark Summit EU talk by Ted Malaska
Spark Summit EU talk by Ted MalaskaSpark Summit EU talk by Ted Malaska
Spark Summit EU talk by Ted Malaska
 
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...
 
No more struggles with Apache Spark workloads in production
No more struggles with Apache Spark workloads in productionNo more struggles with Apache Spark workloads in production
No more struggles with Apache Spark workloads in production
 
Domain Specific Languages In Scala Duse3
Domain Specific Languages In Scala Duse3Domain Specific Languages In Scala Duse3
Domain Specific Languages In Scala Duse3
 
Big Data Day LA 2016/ Hadoop/ Spark/ Kafka track - Data Provenance Support in...
Big Data Day LA 2016/ Hadoop/ Spark/ Kafka track - Data Provenance Support in...Big Data Day LA 2016/ Hadoop/ Spark/ Kafka track - Data Provenance Support in...
Big Data Day LA 2016/ Hadoop/ Spark/ Kafka track - Data Provenance Support in...
 
Spark SQL Deep Dive @ Melbourne Spark Meetup
Spark SQL Deep Dive @ Melbourne Spark MeetupSpark SQL Deep Dive @ Melbourne Spark Meetup
Spark SQL Deep Dive @ Melbourne Spark Meetup
 
Spark4
Spark4Spark4
Spark4
 
DataEngConf SF16 - Spark SQL Workshop
DataEngConf SF16 - Spark SQL WorkshopDataEngConf SF16 - Spark SQL Workshop
DataEngConf SF16 - Spark SQL Workshop
 
Osd ctw spark
Osd ctw sparkOsd ctw spark
Osd ctw spark
 
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 Programming
Spark ProgrammingSpark Programming
Spark Programming
 

More from Databricks

DW Migration Webinar-March 2022.pptx
DW Migration Webinar-March 2022.pptxDW Migration Webinar-March 2022.pptx
DW Migration Webinar-March 2022.pptx
Databricks
 
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
Databricks
 
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
Databricks
 
Data Lakehouse Symposium | Day 2
Data Lakehouse Symposium | Day 2Data Lakehouse Symposium | Day 2
Data Lakehouse Symposium | Day 2
Databricks
 
Data Lakehouse Symposium | Day 4
Data Lakehouse Symposium | Day 4Data Lakehouse Symposium | Day 4
Data Lakehouse Symposium | Day 4
Databricks
 
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
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
 
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
Databricks
 
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
Databricks
 
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
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
 
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
Databricks
 
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
Databricks
 
Sawtooth Windows for Feature Aggregations
Sawtooth Windows for Feature AggregationsSawtooth Windows for Feature Aggregations
Sawtooth Windows for Feature Aggregations
Databricks
 
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
Databricks
 
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
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
 
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
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

一比一原版(UMN文凭证书)明尼苏达大学毕业证如何办理
一比一原版(UMN文凭证书)明尼苏达大学毕业证如何办理一比一原版(UMN文凭证书)明尼苏达大学毕业证如何办理
一比一原版(UMN文凭证书)明尼苏达大学毕业证如何办理
nyfuhyz
 
Analysis insight about a Flyball dog competition team's performance
Analysis insight about a Flyball dog competition team's performanceAnalysis insight about a Flyball dog competition team's performance
Analysis insight about a Flyball dog competition team's performance
roli9797
 
State of Artificial intelligence Report 2023
State of Artificial intelligence Report 2023State of Artificial intelligence Report 2023
State of Artificial intelligence Report 2023
kuntobimo2016
 
University of New South Wales degree offer diploma Transcript
University of New South Wales degree offer diploma TranscriptUniversity of New South Wales degree offer diploma Transcript
University of New South Wales degree offer diploma Transcript
soxrziqu
 
一比一原版(爱大毕业证书)爱丁堡大学毕业证如何办理
一比一原版(爱大毕业证书)爱丁堡大学毕业证如何办理一比一原版(爱大毕业证书)爱丁堡大学毕业证如何办理
一比一原版(爱大毕业证书)爱丁堡大学毕业证如何办理
g4dpvqap0
 
一比一原版(Chester毕业证书)切斯特大学毕业证如何办理
一比一原版(Chester毕业证书)切斯特大学毕业证如何办理一比一原版(Chester毕业证书)切斯特大学毕业证如何办理
一比一原版(Chester毕业证书)切斯特大学毕业证如何办理
74nqk8xf
 
一比一原版(Harvard毕业证书)哈佛大学毕业证如何办理
一比一原版(Harvard毕业证书)哈佛大学毕业证如何办理一比一原版(Harvard毕业证书)哈佛大学毕业证如何办理
一比一原版(Harvard毕业证书)哈佛大学毕业证如何办理
zsjl4mimo
 
A presentation that explain the Power BI Licensing
A presentation that explain the Power BI LicensingA presentation that explain the Power BI Licensing
A presentation that explain the Power BI Licensing
AlessioFois2
 
一比一原版(Glasgow毕业证书)格拉斯哥大学毕业证如何办理
一比一原版(Glasgow毕业证书)格拉斯哥大学毕业证如何办理一比一原版(Glasgow毕业证书)格拉斯哥大学毕业证如何办理
一比一原版(Glasgow毕业证书)格拉斯哥大学毕业证如何办理
g4dpvqap0
 
Intelligence supported media monitoring in veterinary medicine
Intelligence supported media monitoring in veterinary medicineIntelligence supported media monitoring in veterinary medicine
Intelligence supported media monitoring in veterinary medicine
AndrzejJarynowski
 
ViewShift: Hassle-free Dynamic Policy Enforcement for Every Data Lake
ViewShift: Hassle-free Dynamic Policy Enforcement for Every Data LakeViewShift: Hassle-free Dynamic Policy Enforcement for Every Data Lake
ViewShift: Hassle-free Dynamic Policy Enforcement for Every Data Lake
Walaa Eldin Moustafa
 
Population Growth in Bataan: The effects of population growth around rural pl...
Population Growth in Bataan: The effects of population growth around rural pl...Population Growth in Bataan: The effects of population growth around rural pl...
Population Growth in Bataan: The effects of population growth around rural pl...
Bill641377
 
一比一原版(GWU,GW文凭证书)乔治·华盛顿大学毕业证如何办理
一比一原版(GWU,GW文凭证书)乔治·华盛顿大学毕业证如何办理一比一原版(GWU,GW文凭证书)乔治·华盛顿大学毕业证如何办理
一比一原版(GWU,GW文凭证书)乔治·华盛顿大学毕业证如何办理
bopyb
 
Everything you wanted to know about LIHTC
Everything you wanted to know about LIHTCEverything you wanted to know about LIHTC
Everything you wanted to know about LIHTC
Roger Valdez
 
My burning issue is homelessness K.C.M.O.
My burning issue is homelessness K.C.M.O.My burning issue is homelessness K.C.M.O.
My burning issue is homelessness K.C.M.O.
rwarrenll
 
The Building Blocks of QuestDB, a Time Series Database
The Building Blocks of QuestDB, a Time Series DatabaseThe Building Blocks of QuestDB, a Time Series Database
The Building Blocks of QuestDB, a Time Series Database
javier ramirez
 
一比一原版(牛布毕业证书)牛津布鲁克斯大学毕业证如何办理
一比一原版(牛布毕业证书)牛津布鲁克斯大学毕业证如何办理一比一原版(牛布毕业证书)牛津布鲁克斯大学毕业证如何办理
一比一原版(牛布毕业证书)牛津布鲁克斯大学毕业证如何办理
74nqk8xf
 
Challenges of Nation Building-1.pptx with more important
Challenges of Nation Building-1.pptx with more importantChallenges of Nation Building-1.pptx with more important
Challenges of Nation Building-1.pptx with more important
Sm321
 
办(uts毕业证书)悉尼科技大学毕业证学历证书原版一模一样
办(uts毕业证书)悉尼科技大学毕业证学历证书原版一模一样办(uts毕业证书)悉尼科技大学毕业证学历证书原版一模一样
办(uts毕业证书)悉尼科技大学毕业证学历证书原版一模一样
apvysm8
 
一比一原版(UCSB文凭证书)圣芭芭拉分校毕业证如何办理
一比一原版(UCSB文凭证书)圣芭芭拉分校毕业证如何办理一比一原版(UCSB文凭证书)圣芭芭拉分校毕业证如何办理
一比一原版(UCSB文凭证书)圣芭芭拉分校毕业证如何办理
nuttdpt
 

Recently uploaded (20)

一比一原版(UMN文凭证书)明尼苏达大学毕业证如何办理
一比一原版(UMN文凭证书)明尼苏达大学毕业证如何办理一比一原版(UMN文凭证书)明尼苏达大学毕业证如何办理
一比一原版(UMN文凭证书)明尼苏达大学毕业证如何办理
 
Analysis insight about a Flyball dog competition team's performance
Analysis insight about a Flyball dog competition team's performanceAnalysis insight about a Flyball dog competition team's performance
Analysis insight about a Flyball dog competition team's performance
 
State of Artificial intelligence Report 2023
State of Artificial intelligence Report 2023State of Artificial intelligence Report 2023
State of Artificial intelligence Report 2023
 
University of New South Wales degree offer diploma Transcript
University of New South Wales degree offer diploma TranscriptUniversity of New South Wales degree offer diploma Transcript
University of New South Wales degree offer diploma Transcript
 
一比一原版(爱大毕业证书)爱丁堡大学毕业证如何办理
一比一原版(爱大毕业证书)爱丁堡大学毕业证如何办理一比一原版(爱大毕业证书)爱丁堡大学毕业证如何办理
一比一原版(爱大毕业证书)爱丁堡大学毕业证如何办理
 
一比一原版(Chester毕业证书)切斯特大学毕业证如何办理
一比一原版(Chester毕业证书)切斯特大学毕业证如何办理一比一原版(Chester毕业证书)切斯特大学毕业证如何办理
一比一原版(Chester毕业证书)切斯特大学毕业证如何办理
 
一比一原版(Harvard毕业证书)哈佛大学毕业证如何办理
一比一原版(Harvard毕业证书)哈佛大学毕业证如何办理一比一原版(Harvard毕业证书)哈佛大学毕业证如何办理
一比一原版(Harvard毕业证书)哈佛大学毕业证如何办理
 
A presentation that explain the Power BI Licensing
A presentation that explain the Power BI LicensingA presentation that explain the Power BI Licensing
A presentation that explain the Power BI Licensing
 
一比一原版(Glasgow毕业证书)格拉斯哥大学毕业证如何办理
一比一原版(Glasgow毕业证书)格拉斯哥大学毕业证如何办理一比一原版(Glasgow毕业证书)格拉斯哥大学毕业证如何办理
一比一原版(Glasgow毕业证书)格拉斯哥大学毕业证如何办理
 
Intelligence supported media monitoring in veterinary medicine
Intelligence supported media monitoring in veterinary medicineIntelligence supported media monitoring in veterinary medicine
Intelligence supported media monitoring in veterinary medicine
 
ViewShift: Hassle-free Dynamic Policy Enforcement for Every Data Lake
ViewShift: Hassle-free Dynamic Policy Enforcement for Every Data LakeViewShift: Hassle-free Dynamic Policy Enforcement for Every Data Lake
ViewShift: Hassle-free Dynamic Policy Enforcement for Every Data Lake
 
Population Growth in Bataan: The effects of population growth around rural pl...
Population Growth in Bataan: The effects of population growth around rural pl...Population Growth in Bataan: The effects of population growth around rural pl...
Population Growth in Bataan: The effects of population growth around rural pl...
 
一比一原版(GWU,GW文凭证书)乔治·华盛顿大学毕业证如何办理
一比一原版(GWU,GW文凭证书)乔治·华盛顿大学毕业证如何办理一比一原版(GWU,GW文凭证书)乔治·华盛顿大学毕业证如何办理
一比一原版(GWU,GW文凭证书)乔治·华盛顿大学毕业证如何办理
 
Everything you wanted to know about LIHTC
Everything you wanted to know about LIHTCEverything you wanted to know about LIHTC
Everything you wanted to know about LIHTC
 
My burning issue is homelessness K.C.M.O.
My burning issue is homelessness K.C.M.O.My burning issue is homelessness K.C.M.O.
My burning issue is homelessness K.C.M.O.
 
The Building Blocks of QuestDB, a Time Series Database
The Building Blocks of QuestDB, a Time Series DatabaseThe Building Blocks of QuestDB, a Time Series Database
The Building Blocks of QuestDB, a Time Series Database
 
一比一原版(牛布毕业证书)牛津布鲁克斯大学毕业证如何办理
一比一原版(牛布毕业证书)牛津布鲁克斯大学毕业证如何办理一比一原版(牛布毕业证书)牛津布鲁克斯大学毕业证如何办理
一比一原版(牛布毕业证书)牛津布鲁克斯大学毕业证如何办理
 
Challenges of Nation Building-1.pptx with more important
Challenges of Nation Building-1.pptx with more importantChallenges of Nation Building-1.pptx with more important
Challenges of Nation Building-1.pptx with more important
 
办(uts毕业证书)悉尼科技大学毕业证学历证书原版一模一样
办(uts毕业证书)悉尼科技大学毕业证学历证书原版一模一样办(uts毕业证书)悉尼科技大学毕业证学历证书原版一模一样
办(uts毕业证书)悉尼科技大学毕业证学历证书原版一模一样
 
一比一原版(UCSB文凭证书)圣芭芭拉分校毕业证如何办理
一比一原版(UCSB文凭证书)圣芭芭拉分校毕业证如何办理一比一原版(UCSB文凭证书)圣芭芭拉分校毕业证如何办理
一比一原版(UCSB文凭证书)圣芭芭拉分校毕业证如何办理
 

From Query Plan to Query Performance: Supercharging your Apache Spark Queries using the Spark UI SQL Tab

  • 1. From Query Plan to Query Performance: Supercharging your Spark Queries using the Spark UI SQL Tab Max Thone - Resident Solutions Architect Stefan van Wouw - Sr. Resident Solutions Architect
  • 2. Agenda Introduction to Spark SQL Tab The Most Common Components of the Query Plan Supercharge your spark queries
  • 4. Why should you know about the SQL Tab? ▪ Shows how the Spark query is executed ▪ Can be used to reason about query execution time.
  • 5. What is a Query Plan? ▪ A Spark SQL/Dataframe/Dataset query goes through Spark Catalyst Optimizer before being executed by the JVM ▪ With “Query plan” we mean the “Selected Physical Plan”, it is the output of Catalyst Catalyst Optimizer From the Databricks glossary (https://databricks.com/glossary/catalyst-optimizer)
  • 6. Dataframe “action” Query (=physical plan) Spark Job Spark Job Spark Job Stage Stage Stage Stage Stage Stage Stage Stage Tasks Tasks Hierarchy: From Spark Dataframe to Spark task One “dataframe action” can spawn multiple queries, which can spawn multiple jobs Query (=physical plan)
  • 7. A simple example (1) # dfSalesSample is some cached dataframe dfItemSales = (dfSalesSample .filter(f.col("item_id") >= 600000) .groupBy("item_id") .agg(f.sum(f.col("sales")).alias("itemSales"))) # Trigger the query dfItemSales.write.format("noop").mode("overwrite").save() (1) dataframe “action” (2) Query (physical plan) (3) Job (4) Two Stages (5) Nine tasks
  • 8. A simple example (2) # dfSalesSample is some cached dataframe dfItemSales = (dfSalesSample .filter(f.col("item_id") >= 600000) .groupBy("item_id") .agg(f.sum(f.col("sales")).alias("itemSales"))) # Trigger the query dfItemSales.write.format("noop").mode("overwrite").save() == Physical Plan == OverwriteByExpression org.apache.spark.sql.execution.datasources.noop.NoopTable$@dc93aa9, [AlwaysTrue()], org.apache.spark.sql.util.CaseInsensitiveStringMap@1f +- *(2) HashAggregate(keys=[item_id#232L], functions=[finalmerge_sum(merge sum#1247L) AS sum(cast(sales#233 as bigint))#1210L], output=[item_id#232L, itemSales#1211L]) +- Exchange hashpartitioning(item_id#232L, 8), true, [id=#1268] +- *(1) HashAggregate(keys=[item_id#232L], functions=[partial_sum(cast(sales#233 as bigint)) AS sum#1247L], output=[item_id#232L, sum#1247L]) +- *(1) Filter (isnotnull(item_id#232L) AND (item_id#232L >= 600000)) +- InMemoryTableScan [item_id#232L, sales#233], [isnotnull(item_id#232L), (item_id#232L >= 600000)]
  • 9. A simple example (3) == Physical Plan == OverwriteByExpression org.apache.spark.sql.execution.datasources.noop.NoopTable$@dc93aa9, [AlwaysTrue()], org.apache.spark.sql.util.CaseInsensitiveStringMap@1f +- *(2) HashAggregate(keys=[item_id#232L], functions=[finalmerge_sum(merge sum#1247L) AS sum(cast(sales#233 as bigint))#1210L], output=[item_id#232L, itemSales#1211L]) +- Exchange hashpartitioning(item_id#232L, 8), true, [id=#1268] +- *(1) HashAggregate(keys=[item_id#232L], functions=[partial_sum(cast(sales#233 as bigint)) AS sum#1247L], output=[item_id#232L, sum#1247L]) +- *(1) Filter (isnotnull(item_id#232L) AND (item_id#232L >= 600000)) +- InMemoryTableScan [item_id#232L, sales#233], [isnotnull(item_id#232L), (item_id#232L >= 600000)] ▪ What more possible operators exist in Physical plan? ▪ How should we interpret the “details” in the SQL plan? ▪ How can we use above knowledge to optimise our Query?
  • 10. An Overview of Common Components of the Physical Plan
  • 11. The physical plan under the hood What is the physical plan represented by in the Spark Code? ▪ The physical plan is represented by SparkPlan class ▪ SparkPlan is a recursive data structure: ▪ It represents a physical operator in the physical plan, AND the whole plan itself (1) ▪ SparkPlan is the base class, or “blueprint” for these physical operators ▪ These physical operators are “chained” together (1) From Jacek Laskowski’s Mastering Spark SQL (https://jaceklaskowski.gitbooks.io/mastering-spark-sql/content/spark-sql-SparkPlan.html#contract
  • 12. Physical operators of SparkPlan Extending SparkPlan Query Input (LeafExecNode) Output (UnaryExecNode) Binary Transformation (BinaryExecNode) Query Input (LeafExecNode) Unary Transformation (UnaryExecNode)
  • 13. Physical operators of SparkPlan Extending SparkPlan (152 subclasses) Query Input (LeafExecNode) Output (UnaryExecNode) Binary Transformation (BinaryExecNode) Query Input (LeafExecNode) Unary Transformation (UnaryExecNode) ▪ LeafExecNode (27 subclasses) ▪ All file sources, cache read, construction of dataframes from RDDs, range generator, and reused exchanges & subqueries. ▪ BinaryExecNode (8 subclasses) ▪ Operations with 2 dataframes as input (joins, unions, etc.) ▪ UnaryExecNode (82 subclasses) ▪ Operations with one dataframe as input. E.g. sort, aggregates, exchanges, filters, projects, limits ▪ Other (32 traits/abstract/misc classes)
  • 14. The Most Common Components of the Physical Plan ▪ Common Narrow Transformations ▪ Distribution Requirements (Exchange) ▪ Common Wide Transformations ▪ Aggregates ▪ Joins ▪ Ordering Requirements (Sort) ▪ Adaptive Query Execution ▪ Streaming ▪ Datasource V2 specifics ▪ Command specifics (Hive metastore related) ▪ Dataset API specifics ▪ Caching / Reuse ▪ UDFs Parts we will NOT cover.Parts we will cover.
  • 15. Let’s start with the basics: Read/Write
  • 16. Row-based Scan CSV and Write to Delta Lake No dataframe transformations apart from read/write spark .read .format("csv") .option("header", True) .load("/databricks-datasets/airlines") .write .format("delta") .save("/tmp/airlines_delta") Q1 Q2 1 2 3 4
  • 17. Columnar Scan Delta Lake and Write to Delta Lake High level spark .read .format("delta") .load("...path...") .write .format("delta") .save("/tmp/..._delta") Q1 Q2 Parquet is Columnar, while Spark is row-based Anything in this box supports codegen
  • 18. Columnar Scan Delta Lake and Write to Delta Lake Statistics on Columnar Parquet Scan spark .read .format("delta") .load("...path...") .write .format("delta") .save("/tmp/..._delta") Q2 1
  • 19. Columnar Scan Delta Lake and Write to Delta Lake Statistics on WSCG + ColumnarToRow spark .read .format("delta") .load("...path...") .write .format("delta") .save("/tmp/..._delta") Q2 1 2 3
  • 21. Common Narrow Transformations Filter / Project spark .read .format("delta") .load("...path...") .filter(col("item_id") < 1000) .withColumn("doubled_item_id", col("item_id")*2) .write .format("delta") .save("/tmp/..._delta") Filter → Filter withColumn/select → Project
  • 22. Common Narrow Transformations Range / Sample / Union / Coalesce df1 = spark.range(1000000) df2 = spark.range(1000000) df1 .sample(0.1) .union(df2) .coalesce(1) .write .format("delta") .save("/tmp/..._delta") spark.range → Range sample → Sample union → Union coalesce → Coalesce
  • 23. Special Case! Local Sorting sortWithinPartitions df.sortWithinPartitions("item_id") sortWithinPartitions / partitionBy → Sort (global=False) 1 Input (item_id) Result of Sort Global result (unsorted! ) Partition X 33 33 33 Partition Y 34 4 4 66 8 8 4 34 34 8 66 66
  • 24. Special Case! Global Sorting orderBy df.orderBy("item_id") Input (item_id) Result of Exchange (example) Result of Sort Global result (sorted!) Partition X New Partition X 8 4 4 33 4 8 8 Partition Y New Partition Y 34 66 66 33 33 4 33 34 34 8 34 66 66 orderBy → Sort (global=True)
  • 26. What are wide transformations? ▪ Transformations for which re-distribution of data is required ▪ e.g: joins, global sorting, and aggregations ▪ These above requirements are captured through “distribution” requirements
  • 27. Distribution requirements Each node in the physical plan can specify how it expects data to be distributed over the Spark cluster SparkPlan Operator (e.g. Filter) requiredChildDistribution (Default: UnspecifiedDistribution) outputPartitioning (Default: UnknownPartitioning) Required Distribution Satisfied by (roughly) this Partitioning of child Example operator UnspecifiedDistributio n All Scan AllTuples All with 1 partition only Flatmap in Pandas OrderedDistribution RangePartitioning Sort (global) (Hash)ClusteredDistrib ution HashPartitioning HashAggregate / SortMergeJoin BroadcastDistribution BroadcastPartitioning BroadcastHashJoin
  • 28. Distribution requirements Example for Local Sort (global=False) Sort (global=False) requiredChildDistribution = UnspecifiedDistribution outputPartitioning = retain child’s Ensure the requirements Sort (global=False) outputPartitioning = retain child’s
  • 29. Distribution requirements Example for Global Sort (global=True) Sort (global=True) outputPartitioning = RangePartitioning Exchange (rangepartition ing) Sort (global=True) requiredChildDistribution = OrderedDistribution (ASC/DESC) outputPartitioning = retain child’s Ensure the requirements
  • 30. Shuffle Exchange What are the metrics in the Shuffle exchange? Size of shuffle bytes written Size of serialised data read from “local” executor Serialised size of data read from “remote” executors When is it used? Before any operation that requires the same keys on same partitions (e.g. groupBy + aggregation, and for joins (sortMergeJoin)
  • 31. Broadcast Exchange Only output rows are a metric with broadcasts Size of broadcasted data (in memory) # of rows in broadcasted data time to build the broadcast table time to build the broadcast table time to collect all the data When is it used? Before any operation in which copying the same data to all nodes is required. Usually: BroadcastHashJoin, BroadcastNestedLoopJoin
  • 32. Zooming in on Aggregates
  • 33. Aggregates groupBy/agg → HashAggregate Distribution requirement Input (item_id, sales) Result of Exchange Result of HashAggregate 2 Partition X New Partition X (A, 10) (A,10) (A, 13) (B, 5) (A,3) Partition Y New Partition Y (A, 3) (B,1) (B, 9) (B, 1) (B, 1) (B, 1) (B, 1) (B, 2) (B, 2) df .groupBy("item_id") .agg(F.sum("sales"))
  • 34. Aggregate implementations df .groupBy("item_id") .agg(F.sum("sales")) HashAggregateExec (Dataframe API) - Based on HashTable structure. - Supports codegen - When hitting memory limits, spill to disk and start new HashTable - Merge all HashTables using sort based aggregation method. ObjectHashAggregateExec (Dataset API) - Same as HashAggregateExec, but for JVM objects - Does not support codegen - Immediately falls back to sort based aggregation method when hitting memory limits SortAggregateExec - sort based aggregation
  • 35. Aggregates Metrics Only in case of fallback to sorting (too many distinct keys to keep in memory)
  • 36. Partial Aggregation Extra HashAggregate Input (item_id, sales) Result of HashAggregate 1 Result of Exchange Result of HashAggregate 2 Partition X New Partition X (A, 10) (A, 10) (A,10) (A, 13) (B, 5) (B, 5) (A,3) Partition Y New Partition Y (A, 3) (A, 3) (B,5) (B, 9) (B, 1) (B, 4) (B, 4) (B, 1) (B, 2)
  • 37. Zooming in on Joins
  • 38. Joins # Basic aggregation + join dfJoin = dfSalesSample.join(dfItemDim, "item_id") Example “standard join” example (sort merge join) ▪ What kind of join algorithms exist? ▪ How does Spark choose what join algorithm to use? ▪ Where are the sorts and filters coming from? ▪ (We already know Exchanges come from requiredChildDistribution)
  • 39. Join Implementations & Requirements Different joins have different complexities Join Type Required Child Distribution Required Child Ordering Description Complexity (ballpark) BroadcastHashJoinExec One Side: BroadcastDistribution Other: UnspecifiedDistribution None Performs local hash join between broadcast side and other side. O(n) SortMergeJoinExec Both Sides: HashClusteredDistribution Both Sides: Ordered (asc) by join keys Compare keys of sorted data sets and merges if match. O(nlogn) BroadcastNestedLoopJoinExec One Side: BroadcastDistribution Other: UnspecifiedDistribution None For each row of [Left/Right] dataset, compare all rows of [Left/Right] data set. O(n * m), small m CartesianProductExec None None Cartesian product/”cross join” + filter O(n* m), bigger m
  • 40. Join Strategy How does Catalyst choose what join? equiJoin? One side small enough? One side small enough? inner join? BroadcastHashJoinExec SortMergeJoinExec BroadcastNestedLoopJoinExec CartesianProductExec BroadcastNested LoopJoinExec Danger Zone (OOM) No Yes Yes Yes Yes No No No
  • 41. Ordering requirements Example for SortMergeJoinExec SortMergeJoin (left.id=right.id , Inner) outputOrdering = [left.id, right.id] ASC Sort ([left.id], ASC) SortMergeJoin (left.id=right.id , Inner) requiredChildOrdering = [left.id, right.id] (ASC) outputOrdering = depends on join type Ensure the requirements Sort ([right.id], ASC)
  • 42. Revisiting our join # Basic aggregation + join dfJoin = dfSalesSample.join(dfItemDim, "item_id") Example “standard join” example (sort merge join) equi-join? Yes Broadcastable? No RequiredChildDistribution -> Shuffle Exchange RequiredChildOrdering-> Sort } sortMergeJoin Inner join -> Add isNotNull filter to join keys (Logical plan, not physical plan step)
  • 44. Scenario 1: Filter + Union anti-pattern E.g. apply different logic based on a category the data belongs to. final_df = functools.reduce(DataFrame.union, [ logic_cat_0(df.filter(F.col("category") == 0)), logic_cat_1(df.filter(F.col("category") == 1)), logic_cat_2(df.filter(F.col("category") == 2)), logic_cat_3(df.filter(F.col("category") == 3)) ] ) … def logic_cat_0(df: DataFrame) -> DataFrame: return df.withColumn("output", F.col("sales") * 2) … Repeated ReadsofData!
  • 45. Scenario 1: Filter + Union anti-pattern FIXED Rewrite code with CASE WHEN :) final_df = ( df .filter((F.col("category") >= 0) & (F.col("category") <= 3)) .withColumn("output", F.when(F.col("category") == 0, logic_cat_0()) .when(F.col("category") == 1, logic_cat_1()) .when(F.col("category") == 2, logic_cat_2()) .otherwise(logic_cat_3()) ) ) def logic_cat_0() -> Column: return F.col("sales") * 2 One read!
  • 46. Scenario 2: Partial Aggregations Partial aggregations do not help with high-cardinality grouping keys transaction_dim = 100000000 # 100 million transactions item_dim = 90000000 # 90 million itemIDs itemDF.groupBy("itemID").agg(sum(col("sales")).alias("sales")) Query duration: 23 seconds This doesn’t help!
  • 47. Scenario 2: Partial Aggregations FIXED Partial aggregations do not help with high-cardinality grouping keys transaction_dim = 100000000 # 100 million transactions item_dim = 90000000 # 90 million itemIDs spark.conf.set("spark.sql.aggregate.partialaggregate.skip.enabled", True) itemDF.groupBy("itemID").agg(sum(col("sales")).alias("sales")) Query duration: 18 seconds (22% reduction) PR for enabling partial aggregation skipping
  • 48. Scenario 3: Join Strategy ship_ports = dfPorts.alias("p").join( dfShips.alias("s"), (col("s.lat") >= col("p.min_lat")) & (col("s.lat") <= col("p.max_lat")) & (col("s.lon") >= col("p.min_lon")) & (col("s.lon") <= col("p.max_lon"))) Query duration: 3.5 minutes Compare coordinates to check if a ship is in a port slow!
  • 49. Scenario 3: Join Strategy FIXED Use a geohash to convert to equi-join ship_ports = dfPorts.alias("p").join( dfShips.alias("s"), (col("s.lat") >= col("p.min_lat")) & (col("s.lat") <= col("p.max_lat")) & (col("s.lon") >= col("p.min_lon")) & (col("s.lon") <= col("p.max_lon")) & (substring(col("s.geohash"),1,2) == substring(col("p.geohash"),1,2))) Query duration: 6 seconds Fast!
  • 51. What we covered The SQL Tab provides insights into how the Spark query is executed We can use the SQL Tab to reason about query execution time. We can answer important questions: What part of my Spark query takes the most time? Is my Spark query choosing the most efficient Spark operators for the task? Want to practice / know more? Mentally visualize what a physical plan might look like for a spark query, and then check the SQL tab if you are correct. Check out the source code of SparkPlan
  • 52. Feedback Your feedback is important to us. Don’t forget to rate and review the sessions.