SlideShare a Scribd company logo
Presto on Spark
A Tale of Two Computation Engines
Andrii Rosa
Software Engineer
Wenlei Xie
Research Scientist
Agenda
Introduction
Design & Implementation
Introduction
SQL Use Cases @ Facebook
▪ Reporting and Dashboarding
▪ Low latency (<1s)
▪ High QPS
▪ Presto
▪ Adhoc Analysis
▪ Moderate latency (seconds to minutes)
▪ Mainly Presto
▪ Batch Processing
▪ High latency (up to tens of hours)
▪ Both Presto and Spark
Towards an Unified SQL Experience
▪ Batch Processing Uses Both Presto and Spark
▪ Presto doesn’t scale for large batch pipelines
▪ Inconsistent SQL Experience
▪ SQL Dialect
▪ Subtle Semantic Difference
▪ Null vs. Exception
▪ UDF/UDAF
▪ Best Practice
Presto and Spark Architecture
▪ Designed for latency
▪ MPP Architecture
▪ In-memory shuffle
▪ Shared executor
▪ Designed for Scalability
▪ MapReduce Architecture
▪ Disaggregated shuffle
▪ Isolated executor
SparkPresto
Why Presto (or Other MPPs) Doesn’t Scale?
A Decade-Old Question
SELECT custkey, SUM(totalprice)
FROM orders
GROUP BY custkey
Scan
Scan
Scan
Aggr
Aggr
Aggr
In-memory shuffle
on custkey
Aggr
Execute everything concurrently
- inflexible schedule
- fault-tolerant is difficult
- might exceed memory limit
Presto Unlimited
Brings MapReduce-style execution to MPP architectured runtime
SELECT custkey, SUM(totalprice)
FROM orders
GROUP BY custkey
Scan
Scan
Scan
Write
Write
Write
In-memory shuffle
on custkey
Write
Independent partition execution on
“reducer” side:
- partition-level retry
- schedule a few partitions
concurrently to reduce memory
Aggr
Aggr
Aggr
Aggr
Presto-on-Spark
Executes Presto Evaluation Library on Spark Runtime
SELECT custkey, SUM(totalprice)
FROM orders
GROUP BY custkey
Scan
Scan
Scan
Read
Read
Read
Disagg shuffle
on custkey
Read
Aggr
Aggr
Aggr
Aggr
Stage 1
Stage 2
Why Presto-on-Spark
▪ What are Missing?
▪ Full Disaggregated Shuffle
▪ Isolated Executor
▪ Different Scheduler, Speculative Execution, etc, ...
▪ Embed a “mini-Spark Runtime” inside Presto!
Instead of Making Presto Unlimited More Scalable?
Design & Implementation
Presto-on-Spark Design Principles
▪ Presto is run as a library
▪ Presto cluster is not needed to run
Presto-on-Spark
▪ Presto on Spark is just a Spark application
▪ Query is passed as a parameter
▪ Implemented on RDD level
▪ Operations done by Presto are opaque to Spark
engine
spark-submit
# spark-submit 
--master spark://spark-master:7077 
presto-spark-launcher-*.jar 
--package presto-spark-package-*.tar.gz 
--config ./config.properties 
--catalogs ./catalogs 
--catalog hive 
--schema default 
--file /tmp/query.sql
Planning
Logical PlanQuery Distributed Plan
SELECT *
FROM lineitem l
JOIN orders o
ON l.orderkey = o.orderkey
WHERE o.orderstatus = 'O'
TABLE SCAN
[lineitem]
JOIN
[on orderkey]
TABLE SCAN
[orders]
FILTER
[o.orderstatus = 'O']
Fragment 1 Fragment 2
Fragment 0
PARTITION BY
[orderkey]
PARTITION BY
[orderkey]
FILTER
[o.orderstatus = 'O']
TABLE SCAN
[orders]
TABLE
SCAN
[lineitem]
JOIN
[on orderkey]
Translating to RDD
Fragment 1 Fragment 2
Fragment 0
PARTITION BY
[orderkey]
PARTITION BY
[orderkey]
FILTER
[o.orderstatus = 'O']
TABLE SCAN
[orders]
TABLE
SCAN
[lineitem]
JOIN
[on orderkey]
sparkContext
.parallelize(lineitemSplits)
PairRDD<Integer, Row> = rdd
.mapPartitionsToPair(
fragment1Processor)
sparkContext
.parallelize(ordersSplits)
PairRDD<Integer, Row> = rdd
.mapPartitionsToPair(
fragment2Processor)
pairRdd.partitionBy() pairRdd.partitionBy()
lineitemRdd.zipPartitions(ordersRdd,
fragment0Processor)
Spark DAG
Execution
Fragment 2
FILTER
[o.orderstatus = 'O']
TABLE SCAN
[orders]
Fragment 0
JOIN
[on orderkey]
Leaf Fragment
Intermediate Fragment
Iterator<Tuple2<Integer, PrestoSparkRow>> process(List<Split> splits)
Iterator<Tuple2<Integer, PrestoSparkRow>> process(
List<Iterator<Tuple2<Integer, PrestoSparkRow>>> inputs)
Columnar Format to Row Format Conversion
STAGE 1
INPUT OUTPUTPROJECT FILTERPAGE PAGE PAGEROW ROW
STAGE 2
INPUT OUTPUT
GROUP
BY
FILTERPAGE PAGE PAGEROW ROW
COL 1 VAL 1
COL 1 VAL 2
COL 1 VAL 3
COL 1 VAL 4
COL 1 VAL 5
COL 2 VAL 1
COL 2 VAL 2
COL 2 VAL 3
COL 2 VAL 4
COL 2 VAL 5
COL 3 VAL 1
COL 3 VAL 2
COL 3 VAL 3
COL 3 VAL 4
COL 3 VAL 5
[COL 1 VAL 1], [COL 2 VAL 1], [COL 3 VAL 1]
SHUFFLE
Broadcast Join
Distributed Plan
TABLE SCAN
[lineitem]
JOIN
[on orderkey]
TABLE SCAN
[orders]
FILTER
[o.orderstatus = 'O']
Logical Plan
Fragment 1
Fragment 0
BROADCAST
FILTER
[o.orderstatus = 'O']
TABLE SCAN
[orders]
TABLE
SCAN
[lineitem]
JOIN
[on orderkey]
Job 1
Job 0
Translating to RDD
Fragment 1
Fragment 0
BROADCAST
FILTER
[o.orderstatus = 'O']
TABLE SCAN
[orders]
TABLE
SCAN
[lineitem]
JOIN
[on orderkey]
sparkContext
.parallelize(lineitemSplits)
RDD<Row> = rdd
.mapPartitions(fragment0Processor)
sparkContext
.parallelize(ordersSplits)
RDD<Row> = rdd
.mapPartitions(fragment1Processor)
sc.broadcast(ordersRdd.collect())
Spark DAG
Execution
Broadcast Fragment
Join Fragment
Fragment 1
FILTER
[o.orderstatus = 'O']
TABLE SCAN
[orders]
Iterator<Tuple2<Integer, PrestoSparkRow>> process(List<Split> splits)
Fragment 0
TABLE
SCAN
[lineitem]
JOIN
[on orderkey]
Iterator<Tuple2<Integer, PrestoSparkRow>> process(
List<Split> splits,
List<Broadcast<List<PrestoSparkRow>>> broadcasts)
Threading Model
Presto Task
Spark Task
INPUT OUTPUTPROJECT UNNEST FILTER
INPUT
OUTPUTLOCAL
SHUFFLE
PROJECT UNNEST FILTERPROJECT
PROJECT UNNEST FILTERPROJECT
PROJECT UNNEST FILTERPROJECT
PROJECT UNNEST FILTERPROJECT
PROJECT
Classloader Isolation
Spark Classloader (presto-spark-launcher.jar)
Presto Classloader (presto-spark-package.tar.gz)
int main() {
...
sparkContext.addFile(
“presto-spark-package.tar.gz”
)
...
...
IPrestoSparkService service =
createService(
“presto-spark-package.tar.gz”
)
...
}
Hive Plugin
Classloader
Pinot Plugin
Classloader
MySQL Plugin
Classloader
IPrestoSparkService {
getQueryExecutionFactory();
getTaskExecutorFactory();
}
Current Status
▪ Under Active Development on GitHub: #13856
▪ Most query shapes supported
▪ Working on supporting remaining query shapes (some flavors of UNION ALL)
▪ Preparing the feature to become GA
▪ Initial Scalability Tests
▪ Scale to 10,000 Mappers / Reducers
▪ Supports Queries Require 50TB+ Distributed Memory in Presto
▪ Up to 3x Wall Time Reduction for Presto Large Batch Queries (6h in Presto vs 2h in Presto on Spark)
Q&A

More Related Content

What's hot

Deep Dive: Memory Management in Apache Spark
Deep Dive: Memory Management in Apache SparkDeep Dive: Memory Management in Apache Spark
Deep Dive: Memory Management in Apache Spark
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
 
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
 
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
 
Improving SparkSQL Performance by 30%: How We Optimize Parquet Pushdown and P...
Improving SparkSQL Performance by 30%: How We Optimize Parquet Pushdown and P...Improving SparkSQL Performance by 30%: How We Optimize Parquet Pushdown and P...
Improving SparkSQL Performance by 30%: How We Optimize Parquet Pushdown and P...
Databricks
 
Materialized Column: An Efficient Way to Optimize Queries on Nested Columns
Materialized Column: An Efficient Way to Optimize Queries on Nested ColumnsMaterialized Column: An Efficient Way to Optimize Queries on Nested Columns
Materialized Column: An Efficient Way to Optimize Queries on Nested Columns
Databricks
 
Dynamic Partition Pruning in Apache Spark
Dynamic Partition Pruning in Apache SparkDynamic Partition Pruning in Apache Spark
Dynamic Partition Pruning in Apache Spark
Databricks
 
A Deep Dive into Query Execution Engine of Spark SQL
A Deep Dive into Query Execution Engine of Spark SQLA Deep Dive into Query Execution Engine of Spark SQL
A Deep Dive into Query Execution Engine of Spark SQL
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
 
Making Apache Spark Better with Delta Lake
Making Apache Spark Better with Delta LakeMaking Apache Spark Better with Delta Lake
Making Apache Spark Better with Delta Lake
Databricks
 
Introduction to DataFusion An Embeddable Query Engine Written in Rust
Introduction to DataFusion  An Embeddable Query Engine Written in RustIntroduction to DataFusion  An Embeddable Query Engine Written in Rust
Introduction to DataFusion An Embeddable Query Engine Written in Rust
Andrew Lamb
 
Evening out the uneven: dealing with skew in Flink
Evening out the uneven: dealing with skew in FlinkEvening out the uneven: dealing with skew in Flink
Evening out the uneven: dealing with skew in Flink
Flink Forward
 
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
Noritaka Sekiyama
 
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
 
Advanced Apache Spark Meetup Project Tungsten Nov 12 2015
Advanced Apache Spark Meetup Project Tungsten Nov 12 2015Advanced Apache Spark Meetup Project Tungsten Nov 12 2015
Advanced Apache Spark Meetup Project Tungsten Nov 12 2015
Chris Fregly
 
Understanding Query Plans and Spark UIs
Understanding Query Plans and Spark UIsUnderstanding Query Plans and Spark UIs
Understanding Query Plans and Spark UIs
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
 
A Tale of Three Apache Spark APIs: RDDs, DataFrames, and Datasets with Jules ...
A Tale of Three Apache Spark APIs: RDDs, DataFrames, and Datasets with Jules ...A Tale of Three Apache Spark APIs: RDDs, DataFrames, and Datasets with Jules ...
A Tale of Three Apache Spark APIs: RDDs, DataFrames, and Datasets with Jules ...
Databricks
 
Spark shuffle introduction
Spark shuffle introductionSpark shuffle introduction
Spark shuffle introduction
colorant
 
Native Support of Prometheus Monitoring in Apache Spark 3.0
Native Support of Prometheus Monitoring in Apache Spark 3.0Native Support of Prometheus Monitoring in Apache Spark 3.0
Native Support of Prometheus Monitoring in Apache Spark 3.0
Databricks
 

What's hot (20)

Deep Dive: Memory Management in Apache Spark
Deep Dive: Memory Management in Apache SparkDeep Dive: Memory Management in Apache Spark
Deep Dive: Memory Management in Apache Spark
 
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
 
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
 
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
 
Improving SparkSQL Performance by 30%: How We Optimize Parquet Pushdown and P...
Improving SparkSQL Performance by 30%: How We Optimize Parquet Pushdown and P...Improving SparkSQL Performance by 30%: How We Optimize Parquet Pushdown and P...
Improving SparkSQL Performance by 30%: How We Optimize Parquet Pushdown and P...
 
Materialized Column: An Efficient Way to Optimize Queries on Nested Columns
Materialized Column: An Efficient Way to Optimize Queries on Nested ColumnsMaterialized Column: An Efficient Way to Optimize Queries on Nested Columns
Materialized Column: An Efficient Way to Optimize Queries on Nested Columns
 
Dynamic Partition Pruning in Apache Spark
Dynamic Partition Pruning in Apache SparkDynamic Partition Pruning in Apache Spark
Dynamic Partition Pruning in Apache Spark
 
A Deep Dive into Query Execution Engine of Spark SQL
A Deep Dive into Query Execution Engine of Spark SQLA Deep Dive into Query Execution Engine of Spark SQL
A Deep Dive into Query Execution Engine of Spark SQL
 
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
 
Making Apache Spark Better with Delta Lake
Making Apache Spark Better with Delta LakeMaking Apache Spark Better with Delta Lake
Making Apache Spark Better with Delta Lake
 
Introduction to DataFusion An Embeddable Query Engine Written in Rust
Introduction to DataFusion  An Embeddable Query Engine Written in RustIntroduction to DataFusion  An Embeddable Query Engine Written in Rust
Introduction to DataFusion An Embeddable Query Engine Written in Rust
 
Evening out the uneven: dealing with skew in Flink
Evening out the uneven: dealing with skew in FlinkEvening out the uneven: dealing with skew in Flink
Evening out the uneven: dealing with skew in Flink
 
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
 
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
 
Advanced Apache Spark Meetup Project Tungsten Nov 12 2015
Advanced Apache Spark Meetup Project Tungsten Nov 12 2015Advanced Apache Spark Meetup Project Tungsten Nov 12 2015
Advanced Apache Spark Meetup Project Tungsten Nov 12 2015
 
Understanding Query Plans and Spark UIs
Understanding Query Plans and Spark UIsUnderstanding Query Plans and Spark UIs
Understanding Query Plans and Spark UIs
 
The Apache Spark File Format Ecosystem
The Apache Spark File Format EcosystemThe Apache Spark File Format Ecosystem
The Apache Spark File Format Ecosystem
 
A Tale of Three Apache Spark APIs: RDDs, DataFrames, and Datasets with Jules ...
A Tale of Three Apache Spark APIs: RDDs, DataFrames, and Datasets with Jules ...A Tale of Three Apache Spark APIs: RDDs, DataFrames, and Datasets with Jules ...
A Tale of Three Apache Spark APIs: RDDs, DataFrames, and Datasets with Jules ...
 
Spark shuffle introduction
Spark shuffle introductionSpark shuffle introduction
Spark shuffle introduction
 
Native Support of Prometheus Monitoring in Apache Spark 3.0
Native Support of Prometheus Monitoring in Apache Spark 3.0Native Support of Prometheus Monitoring in Apache Spark 3.0
Native Support of Prometheus Monitoring in Apache Spark 3.0
 

Similar to Presto on Apache Spark: A Tale of Two Computation Engines

Migrating ETL Workflow to Apache Spark at Scale in Pinterest
Migrating ETL Workflow to Apache Spark at Scale in PinterestMigrating ETL Workflow to Apache Spark at Scale in Pinterest
Migrating ETL Workflow to Apache Spark at Scale in Pinterest
Databricks
 
How Spark Does It Internally?
How Spark Does It Internally?How Spark Does It Internally?
How Spark Does It Internally?
Knoldus Inc.
 
Ten tools for ten big data areas 03_Apache Spark
Ten tools for ten big data areas 03_Apache SparkTen tools for ten big data areas 03_Apache Spark
Ten tools for ten big data areas 03_Apache Spark
Will Du
 
Fast federated SQL with Apache Calcite
Fast federated SQL with Apache CalciteFast federated SQL with Apache Calcite
Fast federated SQL with Apache Calcite
Chris Baynes
 
Extreme Apache Spark: how in 3 months we created a pipeline that can process ...
Extreme Apache Spark: how in 3 months we created a pipeline that can process ...Extreme Apache Spark: how in 3 months we created a pipeline that can process ...
Extreme Apache Spark: how in 3 months we created a pipeline that can process ...
Josef A. Habdank
 
Healthcare Claim Reimbursement using Apache Spark
Healthcare Claim Reimbursement using Apache SparkHealthcare Claim Reimbursement using Apache Spark
Healthcare Claim Reimbursement using Apache Spark
Databricks
 
Tuning and Debugging in Apache Spark
Tuning and Debugging in Apache SparkTuning and Debugging in Apache Spark
Tuning and Debugging in Apache Spark
Databricks
 
Introduction to Spark with Python
Introduction to Spark with PythonIntroduction to Spark with Python
Introduction to Spark with Python
Gokhan Atil
 
Tuning and Debugging in Apache Spark
Tuning and Debugging in Apache SparkTuning and Debugging in Apache Spark
Tuning and Debugging in Apache Spark
Patrick Wendell
 
Introduction to Spark Datasets - Functional and relational together at last
Introduction to Spark Datasets - Functional and relational together at lastIntroduction to Spark Datasets - Functional and relational together at last
Introduction to Spark Datasets - Functional and relational together at last
Holden Karau
 
Intro to Spark
Intro to SparkIntro to Spark
Intro to Spark
Kyle Burke
 
Speed up UDFs with GPUs using the RAPIDS Accelerator
Speed up UDFs with GPUs using the RAPIDS AcceleratorSpeed up UDFs with GPUs using the RAPIDS Accelerator
Speed up UDFs with GPUs using the RAPIDS Accelerator
Databricks
 
Introduction to spark 2.0
Introduction to spark 2.0Introduction to spark 2.0
Introduction to spark 2.0
datamantra
 
From Query Plan to Query Performance: Supercharging your Apache Spark Queries...
From Query Plan to Query Performance: Supercharging your Apache Spark Queries...From Query Plan to Query Performance: Supercharging your Apache Spark Queries...
From Query Plan to Query Performance: Supercharging your Apache Spark Queries...
Databricks
 
From HDFS to S3: Migrate Pinterest Apache Spark Clusters
From HDFS to S3: Migrate Pinterest Apache Spark ClustersFrom HDFS to S3: Migrate Pinterest Apache Spark Clusters
From HDFS to S3: Migrate Pinterest Apache Spark Clusters
Databricks
 
Data Processing with Apache Spark Meetup Talk
Data Processing with Apache Spark Meetup TalkData Processing with Apache Spark Meetup Talk
Data Processing with Apache Spark Meetup Talk
Eren Avşaroğulları
 
A Comparative Performance Evaluation of Apache Flink
A Comparative Performance Evaluation of Apache FlinkA Comparative Performance Evaluation of Apache Flink
A Comparative Performance Evaluation of Apache Flink
Dongwon Kim
 
Dongwon Kim – A Comparative Performance Evaluation of Flink
Dongwon Kim – A Comparative Performance Evaluation of FlinkDongwon Kim – A Comparative Performance Evaluation of Flink
Dongwon Kim – A Comparative Performance Evaluation of Flink
Flink Forward
 
Apache Spark: What's under the hood
Apache Spark: What's under the hoodApache Spark: What's under the hood
Apache Spark: What's under the hood
Adarsh Pannu
 
Sqoop on Spark for Data Ingestion
Sqoop on Spark for Data IngestionSqoop on Spark for Data Ingestion
Sqoop on Spark for Data Ingestion
DataWorks Summit
 

Similar to Presto on Apache Spark: A Tale of Two Computation Engines (20)

Migrating ETL Workflow to Apache Spark at Scale in Pinterest
Migrating ETL Workflow to Apache Spark at Scale in PinterestMigrating ETL Workflow to Apache Spark at Scale in Pinterest
Migrating ETL Workflow to Apache Spark at Scale in Pinterest
 
How Spark Does It Internally?
How Spark Does It Internally?How Spark Does It Internally?
How Spark Does It Internally?
 
Ten tools for ten big data areas 03_Apache Spark
Ten tools for ten big data areas 03_Apache SparkTen tools for ten big data areas 03_Apache Spark
Ten tools for ten big data areas 03_Apache Spark
 
Fast federated SQL with Apache Calcite
Fast federated SQL with Apache CalciteFast federated SQL with Apache Calcite
Fast federated SQL with Apache Calcite
 
Extreme Apache Spark: how in 3 months we created a pipeline that can process ...
Extreme Apache Spark: how in 3 months we created a pipeline that can process ...Extreme Apache Spark: how in 3 months we created a pipeline that can process ...
Extreme Apache Spark: how in 3 months we created a pipeline that can process ...
 
Healthcare Claim Reimbursement using Apache Spark
Healthcare Claim Reimbursement using Apache SparkHealthcare Claim Reimbursement using Apache Spark
Healthcare Claim Reimbursement using Apache Spark
 
Tuning and Debugging in Apache Spark
Tuning and Debugging in Apache SparkTuning and Debugging in Apache Spark
Tuning and Debugging in Apache Spark
 
Introduction to Spark with Python
Introduction to Spark with PythonIntroduction to Spark with Python
Introduction to Spark with Python
 
Tuning and Debugging in Apache Spark
Tuning and Debugging in Apache SparkTuning and Debugging in Apache Spark
Tuning and Debugging in Apache Spark
 
Introduction to Spark Datasets - Functional and relational together at last
Introduction to Spark Datasets - Functional and relational together at lastIntroduction to Spark Datasets - Functional and relational together at last
Introduction to Spark Datasets - Functional and relational together at last
 
Intro to Spark
Intro to SparkIntro to Spark
Intro to Spark
 
Speed up UDFs with GPUs using the RAPIDS Accelerator
Speed up UDFs with GPUs using the RAPIDS AcceleratorSpeed up UDFs with GPUs using the RAPIDS Accelerator
Speed up UDFs with GPUs using the RAPIDS Accelerator
 
Introduction to spark 2.0
Introduction to spark 2.0Introduction to spark 2.0
Introduction to spark 2.0
 
From Query Plan to Query Performance: Supercharging your Apache Spark Queries...
From Query Plan to Query Performance: Supercharging your Apache Spark Queries...From Query Plan to Query Performance: Supercharging your Apache Spark Queries...
From Query Plan to Query Performance: Supercharging your Apache Spark Queries...
 
From HDFS to S3: Migrate Pinterest Apache Spark Clusters
From HDFS to S3: Migrate Pinterest Apache Spark ClustersFrom HDFS to S3: Migrate Pinterest Apache Spark Clusters
From HDFS to S3: Migrate Pinterest Apache Spark Clusters
 
Data Processing with Apache Spark Meetup Talk
Data Processing with Apache Spark Meetup TalkData Processing with Apache Spark Meetup Talk
Data Processing with Apache Spark Meetup Talk
 
A Comparative Performance Evaluation of Apache Flink
A Comparative Performance Evaluation of Apache FlinkA Comparative Performance Evaluation of Apache Flink
A Comparative Performance Evaluation of Apache Flink
 
Dongwon Kim – A Comparative Performance Evaluation of Flink
Dongwon Kim – A Comparative Performance Evaluation of FlinkDongwon Kim – A Comparative Performance Evaluation of Flink
Dongwon Kim – A Comparative Performance Evaluation of Flink
 
Apache Spark: What's under the hood
Apache Spark: What's under the hoodApache Spark: What's under the hood
Apache Spark: What's under the hood
 
Sqoop on Spark for Data Ingestion
Sqoop on Spark for Data IngestionSqoop on Spark for Data Ingestion
Sqoop on Spark for Data Ingestion
 

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

Ch03-Managing the Object-Oriented Information Systems Project a.pdf
Ch03-Managing the Object-Oriented Information Systems Project a.pdfCh03-Managing the Object-Oriented Information Systems Project a.pdf
Ch03-Managing the Object-Oriented Information Systems Project a.pdf
haila53
 
Q1’2024 Update: MYCI’s Leap Year Rebound
Q1’2024 Update: MYCI’s Leap Year ReboundQ1’2024 Update: MYCI’s Leap Year Rebound
Q1’2024 Update: MYCI’s Leap Year Rebound
Oppotus
 
一比一原版(NYU毕业证)纽约大学毕业证成绩单
一比一原版(NYU毕业证)纽约大学毕业证成绩单一比一原版(NYU毕业证)纽约大学毕业证成绩单
一比一原版(NYU毕业证)纽约大学毕业证成绩单
ewymefz
 
一比一原版(CBU毕业证)卡普顿大学毕业证如何办理
一比一原版(CBU毕业证)卡普顿大学毕业证如何办理一比一原版(CBU毕业证)卡普顿大学毕业证如何办理
一比一原版(CBU毕业证)卡普顿大学毕业证如何办理
ahzuo
 
社内勉強会資料_LLM Agents                              .
社内勉強会資料_LLM Agents                              .社内勉強会資料_LLM Agents                              .
社内勉強会資料_LLM Agents                              .
NABLAS株式会社
 
一比一原版(CU毕业证)卡尔顿大学毕业证成绩单
一比一原版(CU毕业证)卡尔顿大学毕业证成绩单一比一原版(CU毕业证)卡尔顿大学毕业证成绩单
一比一原版(CU毕业证)卡尔顿大学毕业证成绩单
yhkoc
 
Levelwise PageRank with Loop-Based Dead End Handling Strategy : SHORT REPORT ...
Levelwise PageRank with Loop-Based Dead End Handling Strategy : SHORT REPORT ...Levelwise PageRank with Loop-Based Dead End Handling Strategy : SHORT REPORT ...
Levelwise PageRank with Loop-Based Dead End Handling Strategy : SHORT REPORT ...
Subhajit Sahu
 
一比一原版(UniSA毕业证书)南澳大学毕业证如何办理
一比一原版(UniSA毕业证书)南澳大学毕业证如何办理一比一原版(UniSA毕业证书)南澳大学毕业证如何办理
一比一原版(UniSA毕业证书)南澳大学毕业证如何办理
slg6lamcq
 
一比一原版(CBU毕业证)不列颠海角大学毕业证成绩单
一比一原版(CBU毕业证)不列颠海角大学毕业证成绩单一比一原版(CBU毕业证)不列颠海角大学毕业证成绩单
一比一原版(CBU毕业证)不列颠海角大学毕业证成绩单
nscud
 
Best best suvichar in gujarati english meaning of this sentence as Silk road ...
Best best suvichar in gujarati english meaning of this sentence as Silk road ...Best best suvichar in gujarati english meaning of this sentence as Silk road ...
Best best suvichar in gujarati english meaning of this sentence as Silk road ...
AbhimanyuSinha9
 
一比一原版(ArtEZ毕业证)ArtEZ艺术学院毕业证成绩单
一比一原版(ArtEZ毕业证)ArtEZ艺术学院毕业证成绩单一比一原版(ArtEZ毕业证)ArtEZ艺术学院毕业证成绩单
一比一原版(ArtEZ毕业证)ArtEZ艺术学院毕业证成绩单
vcaxypu
 
一比一原版(UofM毕业证)明尼苏达大学毕业证成绩单
一比一原版(UofM毕业证)明尼苏达大学毕业证成绩单一比一原版(UofM毕业证)明尼苏达大学毕业证成绩单
一比一原版(UofM毕业证)明尼苏达大学毕业证成绩单
ewymefz
 
原版制作(Deakin毕业证书)迪肯大学毕业证学位证一模一样
原版制作(Deakin毕业证书)迪肯大学毕业证学位证一模一样原版制作(Deakin毕业证书)迪肯大学毕业证学位证一模一样
原版制作(Deakin毕业证书)迪肯大学毕业证学位证一模一样
u86oixdj
 
The affect of service quality and online reviews on customer loyalty in the E...
The affect of service quality and online reviews on customer loyalty in the E...The affect of service quality and online reviews on customer loyalty in the E...
The affect of service quality and online reviews on customer loyalty in the E...
jerlynmaetalle
 
Opendatabay - Open Data Marketplace.pptx
Opendatabay - Open Data Marketplace.pptxOpendatabay - Open Data Marketplace.pptx
Opendatabay - Open Data Marketplace.pptx
Opendatabay
 
一比一原版(QU毕业证)皇后大学毕业证成绩单
一比一原版(QU毕业证)皇后大学毕业证成绩单一比一原版(QU毕业证)皇后大学毕业证成绩单
一比一原版(QU毕业证)皇后大学毕业证成绩单
enxupq
 
FP Growth Algorithm and its Applications
FP Growth Algorithm and its ApplicationsFP Growth Algorithm and its Applications
FP Growth Algorithm and its Applications
MaleehaSheikh2
 
一比一原版(YU毕业证)约克大学毕业证成绩单
一比一原版(YU毕业证)约克大学毕业证成绩单一比一原版(YU毕业证)约克大学毕业证成绩单
一比一原版(YU毕业证)约克大学毕业证成绩单
enxupq
 
一比一原版(UPenn毕业证)宾夕法尼亚大学毕业证成绩单
一比一原版(UPenn毕业证)宾夕法尼亚大学毕业证成绩单一比一原版(UPenn毕业证)宾夕法尼亚大学毕业证成绩单
一比一原版(UPenn毕业证)宾夕法尼亚大学毕业证成绩单
ewymefz
 
一比一原版(UofS毕业证书)萨省大学毕业证如何办理
一比一原版(UofS毕业证书)萨省大学毕业证如何办理一比一原版(UofS毕业证书)萨省大学毕业证如何办理
一比一原版(UofS毕业证书)萨省大学毕业证如何办理
v3tuleee
 

Recently uploaded (20)

Ch03-Managing the Object-Oriented Information Systems Project a.pdf
Ch03-Managing the Object-Oriented Information Systems Project a.pdfCh03-Managing the Object-Oriented Information Systems Project a.pdf
Ch03-Managing the Object-Oriented Information Systems Project a.pdf
 
Q1’2024 Update: MYCI’s Leap Year Rebound
Q1’2024 Update: MYCI’s Leap Year ReboundQ1’2024 Update: MYCI’s Leap Year Rebound
Q1’2024 Update: MYCI’s Leap Year Rebound
 
一比一原版(NYU毕业证)纽约大学毕业证成绩单
一比一原版(NYU毕业证)纽约大学毕业证成绩单一比一原版(NYU毕业证)纽约大学毕业证成绩单
一比一原版(NYU毕业证)纽约大学毕业证成绩单
 
一比一原版(CBU毕业证)卡普顿大学毕业证如何办理
一比一原版(CBU毕业证)卡普顿大学毕业证如何办理一比一原版(CBU毕业证)卡普顿大学毕业证如何办理
一比一原版(CBU毕业证)卡普顿大学毕业证如何办理
 
社内勉強会資料_LLM Agents                              .
社内勉強会資料_LLM Agents                              .社内勉強会資料_LLM Agents                              .
社内勉強会資料_LLM Agents                              .
 
一比一原版(CU毕业证)卡尔顿大学毕业证成绩单
一比一原版(CU毕业证)卡尔顿大学毕业证成绩单一比一原版(CU毕业证)卡尔顿大学毕业证成绩单
一比一原版(CU毕业证)卡尔顿大学毕业证成绩单
 
Levelwise PageRank with Loop-Based Dead End Handling Strategy : SHORT REPORT ...
Levelwise PageRank with Loop-Based Dead End Handling Strategy : SHORT REPORT ...Levelwise PageRank with Loop-Based Dead End Handling Strategy : SHORT REPORT ...
Levelwise PageRank with Loop-Based Dead End Handling Strategy : SHORT REPORT ...
 
一比一原版(UniSA毕业证书)南澳大学毕业证如何办理
一比一原版(UniSA毕业证书)南澳大学毕业证如何办理一比一原版(UniSA毕业证书)南澳大学毕业证如何办理
一比一原版(UniSA毕业证书)南澳大学毕业证如何办理
 
一比一原版(CBU毕业证)不列颠海角大学毕业证成绩单
一比一原版(CBU毕业证)不列颠海角大学毕业证成绩单一比一原版(CBU毕业证)不列颠海角大学毕业证成绩单
一比一原版(CBU毕业证)不列颠海角大学毕业证成绩单
 
Best best suvichar in gujarati english meaning of this sentence as Silk road ...
Best best suvichar in gujarati english meaning of this sentence as Silk road ...Best best suvichar in gujarati english meaning of this sentence as Silk road ...
Best best suvichar in gujarati english meaning of this sentence as Silk road ...
 
一比一原版(ArtEZ毕业证)ArtEZ艺术学院毕业证成绩单
一比一原版(ArtEZ毕业证)ArtEZ艺术学院毕业证成绩单一比一原版(ArtEZ毕业证)ArtEZ艺术学院毕业证成绩单
一比一原版(ArtEZ毕业证)ArtEZ艺术学院毕业证成绩单
 
一比一原版(UofM毕业证)明尼苏达大学毕业证成绩单
一比一原版(UofM毕业证)明尼苏达大学毕业证成绩单一比一原版(UofM毕业证)明尼苏达大学毕业证成绩单
一比一原版(UofM毕业证)明尼苏达大学毕业证成绩单
 
原版制作(Deakin毕业证书)迪肯大学毕业证学位证一模一样
原版制作(Deakin毕业证书)迪肯大学毕业证学位证一模一样原版制作(Deakin毕业证书)迪肯大学毕业证学位证一模一样
原版制作(Deakin毕业证书)迪肯大学毕业证学位证一模一样
 
The affect of service quality and online reviews on customer loyalty in the E...
The affect of service quality and online reviews on customer loyalty in the E...The affect of service quality and online reviews on customer loyalty in the E...
The affect of service quality and online reviews on customer loyalty in the E...
 
Opendatabay - Open Data Marketplace.pptx
Opendatabay - Open Data Marketplace.pptxOpendatabay - Open Data Marketplace.pptx
Opendatabay - Open Data Marketplace.pptx
 
一比一原版(QU毕业证)皇后大学毕业证成绩单
一比一原版(QU毕业证)皇后大学毕业证成绩单一比一原版(QU毕业证)皇后大学毕业证成绩单
一比一原版(QU毕业证)皇后大学毕业证成绩单
 
FP Growth Algorithm and its Applications
FP Growth Algorithm and its ApplicationsFP Growth Algorithm and its Applications
FP Growth Algorithm and its Applications
 
一比一原版(YU毕业证)约克大学毕业证成绩单
一比一原版(YU毕业证)约克大学毕业证成绩单一比一原版(YU毕业证)约克大学毕业证成绩单
一比一原版(YU毕业证)约克大学毕业证成绩单
 
一比一原版(UPenn毕业证)宾夕法尼亚大学毕业证成绩单
一比一原版(UPenn毕业证)宾夕法尼亚大学毕业证成绩单一比一原版(UPenn毕业证)宾夕法尼亚大学毕业证成绩单
一比一原版(UPenn毕业证)宾夕法尼亚大学毕业证成绩单
 
一比一原版(UofS毕业证书)萨省大学毕业证如何办理
一比一原版(UofS毕业证书)萨省大学毕业证如何办理一比一原版(UofS毕业证书)萨省大学毕业证如何办理
一比一原版(UofS毕业证书)萨省大学毕业证如何办理
 

Presto on Apache Spark: A Tale of Two Computation Engines

  • 1.
  • 2. Presto on Spark A Tale of Two Computation Engines Andrii Rosa Software Engineer Wenlei Xie Research Scientist
  • 5. SQL Use Cases @ Facebook ▪ Reporting and Dashboarding ▪ Low latency (<1s) ▪ High QPS ▪ Presto ▪ Adhoc Analysis ▪ Moderate latency (seconds to minutes) ▪ Mainly Presto ▪ Batch Processing ▪ High latency (up to tens of hours) ▪ Both Presto and Spark
  • 6. Towards an Unified SQL Experience ▪ Batch Processing Uses Both Presto and Spark ▪ Presto doesn’t scale for large batch pipelines ▪ Inconsistent SQL Experience ▪ SQL Dialect ▪ Subtle Semantic Difference ▪ Null vs. Exception ▪ UDF/UDAF ▪ Best Practice
  • 7. Presto and Spark Architecture ▪ Designed for latency ▪ MPP Architecture ▪ In-memory shuffle ▪ Shared executor ▪ Designed for Scalability ▪ MapReduce Architecture ▪ Disaggregated shuffle ▪ Isolated executor SparkPresto
  • 8. Why Presto (or Other MPPs) Doesn’t Scale? A Decade-Old Question SELECT custkey, SUM(totalprice) FROM orders GROUP BY custkey Scan Scan Scan Aggr Aggr Aggr In-memory shuffle on custkey Aggr Execute everything concurrently - inflexible schedule - fault-tolerant is difficult - might exceed memory limit
  • 9. Presto Unlimited Brings MapReduce-style execution to MPP architectured runtime SELECT custkey, SUM(totalprice) FROM orders GROUP BY custkey Scan Scan Scan Write Write Write In-memory shuffle on custkey Write Independent partition execution on “reducer” side: - partition-level retry - schedule a few partitions concurrently to reduce memory Aggr Aggr Aggr Aggr
  • 10. Presto-on-Spark Executes Presto Evaluation Library on Spark Runtime SELECT custkey, SUM(totalprice) FROM orders GROUP BY custkey Scan Scan Scan Read Read Read Disagg shuffle on custkey Read Aggr Aggr Aggr Aggr Stage 1 Stage 2
  • 11. Why Presto-on-Spark ▪ What are Missing? ▪ Full Disaggregated Shuffle ▪ Isolated Executor ▪ Different Scheduler, Speculative Execution, etc, ... ▪ Embed a “mini-Spark Runtime” inside Presto! Instead of Making Presto Unlimited More Scalable?
  • 13. Presto-on-Spark Design Principles ▪ Presto is run as a library ▪ Presto cluster is not needed to run Presto-on-Spark ▪ Presto on Spark is just a Spark application ▪ Query is passed as a parameter ▪ Implemented on RDD level ▪ Operations done by Presto are opaque to Spark engine spark-submit # spark-submit --master spark://spark-master:7077 presto-spark-launcher-*.jar --package presto-spark-package-*.tar.gz --config ./config.properties --catalogs ./catalogs --catalog hive --schema default --file /tmp/query.sql
  • 14. Planning Logical PlanQuery Distributed Plan SELECT * FROM lineitem l JOIN orders o ON l.orderkey = o.orderkey WHERE o.orderstatus = 'O' TABLE SCAN [lineitem] JOIN [on orderkey] TABLE SCAN [orders] FILTER [o.orderstatus = 'O'] Fragment 1 Fragment 2 Fragment 0 PARTITION BY [orderkey] PARTITION BY [orderkey] FILTER [o.orderstatus = 'O'] TABLE SCAN [orders] TABLE SCAN [lineitem] JOIN [on orderkey]
  • 15. Translating to RDD Fragment 1 Fragment 2 Fragment 0 PARTITION BY [orderkey] PARTITION BY [orderkey] FILTER [o.orderstatus = 'O'] TABLE SCAN [orders] TABLE SCAN [lineitem] JOIN [on orderkey] sparkContext .parallelize(lineitemSplits) PairRDD<Integer, Row> = rdd .mapPartitionsToPair( fragment1Processor) sparkContext .parallelize(ordersSplits) PairRDD<Integer, Row> = rdd .mapPartitionsToPair( fragment2Processor) pairRdd.partitionBy() pairRdd.partitionBy() lineitemRdd.zipPartitions(ordersRdd, fragment0Processor)
  • 17. Execution Fragment 2 FILTER [o.orderstatus = 'O'] TABLE SCAN [orders] Fragment 0 JOIN [on orderkey] Leaf Fragment Intermediate Fragment Iterator<Tuple2<Integer, PrestoSparkRow>> process(List<Split> splits) Iterator<Tuple2<Integer, PrestoSparkRow>> process( List<Iterator<Tuple2<Integer, PrestoSparkRow>>> inputs)
  • 18. Columnar Format to Row Format Conversion STAGE 1 INPUT OUTPUTPROJECT FILTERPAGE PAGE PAGEROW ROW STAGE 2 INPUT OUTPUT GROUP BY FILTERPAGE PAGE PAGEROW ROW COL 1 VAL 1 COL 1 VAL 2 COL 1 VAL 3 COL 1 VAL 4 COL 1 VAL 5 COL 2 VAL 1 COL 2 VAL 2 COL 2 VAL 3 COL 2 VAL 4 COL 2 VAL 5 COL 3 VAL 1 COL 3 VAL 2 COL 3 VAL 3 COL 3 VAL 4 COL 3 VAL 5 [COL 1 VAL 1], [COL 2 VAL 1], [COL 3 VAL 1] SHUFFLE
  • 19. Broadcast Join Distributed Plan TABLE SCAN [lineitem] JOIN [on orderkey] TABLE SCAN [orders] FILTER [o.orderstatus = 'O'] Logical Plan Fragment 1 Fragment 0 BROADCAST FILTER [o.orderstatus = 'O'] TABLE SCAN [orders] TABLE SCAN [lineitem] JOIN [on orderkey]
  • 20. Job 1 Job 0 Translating to RDD Fragment 1 Fragment 0 BROADCAST FILTER [o.orderstatus = 'O'] TABLE SCAN [orders] TABLE SCAN [lineitem] JOIN [on orderkey] sparkContext .parallelize(lineitemSplits) RDD<Row> = rdd .mapPartitions(fragment0Processor) sparkContext .parallelize(ordersSplits) RDD<Row> = rdd .mapPartitions(fragment1Processor) sc.broadcast(ordersRdd.collect())
  • 22. Execution Broadcast Fragment Join Fragment Fragment 1 FILTER [o.orderstatus = 'O'] TABLE SCAN [orders] Iterator<Tuple2<Integer, PrestoSparkRow>> process(List<Split> splits) Fragment 0 TABLE SCAN [lineitem] JOIN [on orderkey] Iterator<Tuple2<Integer, PrestoSparkRow>> process( List<Split> splits, List<Broadcast<List<PrestoSparkRow>>> broadcasts)
  • 23. Threading Model Presto Task Spark Task INPUT OUTPUTPROJECT UNNEST FILTER INPUT OUTPUTLOCAL SHUFFLE PROJECT UNNEST FILTERPROJECT PROJECT UNNEST FILTERPROJECT PROJECT UNNEST FILTERPROJECT PROJECT UNNEST FILTERPROJECT PROJECT
  • 24. Classloader Isolation Spark Classloader (presto-spark-launcher.jar) Presto Classloader (presto-spark-package.tar.gz) int main() { ... sparkContext.addFile( “presto-spark-package.tar.gz” ) ... ... IPrestoSparkService service = createService( “presto-spark-package.tar.gz” ) ... } Hive Plugin Classloader Pinot Plugin Classloader MySQL Plugin Classloader IPrestoSparkService { getQueryExecutionFactory(); getTaskExecutorFactory(); }
  • 25. Current Status ▪ Under Active Development on GitHub: #13856 ▪ Most query shapes supported ▪ Working on supporting remaining query shapes (some flavors of UNION ALL) ▪ Preparing the feature to become GA ▪ Initial Scalability Tests ▪ Scale to 10,000 Mappers / Reducers ▪ Supports Queries Require 50TB+ Distributed Memory in Presto ▪ Up to 3x Wall Time Reduction for Presto Large Batch Queries (6h in Presto vs 2h in Presto on Spark)
  • 26. Q&A