SlideShare a Scribd company logo
1 of 50
Download to read offline
Deep Dive Into Catalyst:
Apache Spark’s Optimizer
Herman van Hövell
Spark SummitEurope, Brussels
October 26th 2016
2
Who is Databricks
Why Us Our Product
• Created Apache Spark to enable big data
use cases with a single engine.
• Contributes 75%of Spark’s code
• Bring Spark to the enterprise: The just-
in-time data platform.
• Fully managed platform powered by
Apache Spark.
• A unified solution for data science and
engineering teams.
3
Spark Core (RDD)
Catalyst
DataFrame/DatasetSQL
ML Pipelines
Structured
Streaming
{ JSON }
JDBC
and more…
Spark SQL
GraphFrames
Overview
Why structure?
• By definition, structure will limit what can be
expressed.
• In practice, we can accommodate the vast
majority of computations.
4
Limiting the space of what can be
expressed enables optimizations.
Why structure?
5
pdata.map { case (dpt, age) => dpt -> (age, 1) }
.reduceByKey { case ((a1, c1), (a2, c2)) => (a1 + a2, c1 + c2)}
.map { case (dpt, (age, c)) => dpt -> age/ c }
select dept, avg(age) from data group by 1
SQL
Dataframe
RDD
data.groupBy(“dept”).avg(“age”)
6
0 1 2 3 4 5
RDD
Dataframe
SQL
Runtime performance of aggregating 10 million int pairs (secs)
Why structure?
How?
• Write programs using high level programming interfaces
• Programs are used to describe what data operations are needed
withoutspecifying how to execute those operations
• High level programming interfaces: SQL, DataFrames, and Dataset
• Get an optimizer that automatically finds out the most
efficient plan to execute data operations specified in the
user’s program
7
8
Catalyst:
Apache Spark’s Optimizer
9
SQL AST
DataFrame
Dataset
(Java/Scala)
Query Plan
Optimized
Query Plan
RDDs
Unresolved
Logical Plan
Logical Plan
Optimized
Logical Plan
Selected
Physical Plan
CostModel
Physical
Plans
Catalog
Analysis
Logical
Optimization
Physical
Planning
Catalyst
10
Unresolved
Logical Plan
Logical Plan
Optimized
Logical Plan
Selected
Physical Plan
CostModel
Physical
Plans
Catalog
Analysis
Logical
Optimization
Physical
Planning
• Analysis (Rule Executor): Transforms an Unresolved Logical
Plan to a Resolved Logical Plan
• Unresolved => Resolved: Use Catalog to find where datasets and
columns are coming from and types of columns
• Logical Optimization (Rule Executor): Transforms a Resolved
Logical Plan to an Optimized Logical Plan
• Physical Planning (Strategies + Rule Executor): Transforms a
Optimized Logical Plan to a Physical Plan
11
How Catalyst Works: An Overview
SQL AST
DataFrame
Dataset
(Java/Scala)
Query Plan
Optimized
Query Plan
RDDs
Code GenerationTransformations
Catalyst
Abstractions of users’ programs
(Trees)
12
How Catalyst Works: An Overview
SQL AST
DataFrame
Dataset
(Java/Scala)
Query Plan
Optimized
Query Plan
RDDs
Code GenerationTransformations
Catalyst
Abstractions of users’ programs
(Trees)
13
Trees: Abstractionsof Users’ Programs
SELECT sum(v)
FROM (
SELECT
t1.id,
1 + 2 + t1.value AS v
FROM t1 JOIN t2
WHERE
t1.id = t2.id AND
t2.id > 50 * 1000) tmp
14
Trees: Abstractionsof Users’ Programs
SELECT sum(v)
FROM (
SELECT
t1.id,
1 + 2 + t1.value AS v
FROM t1 JOIN t2
WHERE
t1.id = t2.id AND
t2.id > 50 * 1000) tmp
Expression
• An expressionrepresentsa
new value, computedbased
on input values
• e.g. 1 + 2 + t1.value
• Attribute: A columnof a
dataset(e.g.t1.id) or a
column generatedbya
specificdata operation(e.g.
v)
15
Trees: Abstractionsof Users’ Programs
SELECT sum(v)
FROM (
SELECT
t1.id,
1 + 2 + t1.value AS v
FROM t1 JOIN t2
WHERE
t1.id = t2.id AND
t2.id > 50 * 1000) tmp
Query Plan
Scan
(t1)
Scan
(t2)
Join
Filter
Project
Aggregate sum(v)
t1.id,
1+2+t1.value
as v
t1.id=t2.id
t2.id>50*1000
Logical Plan
• A LogicalPlan describescomputation
on datasetswithout defininghow to
conductthe computation
• output: a listof attributesgenerated
by this LogicalPlan, e.g.[id, v]
• constraints:a setof invariants about
the rows generatedby this plan, e.g.
t2.id > 50 * 1000
16
Scan
(t1)
Scan
(t2)
Join
Filter
Project
Aggregate sum(v)
t1.id,
1+2+t1.value
as v
t1.id=t2.id
t2.id>50*1000
Physical Plan
• A PhysicalPlan describescomputation
on datasetswith specificdefinitionson
how to conductthe computation
• A PhysicalPlan is executable
17
Parquet Scan
(t1)
JSON Scan
(t2)
Sort-Merge
Join
Filter
Project
Hash-
Aggregate
sum(v)
t1.id,
1+2+t1.value
as v
t1.id=t2.id
t2.id>50*1000
18
How Catalyst Works: An Overview
SQL AST
DataFrame
Dataset
(Java/Scala)
Query Plan
Optimized
Query Plan
RDDs
Code GenerationTransformations
Catalyst
Abstractions of users’ programs
(Trees)
Transformations
• Transformationswithoutchangingthetree type
(Transform andRule Executor)
• Expression => Expression
• Logical Plan => Logical Plan
• Physical Plan => Physical Plan
• Transforminga tree to anotherkind of tree
• Logical Plan => Physical Plan
19
• A functionassociatedwith everytree usedto
implementa singlerule
Transform
20
Attribute
(t1.value)
Add
Add
Literal(1) Literal(2)
1 + 2 + t1.value
Attribute
(t1.value)
Add
Literal(3)
3+ t1.valueEvaluate 1 + 2 onceEvaluate 1 + 2
for every row
Transform
• A transformationis definedas a Partial Function
• Partial Function:A functionthat is definedfor a subset
of its possiblearguments
21
val expression: Expression = ...
expression.transform {
case Add(Literal(x, IntegerType), Literal(y, IntegerType)) =>
Literal(x + y)
}
Case statement determine if the partial function is defined for agiven input
val expression: Expression = ...
expression.transform {
case Add(Literal(x, IntegerType), Literal(y, IntegerType)) =>
Literal(x + y)
}
Transform
22
Attribute
(t1.value)
Add
Add
Literal(1) Literal(2)
1 + 2 + t1.value
val expression: Expression = ...
expression.transform {
case Add(Literal(x, IntegerType), Literal(y, IntegerType)) =>
Literal(x + y)
}
Transform
23
Attribute
(t1.value)
Add
Add
Literal(1) Literal(2)
1 + 2 + t1.value
val expression: Expression = ...
expression.transform {
case Add(Literal(x, IntegerType), Literal(y, IntegerType)) =>
Literal(x + y)
}
Transform
24
Attribute
(t1.value)
Add
Add
Literal(1) Literal(2)
1 + 2 + t1.value
val expression: Expression = ...
expression.transform {
case Add(Literal(x, IntegerType), Literal(y, IntegerType)) =>
Literal(x + y)
}
Transform
25
Attribute
(t1.value)
Add
Add
Literal(1) Literal(2)
1 + 2 + t1.value
Attribute
(t1.value)
Add
Literal(3)
3+ t1.value
Combining Multiple Rules
26
Scan
(t1)
Scan
(t2)
Join
Filter
Project
Aggregate sum(v)
t1.id,
1+2+t1.value
as v
t1.id=t2.id
t2.id>50*1000
Predicate Pushdown
Scan
(t1)
Scan
(t2)
Join
Filter
Project
Aggregate sum(v)
t1.id,
1+2+t1.value
as v
t2.id>50*1000
t1.id=t2.id
Combining Multiple Rules
27
Constant Folding
Scan
(t1)
Scan
(t2)
Join
Filter
Project
Aggregate sum(v)
t1.id,
1+2+t1.value
as v
t2.id>50*1000
t1.id=t2.id
Scan
(t1)
Scan
(t2)
Join
Filter
Project
Aggregate sum(v)
t1.id,
3+t1.value as
v
t2.id>50000
t1.id=t2.id
Combining Multiple Rules
28
Column Pruning
Scan
(t1)
Scan
(t2)
Join
Filter
Project
Aggregate sum(v)
t1.id,
3+t1.value as
v
t2.id>50000
t1.id=t2.id
Scan
(t1)
Scan
(t2)
Join
Filter
Project
Aggregate sum(v)
t1.id,
3+t1.value as
v
t2.id>50000
t1.id=t2.id
Project Project
t1.id
t1.value t2.id
Combining Multiple Rules
29
Scan
(t1)
Scan
(t2)
Join
Filter
Project
Aggregate sum(v)
t1.id,
1+2+t1.value
as v
t1.id=t2.id
t2.id>50*1000
Scan
(t1)
Scan
(t2)
Join
Filter
Project
Aggregate sum(v)
t1.id,
3+t1.value as
v
t2.id>50000
t1.id=t2.id
Project Projectt1.id
t1.value
t2.id
Before transformations
After transformations
Combining Multiple Rules: Rule Executor
30
Batch 1 Batch 2 Batch n…
Rule1
Rule2
…
Rule1
Rule2
…
A Rule Executor transforms a Tree to another same type Tree by
applying many rules defined in batches
Every rule is implemented
based on Transform
Approaches of
applying rules
1. Fixed point
2. Once
Transformations
• Transformationswithoutchangingthetree type
(Transform andRule Executor)
• Expression => Expression
• Logical Plan => Logical Plan
• Physical Plan => Physical Plan
• Transforminga tree to anotherkind of tree
• Logical Plan => Physical Plan
31
From Logical Plan to Physical Plan
• A Logical Plan is transformed to a Physical Plan by applying
a set of Strategies
• Every Strategy uses pattern matching to convert a Tree to
another kind of Tree
32
object BasicOperators extends Strategy {
def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match {
…
case logical.Project(projectList, child) =>
execution.ProjectExec(projectList, planLater(child)) :: Nil
case logical.Filter(condition, child) =>
execution.FilterExec(condition, planLater(child)) :: Nil
…
}
} Triggers other Strategies
Spark’s Planner
• 1st Phase:Transformsthe LogicalPlan to the Physical
Plan usingStrategies
• 2nd Phase:Use a Rule Executorto make the Physical
Plan ready forexecution
• Prepare Scalar sub-queries
• Ensure requirements on input rows
• Apply physical optimizations
33
Ensure Requirementson Input Rows
34
Sort-Merge Join
[t1.id = t2.id]
t1
Requirements:
Rows Sorted by t1.id
t2
Requirements:
Rows Sorted by t2.id
Ensure Requirementson Input Rows
35
Sort-Merge Join
[t1.id = t2.id]
t1
Requirements:
Rows Sorted by t1.id
Requirements:
Rows Sorted by t2.id
Sort
[t1.id]
t2
Sort
[t2.id]
Ensure Requirementson Input Rows
36
Sort-Merge Join
[t1.id = t2.id]
t1
[sorted by t1.id]
Requirements:
Rows Sorted by t1.id
Requirements:
Rows Sorted by t2.id
Sort
[t1.id]
t2
Sort
[t2.id]What if t1 has been
sorted by t1.id?
Ensure Requirementson Input Rows
37
Sort-Merge Join
[t1.id = t2.id]
t1
[sorted by t1.id]
Requirements:
Rows Sorted by t1.id
Requirements:
Rows Sorted by t2.id
t2
Sort
[t2.id]
38
Roll your own PlanningRule
39
Roll your own Planner Rule
import org.apache.spark.sql.functions._
val tableA = spark.range(100000000).as('a)
val tableB = spark.range(100000000).as('b)
val result = tableA
.join(tableB, $"a.id" === $"b.id")
.groupBy()
.count()
result.count()
This	takes	~22	Seconds	on	Databricks Community	edition
40
Roll your own Planner Rule
Can we do better?
41
Roll your own Planner Rule - Analysis
== Physical Plan ==
*HashAggregate(keys=[], functions=[count(1)], output=[count#43L])
+- Exchange SinglePartition
+- *HashAggregate(keys=[], functions=[partial_count(1)], output=[count#48L])
+- *Project
+- *SortMergeJoin [id#21L], [id#25L], Inner
:- *Sort [id#21L ASC], false, 0
: +- Exchange hashpartitioning(id#21L, 200)
: +- *Range (0, 100000000, step=1, splits=Some(8))
+- *Sort [id#25L ASC], false, 0
+- Exchange hashpartitioning(id#25L, 200)
+- *Range (0, 100000000, step=1, splits=Some(8))
42
Roll your own Planner Rule
Exploit the structure of the problem
We are joining two intervals; the result will be the intersectionof
these intervals
A
B
A∩B
43
Roll your own Planner Rule - Matching
case object IntervalJoin extends Strategy with Serializable {
def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match {
case Join(
Range(start1, end1, 1, part1, Seq(o1)),
Range(start2, end2, 1, part2, Seq(o2)),
Inner,
Some(EqualTo(e1, e2)))
if ((o1 semanticEquals e1) && (o2 semanticEquals e2)) ||
((o1 semanticEquals e2) && (o2 semanticEquals e1)) =>
// Rule...
case _ => Nil
}
}
44
Roll your own Planner Rule - Body
if astart1 <= end2) && (end1 >= end2)) {
val start = math.max(start1, start2)
val end = math.min(end1, end2)
val part = math.max(part1.getOrElse(200), part2.getOrElse(200))
val result = RangeExec(Range(start, end, 1, part, o1 :: Nil))
val twoColumns = ProjectExec(
Alias(o1, o1.name)(exprId = o1.exprId) :: Nil,
result)
twoColumns :: Nil
} else {
Nil
}
45
Roll your own Planner Rule
Hook it up with Spark
spark.experimental.extraStrategies = IntervalJoin :: Nil
Use it
result.count()
This now takes0.46secondsto complete
46
Roll your own Planner Rule
== Physical Plan ==
*HashAggregate(keys=[], functions=[count(1)],
output=[count#43L])
+- Exchange SinglePartition
+- *HashAggregate(keys=[], functions=[partial_count(1)],
output=[count#48L])
+- *Project
+- *Project [id#21L AS id#21L]
+- *Range (0, 100000000, step=1, splits=Some(8))
Community Contributed Transformations
47
110 line patch took this user’s query from
“never finishing” to 200s.
Overall	200+	people	have	contributed	to	the	analyzer/optimizer/planner	 in	the	last	2	years.
Where to Start
• Source Code:
• Trees: TreeNode, Expression, Logical Plan, and Physical Plan
• Transformations: Analyzer, Optimizer, and Planner
• Check outprevious pull requests
• Start to write code usingCatalyst
• Open a pullrequest!
48
• Try latestversionof ApacheSpark
Try Apache Spark with Databricks
49
http://databricks.com/try
Questions?
I willbeavailableintheDatabricksbooth(D1)afterwards
@Westerflyer

More Related Content

What's hot

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 JobsDatabricks
 
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 SparkBo Yang
 
Optimizing Apache Spark SQL Joins
Optimizing Apache Spark SQL JoinsOptimizing Apache Spark SQL Joins
Optimizing Apache Spark SQL JoinsDatabricks
 
Parquet performance tuning: the missing guide
Parquet performance tuning: the missing guideParquet performance tuning: the missing guide
Parquet performance tuning: the missing guideRyan Blue
 
Spark shuffle introduction
Spark shuffle introductionSpark shuffle introduction
Spark shuffle introductioncolorant
 
The columnar roadmap: Apache Parquet and Apache Arrow
The columnar roadmap: Apache Parquet and Apache ArrowThe columnar roadmap: Apache Parquet and Apache Arrow
The columnar roadmap: Apache Parquet and Apache ArrowDataWorks Summit
 
Deep Dive into the New Features of Apache Spark 3.0
Deep Dive into the New Features of Apache Spark 3.0Deep Dive into the New Features of Apache Spark 3.0
Deep Dive into the New Features of Apache Spark 3.0Databricks
 
Best Practice of Compression/Decompression Codes in Apache Spark with Sophia...
 Best Practice of Compression/Decompression Codes in Apache Spark with Sophia... Best Practice of Compression/Decompression Codes in Apache Spark with Sophia...
Best Practice of Compression/Decompression Codes in Apache Spark with Sophia...Databricks
 
Top 5 Mistakes When Writing Spark Applications
Top 5 Mistakes When Writing Spark ApplicationsTop 5 Mistakes When Writing Spark Applications
Top 5 Mistakes When Writing Spark ApplicationsSpark Summit
 
Deep Dive into Stateful Stream Processing in Structured Streaming with Tathag...
Deep Dive into Stateful Stream Processing in Structured Streaming with Tathag...Deep Dive into Stateful Stream Processing in Structured Streaming with Tathag...
Deep Dive into Stateful Stream Processing in Structured Streaming with Tathag...Databricks
 
The Apache Spark File Format Ecosystem
The Apache Spark File Format EcosystemThe Apache Spark File Format Ecosystem
The Apache Spark File Format EcosystemDatabricks
 
A Deep Dive into Spark SQL's Catalyst Optimizer with Yin Huai
A Deep Dive into Spark SQL's Catalyst Optimizer with Yin HuaiA Deep Dive into Spark SQL's Catalyst Optimizer with Yin Huai
A Deep Dive into Spark SQL's Catalyst Optimizer with Yin HuaiDatabricks
 
Top 5 mistakes when writing Spark applications
Top 5 mistakes when writing Spark applicationsTop 5 mistakes when writing Spark applications
Top 5 mistakes when writing Spark applicationshadooparchbook
 
Amazon S3 Best Practice and Tuning for Hadoop/Spark in the Cloud
Amazon S3 Best Practice and Tuning for Hadoop/Spark in the CloudAmazon S3 Best Practice and Tuning for Hadoop/Spark in the Cloud
Amazon S3 Best Practice and Tuning for Hadoop/Spark in the CloudNoritaka Sekiyama
 
Top 5 Mistakes to Avoid When Writing Apache Spark Applications
Top 5 Mistakes to Avoid When Writing Apache Spark ApplicationsTop 5 Mistakes to Avoid When Writing Apache Spark Applications
Top 5 Mistakes to Avoid When Writing Apache Spark ApplicationsCloudera, Inc.
 
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/AvroDatabricks
 
Memory Management in Apache Spark
Memory Management in Apache SparkMemory Management in Apache Spark
Memory Management in Apache SparkDatabricks
 
Deep Dive into Spark SQL with Advanced Performance Tuning with Xiao Li & Wenc...
Deep Dive into Spark SQL with Advanced Performance Tuning with Xiao Li & Wenc...Deep Dive into Spark SQL with Advanced Performance Tuning with Xiao Li & Wenc...
Deep Dive into Spark SQL with Advanced Performance Tuning with Xiao Li & Wenc...Databricks
 
Data profiling in Apache Calcite
Data profiling in Apache CalciteData profiling in Apache Calcite
Data profiling in Apache CalciteDataWorks Summit
 
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 SQLDatabricks
 

What's hot (20)

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
 
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
 
Optimizing Apache Spark SQL Joins
Optimizing Apache Spark SQL JoinsOptimizing Apache Spark SQL Joins
Optimizing Apache Spark SQL Joins
 
Parquet performance tuning: the missing guide
Parquet performance tuning: the missing guideParquet performance tuning: the missing guide
Parquet performance tuning: the missing guide
 
Spark shuffle introduction
Spark shuffle introductionSpark shuffle introduction
Spark shuffle introduction
 
The columnar roadmap: Apache Parquet and Apache Arrow
The columnar roadmap: Apache Parquet and Apache ArrowThe columnar roadmap: Apache Parquet and Apache Arrow
The columnar roadmap: Apache Parquet and Apache Arrow
 
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
 
Best Practice of Compression/Decompression Codes in Apache Spark with Sophia...
 Best Practice of Compression/Decompression Codes in Apache Spark with Sophia... Best Practice of Compression/Decompression Codes in Apache Spark with Sophia...
Best Practice of Compression/Decompression Codes in Apache Spark with Sophia...
 
Top 5 Mistakes When Writing Spark Applications
Top 5 Mistakes When Writing Spark ApplicationsTop 5 Mistakes When Writing Spark Applications
Top 5 Mistakes When Writing Spark Applications
 
Deep Dive into Stateful Stream Processing in Structured Streaming with Tathag...
Deep Dive into Stateful Stream Processing in Structured Streaming with Tathag...Deep Dive into Stateful Stream Processing in Structured Streaming with Tathag...
Deep Dive into Stateful Stream Processing in Structured Streaming with Tathag...
 
The Apache Spark File Format Ecosystem
The Apache Spark File Format EcosystemThe Apache Spark File Format Ecosystem
The Apache Spark File Format Ecosystem
 
A Deep Dive into Spark SQL's Catalyst Optimizer with Yin Huai
A Deep Dive into Spark SQL's Catalyst Optimizer with Yin HuaiA Deep Dive into Spark SQL's Catalyst Optimizer with Yin Huai
A Deep Dive into Spark SQL's Catalyst Optimizer with Yin Huai
 
Top 5 mistakes when writing Spark applications
Top 5 mistakes when writing Spark applicationsTop 5 mistakes when writing Spark applications
Top 5 mistakes when writing Spark applications
 
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
 
Top 5 Mistakes to Avoid When Writing Apache Spark Applications
Top 5 Mistakes to Avoid When Writing Apache Spark ApplicationsTop 5 Mistakes to Avoid When Writing Apache Spark Applications
Top 5 Mistakes to Avoid When Writing Apache Spark Applications
 
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
 
Memory Management in Apache Spark
Memory Management in Apache SparkMemory Management in Apache Spark
Memory Management in Apache Spark
 
Deep Dive into Spark SQL with Advanced Performance Tuning with Xiao Li & Wenc...
Deep Dive into Spark SQL with Advanced Performance Tuning with Xiao Li & Wenc...Deep Dive into Spark SQL with Advanced Performance Tuning with Xiao Li & Wenc...
Deep Dive into Spark SQL with Advanced Performance Tuning with Xiao Li & Wenc...
 
Data profiling in Apache Calcite
Data profiling in Apache CalciteData profiling in Apache Calcite
Data profiling in Apache Calcite
 
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
 

Similar to Deep Dive Into Catalyst: Apache Spark's Query Optimizer

Deep Dive Into Catalyst: Apache Spark 2.0'S Optimizer
Deep Dive Into Catalyst: Apache Spark 2.0'S OptimizerDeep Dive Into Catalyst: Apache Spark 2.0'S Optimizer
Deep Dive Into Catalyst: Apache Spark 2.0'S OptimizerSpark Summit
 
Deep Dive Into Catalyst: Apache Spark 2.0’s Optimizer
Deep Dive Into Catalyst: Apache Spark 2.0’s OptimizerDeep Dive Into Catalyst: Apache Spark 2.0’s Optimizer
Deep Dive Into Catalyst: Apache Spark 2.0’s OptimizerDatabricks
 
SparkSQL: A Compiler from Queries to RDDs
SparkSQL: A Compiler from Queries to RDDsSparkSQL: A Compiler from Queries to RDDs
SparkSQL: A Compiler from Queries to RDDsDatabricks
 
AI與大數據數據處理 Spark實戰(20171216)
AI與大數據數據處理 Spark實戰(20171216)AI與大數據數據處理 Spark實戰(20171216)
AI與大數據數據處理 Spark實戰(20171216)Paul Chao
 
Structuring Spark: DataFrames, Datasets, and Streaming by Michael Armbrust
Structuring Spark: DataFrames, Datasets, and Streaming by Michael ArmbrustStructuring Spark: DataFrames, Datasets, and Streaming by Michael Armbrust
Structuring Spark: DataFrames, Datasets, and Streaming by Michael ArmbrustSpark Summit
 
Spark Sql and DataFrame
Spark Sql and DataFrameSpark Sql and DataFrame
Spark Sql and DataFramePrashant Gupta
 
Extending Spark for Qbeast's SQL Data Source​ with Paola Pardo and Cesare Cug...
Extending Spark for Qbeast's SQL Data Source​ with Paola Pardo and Cesare Cug...Extending Spark for Qbeast's SQL Data Source​ with Paola Pardo and Cesare Cug...
Extending Spark for Qbeast's SQL Data Source​ with Paola Pardo and Cesare Cug...Qbeast
 
20140908 spark sql & catalyst
20140908 spark sql & catalyst20140908 spark sql & catalyst
20140908 spark sql & catalystTakuya UESHIN
 
Structuring Apache Spark 2.0: SQL, DataFrames, Datasets And Streaming - by Mi...
Structuring Apache Spark 2.0: SQL, DataFrames, Datasets And Streaming - by Mi...Structuring Apache Spark 2.0: SQL, DataFrames, Datasets And Streaming - by Mi...
Structuring Apache Spark 2.0: SQL, DataFrames, Datasets And Streaming - by Mi...Databricks
 
Fast federated SQL with Apache Calcite
Fast federated SQL with Apache CalciteFast federated SQL with Apache Calcite
Fast federated SQL with Apache CalciteChris Baynes
 
Spark SQL In Depth www.syedacademy.com
Spark SQL In Depth www.syedacademy.comSpark SQL In Depth www.syedacademy.com
Spark SQL In Depth www.syedacademy.comSyed Hadoop
 
Structuring Spark: DataFrames, Datasets, and Streaming
Structuring Spark: DataFrames, Datasets, and StreamingStructuring Spark: DataFrames, Datasets, and Streaming
Structuring Spark: DataFrames, Datasets, and StreamingDatabricks
 
Flink Forward SF 2017: Timo Walther - Table & SQL API – unified APIs for bat...
Flink Forward SF 2017: Timo Walther -  Table & SQL API – unified APIs for bat...Flink Forward SF 2017: Timo Walther -  Table & SQL API – unified APIs for bat...
Flink Forward SF 2017: Timo Walther - Table & SQL API – unified APIs for bat...Flink Forward
 
Apache Calcite: A Foundational Framework for Optimized Query Processing Over ...
Apache Calcite: A Foundational Framework for Optimized Query Processing Over ...Apache Calcite: A Foundational Framework for Optimized Query Processing Over ...
Apache Calcite: A Foundational Framework for Optimized Query Processing Over ...Julian Hyde
 
Anatomy of Data Frame API : A deep dive into Spark Data Frame API
Anatomy of Data Frame API :  A deep dive into Spark Data Frame APIAnatomy of Data Frame API :  A deep dive into Spark Data Frame API
Anatomy of Data Frame API : A deep dive into Spark Data Frame APIdatamantra
 
Deep Dive : Spark Data Frames, SQL and Catalyst Optimizer
Deep Dive : Spark Data Frames, SQL and Catalyst OptimizerDeep Dive : Spark Data Frames, SQL and Catalyst Optimizer
Deep Dive : Spark Data Frames, SQL and Catalyst OptimizerSachin Aggarwal
 
Spark what's new what's coming
Spark what's new what's comingSpark what's new what's coming
Spark what's new what's comingDatabricks
 
Volodymyr Lyubinets "Introduction to big data processing with Apache Spark"
Volodymyr Lyubinets "Introduction to big data processing with Apache Spark"Volodymyr Lyubinets "Introduction to big data processing with Apache Spark"
Volodymyr Lyubinets "Introduction to big data processing with Apache Spark"IT Event
 

Similar to Deep Dive Into Catalyst: Apache Spark's Query Optimizer (20)

Deep Dive Into Catalyst: Apache Spark 2.0'S Optimizer
Deep Dive Into Catalyst: Apache Spark 2.0'S OptimizerDeep Dive Into Catalyst: Apache Spark 2.0'S Optimizer
Deep Dive Into Catalyst: Apache Spark 2.0'S Optimizer
 
Deep Dive Into Catalyst: Apache Spark 2.0’s Optimizer
Deep Dive Into Catalyst: Apache Spark 2.0’s OptimizerDeep Dive Into Catalyst: Apache Spark 2.0’s Optimizer
Deep Dive Into Catalyst: Apache Spark 2.0’s Optimizer
 
SparkSQL: A Compiler from Queries to RDDs
SparkSQL: A Compiler from Queries to RDDsSparkSQL: A Compiler from Queries to RDDs
SparkSQL: A Compiler from Queries to RDDs
 
AI與大數據數據處理 Spark實戰(20171216)
AI與大數據數據處理 Spark實戰(20171216)AI與大數據數據處理 Spark實戰(20171216)
AI與大數據數據處理 Spark實戰(20171216)
 
Structuring Spark: DataFrames, Datasets, and Streaming by Michael Armbrust
Structuring Spark: DataFrames, Datasets, and Streaming by Michael ArmbrustStructuring Spark: DataFrames, Datasets, and Streaming by Michael Armbrust
Structuring Spark: DataFrames, Datasets, and Streaming by Michael Armbrust
 
Spark Sql and DataFrame
Spark Sql and DataFrameSpark Sql and DataFrame
Spark Sql and DataFrame
 
Extending Spark for Qbeast's SQL Data Source​ with Paola Pardo and Cesare Cug...
Extending Spark for Qbeast's SQL Data Source​ with Paola Pardo and Cesare Cug...Extending Spark for Qbeast's SQL Data Source​ with Paola Pardo and Cesare Cug...
Extending Spark for Qbeast's SQL Data Source​ with Paola Pardo and Cesare Cug...
 
20140908 spark sql & catalyst
20140908 spark sql & catalyst20140908 spark sql & catalyst
20140908 spark sql & catalyst
 
Structuring Apache Spark 2.0: SQL, DataFrames, Datasets And Streaming - by Mi...
Structuring Apache Spark 2.0: SQL, DataFrames, Datasets And Streaming - by Mi...Structuring Apache Spark 2.0: SQL, DataFrames, Datasets And Streaming - by Mi...
Structuring Apache Spark 2.0: SQL, DataFrames, Datasets And Streaming - by Mi...
 
Fast federated SQL with Apache Calcite
Fast federated SQL with Apache CalciteFast federated SQL with Apache Calcite
Fast federated SQL with Apache Calcite
 
Meetup ml spark_ppt
Meetup ml spark_pptMeetup ml spark_ppt
Meetup ml spark_ppt
 
Spark SQL In Depth www.syedacademy.com
Spark SQL In Depth www.syedacademy.comSpark SQL In Depth www.syedacademy.com
Spark SQL In Depth www.syedacademy.com
 
Structuring Spark: DataFrames, Datasets, and Streaming
Structuring Spark: DataFrames, Datasets, and StreamingStructuring Spark: DataFrames, Datasets, and Streaming
Structuring Spark: DataFrames, Datasets, and Streaming
 
Flink Forward SF 2017: Timo Walther - Table & SQL API – unified APIs for bat...
Flink Forward SF 2017: Timo Walther -  Table & SQL API – unified APIs for bat...Flink Forward SF 2017: Timo Walther -  Table & SQL API – unified APIs for bat...
Flink Forward SF 2017: Timo Walther - Table & SQL API – unified APIs for bat...
 
Apache Calcite: A Foundational Framework for Optimized Query Processing Over ...
Apache Calcite: A Foundational Framework for Optimized Query Processing Over ...Apache Calcite: A Foundational Framework for Optimized Query Processing Over ...
Apache Calcite: A Foundational Framework for Optimized Query Processing Over ...
 
Anatomy of Data Frame API : A deep dive into Spark Data Frame API
Anatomy of Data Frame API :  A deep dive into Spark Data Frame APIAnatomy of Data Frame API :  A deep dive into Spark Data Frame API
Anatomy of Data Frame API : A deep dive into Spark Data Frame API
 
Deep Dive : Spark Data Frames, SQL and Catalyst Optimizer
Deep Dive : Spark Data Frames, SQL and Catalyst OptimizerDeep Dive : Spark Data Frames, SQL and Catalyst Optimizer
Deep Dive : Spark Data Frames, SQL and Catalyst Optimizer
 
Shooting the Rapids
Shooting the RapidsShooting the Rapids
Shooting the Rapids
 
Spark what's new what's coming
Spark what's new what's comingSpark what's new what's coming
Spark what's new what's coming
 
Volodymyr Lyubinets "Introduction to big data processing with Apache Spark"
Volodymyr Lyubinets "Introduction to big data processing with Apache Spark"Volodymyr Lyubinets "Introduction to big data processing with Apache Spark"
Volodymyr Lyubinets "Introduction to big data processing with Apache Spark"
 

More from Spark Summit

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

More from Spark Summit (20)

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

Recently uploaded

RA-11058_IRR-COMPRESS Do 198 series of 1998
RA-11058_IRR-COMPRESS Do 198 series of 1998RA-11058_IRR-COMPRESS Do 198 series of 1998
RA-11058_IRR-COMPRESS Do 198 series of 1998YohFuh
 
Delhi Call Girls CP 9711199171 ☎✔👌✔ Whatsapp Hard And Sexy Vip Call
Delhi Call Girls CP 9711199171 ☎✔👌✔ Whatsapp Hard And Sexy Vip CallDelhi Call Girls CP 9711199171 ☎✔👌✔ Whatsapp Hard And Sexy Vip Call
Delhi Call Girls CP 9711199171 ☎✔👌✔ Whatsapp Hard And Sexy Vip Callshivangimorya083
 
Ravak dropshipping via API with DroFx.pptx
Ravak dropshipping via API with DroFx.pptxRavak dropshipping via API with DroFx.pptx
Ravak dropshipping via API with DroFx.pptxolyaivanovalion
 
CebaBaby dropshipping via API with DroFX.pptx
CebaBaby dropshipping via API with DroFX.pptxCebaBaby dropshipping via API with DroFX.pptx
CebaBaby dropshipping via API with DroFX.pptxolyaivanovalion
 
Carero dropshipping via API with DroFx.pptx
Carero dropshipping via API with DroFx.pptxCarero dropshipping via API with DroFx.pptx
Carero dropshipping via API with DroFx.pptxolyaivanovalion
 
dokumen.tips_chapter-4-transient-heat-conduction-mehmet-kanoglu.ppt
dokumen.tips_chapter-4-transient-heat-conduction-mehmet-kanoglu.pptdokumen.tips_chapter-4-transient-heat-conduction-mehmet-kanoglu.ppt
dokumen.tips_chapter-4-transient-heat-conduction-mehmet-kanoglu.pptSonatrach
 
100-Concepts-of-AI by Anupama Kate .pptx
100-Concepts-of-AI by Anupama Kate .pptx100-Concepts-of-AI by Anupama Kate .pptx
100-Concepts-of-AI by Anupama Kate .pptxAnupama Kate
 
VIP Call Girls in Amravati Aarohi 8250192130 Independent Escort Service Amravati
VIP Call Girls in Amravati Aarohi 8250192130 Independent Escort Service AmravatiVIP Call Girls in Amravati Aarohi 8250192130 Independent Escort Service Amravati
VIP Call Girls in Amravati Aarohi 8250192130 Independent Escort Service AmravatiSuhani Kapoor
 
꧁❤ Greater Noida Call Girls Delhi ❤꧂ 9711199171 ☎️ Hard And Sexy Vip Call
꧁❤ Greater Noida Call Girls Delhi ❤꧂ 9711199171 ☎️ Hard And Sexy Vip Call꧁❤ Greater Noida Call Girls Delhi ❤꧂ 9711199171 ☎️ Hard And Sexy Vip Call
꧁❤ Greater Noida Call Girls Delhi ❤꧂ 9711199171 ☎️ Hard And Sexy Vip Callshivangimorya083
 
Midocean dropshipping via API with DroFx
Midocean dropshipping via API with DroFxMidocean dropshipping via API with DroFx
Midocean dropshipping via API with DroFxolyaivanovalion
 
Week-01-2.ppt BBB human Computer interaction
Week-01-2.ppt BBB human Computer interactionWeek-01-2.ppt BBB human Computer interaction
Week-01-2.ppt BBB human Computer interactionfulawalesam
 
Schema on read is obsolete. Welcome metaprogramming..pdf
Schema on read is obsolete. Welcome metaprogramming..pdfSchema on read is obsolete. Welcome metaprogramming..pdf
Schema on read is obsolete. Welcome metaprogramming..pdfLars Albertsson
 
Introduction-to-Machine-Learning (1).pptx
Introduction-to-Machine-Learning (1).pptxIntroduction-to-Machine-Learning (1).pptx
Introduction-to-Machine-Learning (1).pptxfirstjob4
 
Smarteg dropshipping via API with DroFx.pptx
Smarteg dropshipping via API with DroFx.pptxSmarteg dropshipping via API with DroFx.pptx
Smarteg dropshipping via API with DroFx.pptxolyaivanovalion
 
VIP High Profile Call Girls Amravati Aarushi 8250192130 Independent Escort Se...
VIP High Profile Call Girls Amravati Aarushi 8250192130 Independent Escort Se...VIP High Profile Call Girls Amravati Aarushi 8250192130 Independent Escort Se...
VIP High Profile Call Girls Amravati Aarushi 8250192130 Independent Escort Se...Suhani Kapoor
 
VIP Call Girls Service Miyapur Hyderabad Call +91-8250192130
VIP Call Girls Service Miyapur Hyderabad Call +91-8250192130VIP Call Girls Service Miyapur Hyderabad Call +91-8250192130
VIP Call Girls Service Miyapur Hyderabad Call +91-8250192130Suhani Kapoor
 
B2 Creative Industry Response Evaluation.docx
B2 Creative Industry Response Evaluation.docxB2 Creative Industry Response Evaluation.docx
B2 Creative Industry Response Evaluation.docxStephen266013
 
FESE Capital Markets Fact Sheet 2024 Q1.pdf
FESE Capital Markets Fact Sheet 2024 Q1.pdfFESE Capital Markets Fact Sheet 2024 Q1.pdf
FESE Capital Markets Fact Sheet 2024 Q1.pdfMarinCaroMartnezBerg
 
Brighton SEO | April 2024 | Data Storytelling
Brighton SEO | April 2024 | Data StorytellingBrighton SEO | April 2024 | Data Storytelling
Brighton SEO | April 2024 | Data StorytellingNeil Barnes
 

Recently uploaded (20)

RA-11058_IRR-COMPRESS Do 198 series of 1998
RA-11058_IRR-COMPRESS Do 198 series of 1998RA-11058_IRR-COMPRESS Do 198 series of 1998
RA-11058_IRR-COMPRESS Do 198 series of 1998
 
Delhi Call Girls CP 9711199171 ☎✔👌✔ Whatsapp Hard And Sexy Vip Call
Delhi Call Girls CP 9711199171 ☎✔👌✔ Whatsapp Hard And Sexy Vip CallDelhi Call Girls CP 9711199171 ☎✔👌✔ Whatsapp Hard And Sexy Vip Call
Delhi Call Girls CP 9711199171 ☎✔👌✔ Whatsapp Hard And Sexy Vip Call
 
Delhi 99530 vip 56974 Genuine Escort Service Call Girls in Kishangarh
Delhi 99530 vip 56974 Genuine Escort Service Call Girls in  KishangarhDelhi 99530 vip 56974 Genuine Escort Service Call Girls in  Kishangarh
Delhi 99530 vip 56974 Genuine Escort Service Call Girls in Kishangarh
 
Ravak dropshipping via API with DroFx.pptx
Ravak dropshipping via API with DroFx.pptxRavak dropshipping via API with DroFx.pptx
Ravak dropshipping via API with DroFx.pptx
 
CebaBaby dropshipping via API with DroFX.pptx
CebaBaby dropshipping via API with DroFX.pptxCebaBaby dropshipping via API with DroFX.pptx
CebaBaby dropshipping via API with DroFX.pptx
 
Carero dropshipping via API with DroFx.pptx
Carero dropshipping via API with DroFx.pptxCarero dropshipping via API with DroFx.pptx
Carero dropshipping via API with DroFx.pptx
 
dokumen.tips_chapter-4-transient-heat-conduction-mehmet-kanoglu.ppt
dokumen.tips_chapter-4-transient-heat-conduction-mehmet-kanoglu.pptdokumen.tips_chapter-4-transient-heat-conduction-mehmet-kanoglu.ppt
dokumen.tips_chapter-4-transient-heat-conduction-mehmet-kanoglu.ppt
 
100-Concepts-of-AI by Anupama Kate .pptx
100-Concepts-of-AI by Anupama Kate .pptx100-Concepts-of-AI by Anupama Kate .pptx
100-Concepts-of-AI by Anupama Kate .pptx
 
VIP Call Girls in Amravati Aarohi 8250192130 Independent Escort Service Amravati
VIP Call Girls in Amravati Aarohi 8250192130 Independent Escort Service AmravatiVIP Call Girls in Amravati Aarohi 8250192130 Independent Escort Service Amravati
VIP Call Girls in Amravati Aarohi 8250192130 Independent Escort Service Amravati
 
꧁❤ Greater Noida Call Girls Delhi ❤꧂ 9711199171 ☎️ Hard And Sexy Vip Call
꧁❤ Greater Noida Call Girls Delhi ❤꧂ 9711199171 ☎️ Hard And Sexy Vip Call꧁❤ Greater Noida Call Girls Delhi ❤꧂ 9711199171 ☎️ Hard And Sexy Vip Call
꧁❤ Greater Noida Call Girls Delhi ❤꧂ 9711199171 ☎️ Hard And Sexy Vip Call
 
Midocean dropshipping via API with DroFx
Midocean dropshipping via API with DroFxMidocean dropshipping via API with DroFx
Midocean dropshipping via API with DroFx
 
Week-01-2.ppt BBB human Computer interaction
Week-01-2.ppt BBB human Computer interactionWeek-01-2.ppt BBB human Computer interaction
Week-01-2.ppt BBB human Computer interaction
 
Schema on read is obsolete. Welcome metaprogramming..pdf
Schema on read is obsolete. Welcome metaprogramming..pdfSchema on read is obsolete. Welcome metaprogramming..pdf
Schema on read is obsolete. Welcome metaprogramming..pdf
 
Introduction-to-Machine-Learning (1).pptx
Introduction-to-Machine-Learning (1).pptxIntroduction-to-Machine-Learning (1).pptx
Introduction-to-Machine-Learning (1).pptx
 
Smarteg dropshipping via API with DroFx.pptx
Smarteg dropshipping via API with DroFx.pptxSmarteg dropshipping via API with DroFx.pptx
Smarteg dropshipping via API with DroFx.pptx
 
VIP High Profile Call Girls Amravati Aarushi 8250192130 Independent Escort Se...
VIP High Profile Call Girls Amravati Aarushi 8250192130 Independent Escort Se...VIP High Profile Call Girls Amravati Aarushi 8250192130 Independent Escort Se...
VIP High Profile Call Girls Amravati Aarushi 8250192130 Independent Escort Se...
 
VIP Call Girls Service Miyapur Hyderabad Call +91-8250192130
VIP Call Girls Service Miyapur Hyderabad Call +91-8250192130VIP Call Girls Service Miyapur Hyderabad Call +91-8250192130
VIP Call Girls Service Miyapur Hyderabad Call +91-8250192130
 
B2 Creative Industry Response Evaluation.docx
B2 Creative Industry Response Evaluation.docxB2 Creative Industry Response Evaluation.docx
B2 Creative Industry Response Evaluation.docx
 
FESE Capital Markets Fact Sheet 2024 Q1.pdf
FESE Capital Markets Fact Sheet 2024 Q1.pdfFESE Capital Markets Fact Sheet 2024 Q1.pdf
FESE Capital Markets Fact Sheet 2024 Q1.pdf
 
Brighton SEO | April 2024 | Data Storytelling
Brighton SEO | April 2024 | Data StorytellingBrighton SEO | April 2024 | Data Storytelling
Brighton SEO | April 2024 | Data Storytelling
 

Deep Dive Into Catalyst: Apache Spark's Query Optimizer

  • 1. Deep Dive Into Catalyst: Apache Spark’s Optimizer Herman van Hövell Spark SummitEurope, Brussels October 26th 2016
  • 2. 2 Who is Databricks Why Us Our Product • Created Apache Spark to enable big data use cases with a single engine. • Contributes 75%of Spark’s code • Bring Spark to the enterprise: The just- in-time data platform. • Fully managed platform powered by Apache Spark. • A unified solution for data science and engineering teams.
  • 3. 3 Spark Core (RDD) Catalyst DataFrame/DatasetSQL ML Pipelines Structured Streaming { JSON } JDBC and more… Spark SQL GraphFrames Overview
  • 4. Why structure? • By definition, structure will limit what can be expressed. • In practice, we can accommodate the vast majority of computations. 4 Limiting the space of what can be expressed enables optimizations.
  • 5. Why structure? 5 pdata.map { case (dpt, age) => dpt -> (age, 1) } .reduceByKey { case ((a1, c1), (a2, c2)) => (a1 + a2, c1 + c2)} .map { case (dpt, (age, c)) => dpt -> age/ c } select dept, avg(age) from data group by 1 SQL Dataframe RDD data.groupBy(“dept”).avg(“age”)
  • 6. 6 0 1 2 3 4 5 RDD Dataframe SQL Runtime performance of aggregating 10 million int pairs (secs) Why structure?
  • 7. How? • Write programs using high level programming interfaces • Programs are used to describe what data operations are needed withoutspecifying how to execute those operations • High level programming interfaces: SQL, DataFrames, and Dataset • Get an optimizer that automatically finds out the most efficient plan to execute data operations specified in the user’s program 7
  • 9. 9 SQL AST DataFrame Dataset (Java/Scala) Query Plan Optimized Query Plan RDDs Unresolved Logical Plan Logical Plan Optimized Logical Plan Selected Physical Plan CostModel Physical Plans Catalog Analysis Logical Optimization Physical Planning Catalyst
  • 10. 10 Unresolved Logical Plan Logical Plan Optimized Logical Plan Selected Physical Plan CostModel Physical Plans Catalog Analysis Logical Optimization Physical Planning • Analysis (Rule Executor): Transforms an Unresolved Logical Plan to a Resolved Logical Plan • Unresolved => Resolved: Use Catalog to find where datasets and columns are coming from and types of columns • Logical Optimization (Rule Executor): Transforms a Resolved Logical Plan to an Optimized Logical Plan • Physical Planning (Strategies + Rule Executor): Transforms a Optimized Logical Plan to a Physical Plan
  • 11. 11 How Catalyst Works: An Overview SQL AST DataFrame Dataset (Java/Scala) Query Plan Optimized Query Plan RDDs Code GenerationTransformations Catalyst Abstractions of users’ programs (Trees)
  • 12. 12 How Catalyst Works: An Overview SQL AST DataFrame Dataset (Java/Scala) Query Plan Optimized Query Plan RDDs Code GenerationTransformations Catalyst Abstractions of users’ programs (Trees)
  • 13. 13 Trees: Abstractionsof Users’ Programs SELECT sum(v) FROM ( SELECT t1.id, 1 + 2 + t1.value AS v FROM t1 JOIN t2 WHERE t1.id = t2.id AND t2.id > 50 * 1000) tmp
  • 14. 14 Trees: Abstractionsof Users’ Programs SELECT sum(v) FROM ( SELECT t1.id, 1 + 2 + t1.value AS v FROM t1 JOIN t2 WHERE t1.id = t2.id AND t2.id > 50 * 1000) tmp Expression • An expressionrepresentsa new value, computedbased on input values • e.g. 1 + 2 + t1.value • Attribute: A columnof a dataset(e.g.t1.id) or a column generatedbya specificdata operation(e.g. v)
  • 15. 15 Trees: Abstractionsof Users’ Programs SELECT sum(v) FROM ( SELECT t1.id, 1 + 2 + t1.value AS v FROM t1 JOIN t2 WHERE t1.id = t2.id AND t2.id > 50 * 1000) tmp Query Plan Scan (t1) Scan (t2) Join Filter Project Aggregate sum(v) t1.id, 1+2+t1.value as v t1.id=t2.id t2.id>50*1000
  • 16. Logical Plan • A LogicalPlan describescomputation on datasetswithout defininghow to conductthe computation • output: a listof attributesgenerated by this LogicalPlan, e.g.[id, v] • constraints:a setof invariants about the rows generatedby this plan, e.g. t2.id > 50 * 1000 16 Scan (t1) Scan (t2) Join Filter Project Aggregate sum(v) t1.id, 1+2+t1.value as v t1.id=t2.id t2.id>50*1000
  • 17. Physical Plan • A PhysicalPlan describescomputation on datasetswith specificdefinitionson how to conductthe computation • A PhysicalPlan is executable 17 Parquet Scan (t1) JSON Scan (t2) Sort-Merge Join Filter Project Hash- Aggregate sum(v) t1.id, 1+2+t1.value as v t1.id=t2.id t2.id>50*1000
  • 18. 18 How Catalyst Works: An Overview SQL AST DataFrame Dataset (Java/Scala) Query Plan Optimized Query Plan RDDs Code GenerationTransformations Catalyst Abstractions of users’ programs (Trees)
  • 19. Transformations • Transformationswithoutchangingthetree type (Transform andRule Executor) • Expression => Expression • Logical Plan => Logical Plan • Physical Plan => Physical Plan • Transforminga tree to anotherkind of tree • Logical Plan => Physical Plan 19
  • 20. • A functionassociatedwith everytree usedto implementa singlerule Transform 20 Attribute (t1.value) Add Add Literal(1) Literal(2) 1 + 2 + t1.value Attribute (t1.value) Add Literal(3) 3+ t1.valueEvaluate 1 + 2 onceEvaluate 1 + 2 for every row
  • 21. Transform • A transformationis definedas a Partial Function • Partial Function:A functionthat is definedfor a subset of its possiblearguments 21 val expression: Expression = ... expression.transform { case Add(Literal(x, IntegerType), Literal(y, IntegerType)) => Literal(x + y) } Case statement determine if the partial function is defined for agiven input
  • 22. val expression: Expression = ... expression.transform { case Add(Literal(x, IntegerType), Literal(y, IntegerType)) => Literal(x + y) } Transform 22 Attribute (t1.value) Add Add Literal(1) Literal(2) 1 + 2 + t1.value
  • 23. val expression: Expression = ... expression.transform { case Add(Literal(x, IntegerType), Literal(y, IntegerType)) => Literal(x + y) } Transform 23 Attribute (t1.value) Add Add Literal(1) Literal(2) 1 + 2 + t1.value
  • 24. val expression: Expression = ... expression.transform { case Add(Literal(x, IntegerType), Literal(y, IntegerType)) => Literal(x + y) } Transform 24 Attribute (t1.value) Add Add Literal(1) Literal(2) 1 + 2 + t1.value
  • 25. val expression: Expression = ... expression.transform { case Add(Literal(x, IntegerType), Literal(y, IntegerType)) => Literal(x + y) } Transform 25 Attribute (t1.value) Add Add Literal(1) Literal(2) 1 + 2 + t1.value Attribute (t1.value) Add Literal(3) 3+ t1.value
  • 26. Combining Multiple Rules 26 Scan (t1) Scan (t2) Join Filter Project Aggregate sum(v) t1.id, 1+2+t1.value as v t1.id=t2.id t2.id>50*1000 Predicate Pushdown Scan (t1) Scan (t2) Join Filter Project Aggregate sum(v) t1.id, 1+2+t1.value as v t2.id>50*1000 t1.id=t2.id
  • 27. Combining Multiple Rules 27 Constant Folding Scan (t1) Scan (t2) Join Filter Project Aggregate sum(v) t1.id, 1+2+t1.value as v t2.id>50*1000 t1.id=t2.id Scan (t1) Scan (t2) Join Filter Project Aggregate sum(v) t1.id, 3+t1.value as v t2.id>50000 t1.id=t2.id
  • 28. Combining Multiple Rules 28 Column Pruning Scan (t1) Scan (t2) Join Filter Project Aggregate sum(v) t1.id, 3+t1.value as v t2.id>50000 t1.id=t2.id Scan (t1) Scan (t2) Join Filter Project Aggregate sum(v) t1.id, 3+t1.value as v t2.id>50000 t1.id=t2.id Project Project t1.id t1.value t2.id
  • 29. Combining Multiple Rules 29 Scan (t1) Scan (t2) Join Filter Project Aggregate sum(v) t1.id, 1+2+t1.value as v t1.id=t2.id t2.id>50*1000 Scan (t1) Scan (t2) Join Filter Project Aggregate sum(v) t1.id, 3+t1.value as v t2.id>50000 t1.id=t2.id Project Projectt1.id t1.value t2.id Before transformations After transformations
  • 30. Combining Multiple Rules: Rule Executor 30 Batch 1 Batch 2 Batch n… Rule1 Rule2 … Rule1 Rule2 … A Rule Executor transforms a Tree to another same type Tree by applying many rules defined in batches Every rule is implemented based on Transform Approaches of applying rules 1. Fixed point 2. Once
  • 31. Transformations • Transformationswithoutchangingthetree type (Transform andRule Executor) • Expression => Expression • Logical Plan => Logical Plan • Physical Plan => Physical Plan • Transforminga tree to anotherkind of tree • Logical Plan => Physical Plan 31
  • 32. From Logical Plan to Physical Plan • A Logical Plan is transformed to a Physical Plan by applying a set of Strategies • Every Strategy uses pattern matching to convert a Tree to another kind of Tree 32 object BasicOperators extends Strategy { def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { … case logical.Project(projectList, child) => execution.ProjectExec(projectList, planLater(child)) :: Nil case logical.Filter(condition, child) => execution.FilterExec(condition, planLater(child)) :: Nil … } } Triggers other Strategies
  • 33. Spark’s Planner • 1st Phase:Transformsthe LogicalPlan to the Physical Plan usingStrategies • 2nd Phase:Use a Rule Executorto make the Physical Plan ready forexecution • Prepare Scalar sub-queries • Ensure requirements on input rows • Apply physical optimizations 33
  • 34. Ensure Requirementson Input Rows 34 Sort-Merge Join [t1.id = t2.id] t1 Requirements: Rows Sorted by t1.id t2 Requirements: Rows Sorted by t2.id
  • 35. Ensure Requirementson Input Rows 35 Sort-Merge Join [t1.id = t2.id] t1 Requirements: Rows Sorted by t1.id Requirements: Rows Sorted by t2.id Sort [t1.id] t2 Sort [t2.id]
  • 36. Ensure Requirementson Input Rows 36 Sort-Merge Join [t1.id = t2.id] t1 [sorted by t1.id] Requirements: Rows Sorted by t1.id Requirements: Rows Sorted by t2.id Sort [t1.id] t2 Sort [t2.id]What if t1 has been sorted by t1.id?
  • 37. Ensure Requirementson Input Rows 37 Sort-Merge Join [t1.id = t2.id] t1 [sorted by t1.id] Requirements: Rows Sorted by t1.id Requirements: Rows Sorted by t2.id t2 Sort [t2.id]
  • 38. 38 Roll your own PlanningRule
  • 39. 39 Roll your own Planner Rule import org.apache.spark.sql.functions._ val tableA = spark.range(100000000).as('a) val tableB = spark.range(100000000).as('b) val result = tableA .join(tableB, $"a.id" === $"b.id") .groupBy() .count() result.count() This takes ~22 Seconds on Databricks Community edition
  • 40. 40 Roll your own Planner Rule Can we do better?
  • 41. 41 Roll your own Planner Rule - Analysis == Physical Plan == *HashAggregate(keys=[], functions=[count(1)], output=[count#43L]) +- Exchange SinglePartition +- *HashAggregate(keys=[], functions=[partial_count(1)], output=[count#48L]) +- *Project +- *SortMergeJoin [id#21L], [id#25L], Inner :- *Sort [id#21L ASC], false, 0 : +- Exchange hashpartitioning(id#21L, 200) : +- *Range (0, 100000000, step=1, splits=Some(8)) +- *Sort [id#25L ASC], false, 0 +- Exchange hashpartitioning(id#25L, 200) +- *Range (0, 100000000, step=1, splits=Some(8))
  • 42. 42 Roll your own Planner Rule Exploit the structure of the problem We are joining two intervals; the result will be the intersectionof these intervals A B A∩B
  • 43. 43 Roll your own Planner Rule - Matching case object IntervalJoin extends Strategy with Serializable { def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { case Join( Range(start1, end1, 1, part1, Seq(o1)), Range(start2, end2, 1, part2, Seq(o2)), Inner, Some(EqualTo(e1, e2))) if ((o1 semanticEquals e1) && (o2 semanticEquals e2)) || ((o1 semanticEquals e2) && (o2 semanticEquals e1)) => // Rule... case _ => Nil } }
  • 44. 44 Roll your own Planner Rule - Body if astart1 <= end2) && (end1 >= end2)) { val start = math.max(start1, start2) val end = math.min(end1, end2) val part = math.max(part1.getOrElse(200), part2.getOrElse(200)) val result = RangeExec(Range(start, end, 1, part, o1 :: Nil)) val twoColumns = ProjectExec( Alias(o1, o1.name)(exprId = o1.exprId) :: Nil, result) twoColumns :: Nil } else { Nil }
  • 45. 45 Roll your own Planner Rule Hook it up with Spark spark.experimental.extraStrategies = IntervalJoin :: Nil Use it result.count() This now takes0.46secondsto complete
  • 46. 46 Roll your own Planner Rule == Physical Plan == *HashAggregate(keys=[], functions=[count(1)], output=[count#43L]) +- Exchange SinglePartition +- *HashAggregate(keys=[], functions=[partial_count(1)], output=[count#48L]) +- *Project +- *Project [id#21L AS id#21L] +- *Range (0, 100000000, step=1, splits=Some(8))
  • 47. Community Contributed Transformations 47 110 line patch took this user’s query from “never finishing” to 200s. Overall 200+ people have contributed to the analyzer/optimizer/planner in the last 2 years.
  • 48. Where to Start • Source Code: • Trees: TreeNode, Expression, Logical Plan, and Physical Plan • Transformations: Analyzer, Optimizer, and Planner • Check outprevious pull requests • Start to write code usingCatalyst • Open a pullrequest! 48
  • 49. • Try latestversionof ApacheSpark Try Apache Spark with Databricks 49 http://databricks.com/try