SlideShare a Scribd company logo
1 of 43
Download to read offline
Skew Mitigation For Facebook’s
Petabyte-Scale Joins
Suganthi Dewakar & Guanzhong (Tony) Xu
Agenda
Skew Join Journey
▪ Skew Hint
▪ Runtime Skew Mitigation
▪ Customized AQE Skew Mitigation
Cosco + Skew Join
▪ Shuffle Recap
▪ Working with Original Cosco
▪ Splitting in File Boundaries
What Is Data Skew
▪ Uneven distribution of partitioned data
▪ Affects aggregate operations e.g Group By and Join
▪ E.g. count number of people group by country
▪ Data skew at FB
▪ PB sized table Joins with TB sized skewed partitions
▪ Task latency
▪ Non-skewed partition: mins to few hours
▪ Skewed partition: hours to days
▪ Skewed pipelines
▪ Latency sensitive daily jobs
▪ Complex DAG with several upstream/downstream dependencies
▪ Can cause delay in hundreds of downsteam pipelines
0.01GB 1TB
Percentile
shuffle
partition 1
partition 2
partition 3 (skewed)
partition 4
Data Skew In Join
▪ Join strategies in Spark
▪ SortMergeJoin/ ShuffleHashJoin
▪ Not skew resistant – requires input data to be partitioned
▪ BroadcastHashJoin
▪ Skew resistant – requires no data partitioning
▪ Requires one of side of join to be small
▪ Skew Join
▪ Hybrid join that combines above strategies
Mitigating Data Skew In Joins
Skew Hint
Runtime skew
mitigation
Customized AQE
skew mitigation
1 2 3
Optimizer rule-based solution Built on Spark 2.0
adaptive framework
Based on Spark 3.0
AQE framework
Mitigating Data Skew In Joins
Skew Hint
Runtime skew
mitigation
Customized AQE
skew mitigation
1 2 3
Optimizer rule-based solution Built on Spark 2.0
adaptive framework
Based on Spark 3.0
AQE framework
Skew Hint
▪ User provides skew information as
hints
▪ /*+ SKEWED_ON(column=value1,..) */
▪ Split input into two parts
▪ Skewed keys
▪ Non-skewed keys
▪ Broadcast join skewed keys
▪ Shuffle/Sort join non-skewed keys
▪ Union both join outputs
Skew Hint
Scan table_A Scan table_B
SortMergeJoin
Exchange Exchange
Skew Hint
Union
Scan table_A Scan table_B
Filter
[skewed keys]
BroadcastHashJoin
Filter
[skewed keys]
Scan table_A Scan table_B
SortMergeJoin
Filter
[non-skewed keys]
Filter
[non-skewed keys]
Exchange Exchange
Skewed dataNon-skewed data
Skew Hint
Skew Hint
Pros
• Reduces runtime latency
Cons
• Requires prior knowledge of skewed
keys
• Double scan of data
Mitigating Data Skew In Joins
Skew Hint
Runtime skew
mitigation
Customized AQE
skew mitigation
1 2 3
Optimizer rule-based solution Built on Spark 2.0
adaptive framework
Based on Spark 3.0
AQE framework
Runtime skew mitigation
§ Spark 2.0 adaptive framework
§ Adds ExchangeCoordinator to all shuffle Exchange nodes
§ ExchangeCoordinator merges small partitions based on runtime stats
§ Runtime skewed partition detection
§ Collect MapOutputStatistics of a joins input stages
§ Partition is skewed iff size of partition is:
§ > min_threshold_config_value (e.g. 1GB)
§ > median_ratio X median_size(shuffle) (e.g. 10 times median size)
§ > pct_99_ratio X pct99_size(shuffle) (e.g. 3 times 99 percentile size)
Built on Spark 2.0 adaptive framework
Runtime skew mitigation
§ Split large partitions into smaller sub-partitions
shuffle table_A shuffle table_B
skewed partition split
Built on Spark 2.0 adaptive framework
table_A join table_B
Built on Spark 2.0 adaptive framework
Skew Hint
Runtime Skew Mitigation
Spark 2.0 adaptive framework
Pros
• Reduces runtime latency
Cons
• Requires prior knowledge of skewed
keys
• Double scan of data
Pros
• Reduces runtime latency
• No prior skew key knowledge
required
• No double scan of data
Cons
Additional shuffles even when a join is
not skewed
• Between joins
• Between joins and aggregates
Runtime skew mitigation
Mitigating Data Skew In Joins
Skew Hint
Runtime skew
mitigation
Customized AQE
skew mitigation
1 2 3
Optimizer rule-based solution Built on Spark 2.0
adaptive framework
Based on Spark 3.0
AQE framework
AQE skew mitigation
▪ AQE recap
▪ New adaptive framework in OSS
▪ Features
▪ Switching join strategy
▪ Small shuffle partitions coalescing
▪ Optimizing skew join
▪ Runtime DAG change
Spark 3.0 adaptive framework
AQE skew mitigation
Limitation
▪ Supports two table join only
▪ Single stage skewed join + reducer operation
▪ Skewed Join + Aggregate
▪ Skewed multi table joins
▪ Single stage skewed join + mapper operation
▪ Skewed join + union
▪ Skewed join + broadcast join
Customized AQE skew mitigation
Customization 1: Runtime Shuffle Insertion
Lifecycle of AQE execution
EnsureRequirements
Execute
Stats from previous stage
No new Exchange beyond
this point
Optimize skew
Create sub-stages
Re-plan
Customized AQE skew mitigation
Customization 1: Runtime Shuffle Insertion
Lifecycle of AQE execution
EnsureRequirements
Execute
Stats from previous stage
No new Exchange beyond
this point
Optimize skew
Create sub-stages
Re-plan
If skewed, set
outputPartitioning of join to
Unknown
Detect skew
table_A (petabye) table_B
Left Outer Join
table_C
Left Outer Join
Customized AQE skew mitigation
[Limitation] Customization 1: Runtime Shuffle Insertion
table_D
Left Outer Join
table_A (petabye) table_B
Left Outer Join
table_C
Left Outer Join
Customized AQE skew mitigation
[Limitation] Customization 1: Runtime Shuffle Insertion
table_D
Left Outer Join
Exchange
Exchange
Shuffles PB sized
data
Shuffles PB sized
data
Customized AQE skew mitigation
Customization 2: No Shuffle Multi Table Join
shuffle table_A
shuffle table_B
skewed partition split
shuffle table_C
▪ Split skewed partition in one table
▪ Replicate the corresponding partitions in the rest of the input tables
shuffle table_Dshuffle table_A
Skew Hint
Runtime Skew Mitigation
Spark 2.0 adaptive framework
Customized AQE Skew Mitigation
Spark 3.0 adaptive framework
Pros
• Reduces runtime latency
Cons
• Requires prior knowledge of skewed
keys
• Double scan of data
Pros
• Reduces runtime latency
• No prior skew key knowledge
required
• No double scan of data
Cons
Additional shuffles even when a join is
not skewed
• Between joins
• Between joins and aggregates
Pros
• Reduces runtime latency
• No prior skew key
knowledge required
• No double scan of data
• No unnecessary shuffles when a join
is not skewed
Customized AQE skew mitigation
Customization 2: No Shuffle Multi Table Join
Skew Join With Cosco
▪ Adaptive Skew Mitigation Splits in Mapper Boundaries
▪ Each sub-reducer is only required to read data from a subset of mappers.
▪ Only partial data is required
▪ E.g.: a skewed partition 0 is split that: sub-reducer 0 reads {mapper 1}, sub-reducer 1 reads {mapper 0, mapper 2}, sub-reducer 1
reads {mapper 0, mapper 2}
Partition 0
The Problem
File 0 File 1 File 3
Sub-reducer 0 Sub-reducer 1 Sub-reducer 2 Sub-reducer i…
Sub-reducer i
Mapper outputs
are merged in
files :(
Cosco Shuffle Recap
▪ Why Cosco?
▪ Shuffle as a service: IO Efficiency
▪ Write-ahead buffer
▪ Groups shuffled output based on partition id instead of mapper id.
▪ Data deduplication is performed in reducer side
▪ Efficiency Wins:
▪ Solve the write amplification problem: avoid spills: 3X -> 1X
▪ Solve the small IO problem: average IO size: 200KB -> 2.5MB
▪ Previous Talk
▪ Cosco: An Efficient Facebook-Scale Shuffle Service
DFS
Cosco
Shuffle ServiceCosco
Shuffle Service
Cosco Shuffle Recap
Cosco
Shuffle Service
Partition 0
(file 2 buffer)
Partition 0
(file 1 buffer)
Mapper 0
Mapper 0
Mapper 0
Reducer 0
Reducer 1
File 1
File 0
File 2
File 0
File 1
Cosco
Metadata Service
Working with Mapper Boundaries
▪ Load all data but filter out uninterested ones
▪ Each sub-reducer still has to load all data from all file chunks of the partition.
▪ Filtering out records from uninterested mappers.
▪ IO inefficient
Record (mapper-1)
Record (mapper-3)
Record (mapper-2)
Record (mapper-2)
Record (mapper-4)
Record (mapper-1)
…
File 0
Read
Drop
Drop
Drop
Read
Read
Sub-reducer for
mapper {1, 4}
Skewed partition worst cases
▪ For the most heavily skewed partitions, it
could be up to XXX TB and requires XX k
splits, which means the skewed partition
data has to be read XX k times as well.
▪ Very inefficient, most times are spent on
disk IO.
▪ Compared to vanilla Spark shuffle, it no
longer has any advantages in terms of IO.
Why cannot we split in file boundaries?
Terminology
▪ Record
▪ Unit of a data row
▪ Package
▪ Unit to be sent from mapper to shuffle service
▪ Chunk
▪ Unit to be flushed
Record
Record
Record
Record
Package
Package
Package
Package
Chunk
Multiple records to form a package
Multiple packages to form a chunk
▪ Duplicated records
▪ A record could be duplicated in multiple files due to package resending which might be caused by various reasons.
▪ Cosco relies on reducer client to perform deduplication which requires it to read the entire partition data from a mapper.
▪ Simply splitting on file boundaries requires no overlap of mappers between file sets for different sub-reducers, which is not feasible.
File 2 File 3
Restrictions
Sub-reducer 0
File 0 File 1
Sub-reducer 0 is assigned to
read file 0 & 1. But it has to
read file 2 & 3 as well
because they have mapper
overlap with file 0 & 1.
Duplication
▪ Mapper failure
▪ Each mapper is assigned a unique id.
▪ Each package is attached with its mapper’s unique id.
▪ Reducer consumes records from interested mappers based on those unique ids.
▪ It would not cause duplicated records in reducer if a mapper output is not fully read by a reducer.
Mapper 0
(id: abc)
Mapper 0
(id: xyz)
Partition Package Id MapperId
...
0 3 abc
0 3 xyz
...
Failed
Sub-reducer 0
(read from `xyz`)
Dropped
Accepted
Duplication
▪ Network issues
▪ A mapper fails to receive ack message after timeout, it will resend the package.
▪ We call such packages "suspected packages”: a suspected package is not necessary to be a duplicated packaged, but a duplicated
package is always a suspected package.
Shuffle service - 1
Shuffle service - 2Package-1
Package-2
Package-3
Mapper - 1
Ack Queue
Chunk - i
Records from package-
{1,2,3} are flushed to chunks
Chunk - j
Records from package-
{1,2,3} are duplicated to
chunks
Resend packages
in ack queues
Identify suspected packages in mapper
Suspected Map
▪ Map<partition, Map<packageId, chunkId>>
▪ Each mapper keeps a map to track the resent packages due to connection.
▪ Whenever a resend happens, all the packages in the ACK queue are added to the map.
Shuffle service - 1
Shuffle service - 2
Package-1
Pakcage-2
Package-3
Mapper - 1
ACK Queue
Resend packages
in ack queues
Add to suspected
package map
packageId chunkId
Package-x Chunk-xyz
Package-1 null
Package-2 null
Package-3 null
Partition - 1
Suspected Map
▪ Map<partition, Map<packageId, chunkId>>
▪ An ACK message eventually returned contains a unique identifier of the chunk containing the package.
▪ Mapper keeps the mapping from those suspected packages to their authorized chunks.
▪ Once the mapper is done, reports the suspected map to Spark driver.
▪ Spark driver aggregate the suspected package info from different mappers.
Shuffle service - 2
Pakcage-2
Package-3
Mapper - 1
ACK Queue
Return ACK for Package-1
with chunkId
1. Remove package-1
from ACK queue.
2. Update the chunkId in
suspected map
packageId chunkId
Package-x Chunk-xyz
Package-1 Chunk-abc
Package-2 null
Package-3 null
Partition - 1
▪ Each sub-reducer would be assigned a subset of files along with the
corresponding suspected map of the partition.
Reading subset of files on sub-reducer
Chunk Set - 1 Chunk Set - 2
...
Skewed Partition
Sub-reducer - 1 Sub-reducer - 2 ...
▪ Duplicated records would only be accepted if it is read from
authorized chunk.
Reading subset of files on sub-reducer
Record
{mapper-3,package-4,chunk-15}
Chunk-15
Record
{mapper-3,package-4,chunk-15}
Chunk-4
Sub-reducer-2 Sub-reducer-7
Record Accepted Record Dropped
{MapperId, PackageId} ChunkId
... ...
{mapper-3, package-4} chunk-15
... ...
Chunk Lost
▪ Chunk lost failure requires restart of all sub-reducers accordingly: .
▪ Spark driver to restart mappers to regenerate data of that particular partition.
▪ Data in new chunks are non-deterministic.
▪ Spark driver broadcasts failures to all corresponding sub-reducer tasks and restarts them all.
▪ This is rare since chunks are RS-encoded
Putting It All Together
▪ Motivation:
▪ To split a skewed partition into multiple non-overlapping file sets that each sub-reducer would be assigned one of them.
▪ Mapper:
▪ Keeps tracking of suspected packages along with authorized chunks.
▪ Report suspected package info to Spark driver when finishes.
▪ Driver:
▪ Aggregate suspect package info from all mappers and pass along to sub-reducers if necessary.
▪ Detects skewed partitions and split them based on file boundaries of each partition.
▪ Fail and restart all corresponding sub-reducer tasks of a partition if there is a chunk lost.
▪ Reducer:
▪ Fetches file list from driver along with suspected package info.
▪ Accept a record of a suspected package only if it is read from the authorized chunk.
▪ Data Correctness
▪ End-to-end checksum
Summary
Skew Join Journey
▪ Skew Hint
▪ Runtime Skew Mitigation
▪ Customized AQE Skew Mitigation
Cosco + Skew Join
▪ Shuffle Recap
▪ Working with Original Cosco
▪ Splitting in File Boundaries
Feedback
Your feedback is important to us.
Don’t forget to rate
and review the sessions.

More Related Content

What's hot

[Hadoop Meetup] Yarn at Microsoft - The challenges of scale
[Hadoop Meetup] Yarn at Microsoft - The challenges of scale[Hadoop Meetup] Yarn at Microsoft - The challenges of scale
[Hadoop Meetup] Yarn at Microsoft - The challenges of scaleNewton Alex
 
Realtime olap architecture in apache kylin 3.0
Realtime olap architecture in apache kylin 3.0Realtime olap architecture in apache kylin 3.0
Realtime olap architecture in apache kylin 3.0Shi Shao Feng
 
[Hadoop Meetup] Apache Hadoop 3 community update - Rohith Sharma
[Hadoop Meetup] Apache Hadoop 3 community update - Rohith Sharma[Hadoop Meetup] Apache Hadoop 3 community update - Rohith Sharma
[Hadoop Meetup] Apache Hadoop 3 community update - Rohith SharmaNewton Alex
 
Building large scale transactional data lake using apache hudi
Building large scale transactional data lake using apache hudiBuilding large scale transactional data lake using apache hudi
Building large scale transactional data lake using apache hudiBill Liu
 
Hadoop 3.0 - Revolution or evolution?
Hadoop 3.0 - Revolution or evolution?Hadoop 3.0 - Revolution or evolution?
Hadoop 3.0 - Revolution or evolution?Uwe Printz
 
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 FlinkDongwon Kim
 
Hive join optimizations
Hive join optimizationsHive join optimizations
Hive join optimizationsSzehon Ho
 
Apache HBase 1.0 Release
Apache HBase 1.0 ReleaseApache HBase 1.0 Release
Apache HBase 1.0 ReleaseNick Dimiduk
 
Millions of Regions in HBase: Size Matters
Millions of Regions in HBase: Size MattersMillions of Regions in HBase: Size Matters
Millions of Regions in HBase: Size MattersDataWorks Summit
 
Flexible and Real-Time Stream Processing with Apache Flink
Flexible and Real-Time Stream Processing with Apache FlinkFlexible and Real-Time Stream Processing with Apache Flink
Flexible and Real-Time Stream Processing with Apache FlinkDataWorks Summit
 
Unified Batch & Stream Processing with Apache Samza
Unified Batch & Stream Processing with Apache SamzaUnified Batch & Stream Processing with Apache Samza
Unified Batch & Stream Processing with Apache SamzaDataWorks Summit
 
[Hadoop Meetup] Tensorflow on Apache Hadoop YARN - Sunil Govindan
[Hadoop Meetup] Tensorflow on Apache Hadoop YARN - Sunil Govindan[Hadoop Meetup] Tensorflow on Apache Hadoop YARN - Sunil Govindan
[Hadoop Meetup] Tensorflow on Apache Hadoop YARN - Sunil GovindanNewton Alex
 
Hive, Presto, and Spark on TPC-DS benchmark
Hive, Presto, and Spark on TPC-DS benchmarkHive, Presto, and Spark on TPC-DS benchmark
Hive, Presto, and Spark on TPC-DS benchmarkDongwon Kim
 
TeraCache: Efficient Caching Over Fast Storage Devices
TeraCache: Efficient Caching Over Fast Storage DevicesTeraCache: Efficient Caching Over Fast Storage Devices
TeraCache: Efficient Caching Over Fast Storage DevicesDatabricks
 
Scaling HDFS to Manage Billions of Files with Key-Value Stores
Scaling HDFS to Manage Billions of Files with Key-Value StoresScaling HDFS to Manage Billions of Files with Key-Value Stores
Scaling HDFS to Manage Billions of Files with Key-Value StoresDataWorks Summit
 
Tez Shuffle Handler: Shuffling at Scale with Apache Hadoop
Tez Shuffle Handler: Shuffling at Scale with Apache HadoopTez Shuffle Handler: Shuffling at Scale with Apache Hadoop
Tez Shuffle Handler: Shuffling at Scale with Apache HadoopDataWorks Summit
 
Optimizing Hive Queries
Optimizing Hive QueriesOptimizing Hive Queries
Optimizing Hive QueriesOwen O'Malley
 

What's hot (20)

[Hadoop Meetup] Yarn at Microsoft - The challenges of scale
[Hadoop Meetup] Yarn at Microsoft - The challenges of scale[Hadoop Meetup] Yarn at Microsoft - The challenges of scale
[Hadoop Meetup] Yarn at Microsoft - The challenges of scale
 
Realtime olap architecture in apache kylin 3.0
Realtime olap architecture in apache kylin 3.0Realtime olap architecture in apache kylin 3.0
Realtime olap architecture in apache kylin 3.0
 
[Hadoop Meetup] Apache Hadoop 3 community update - Rohith Sharma
[Hadoop Meetup] Apache Hadoop 3 community update - Rohith Sharma[Hadoop Meetup] Apache Hadoop 3 community update - Rohith Sharma
[Hadoop Meetup] Apache Hadoop 3 community update - Rohith Sharma
 
Rds data lake @ Robinhood
Rds data lake @ Robinhood Rds data lake @ Robinhood
Rds data lake @ Robinhood
 
Building large scale transactional data lake using apache hudi
Building large scale transactional data lake using apache hudiBuilding large scale transactional data lake using apache hudi
Building large scale transactional data lake using apache hudi
 
Hadoop 3.0 - Revolution or evolution?
Hadoop 3.0 - Revolution or evolution?Hadoop 3.0 - Revolution or evolution?
Hadoop 3.0 - Revolution or evolution?
 
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
 
Hive join optimizations
Hive join optimizationsHive join optimizations
Hive join optimizations
 
Apache HBase 1.0 Release
Apache HBase 1.0 ReleaseApache HBase 1.0 Release
Apache HBase 1.0 Release
 
Achieving 100k Queries per Hour on Hive on Tez
Achieving 100k Queries per Hour on Hive on TezAchieving 100k Queries per Hour on Hive on Tez
Achieving 100k Queries per Hour on Hive on Tez
 
Millions of Regions in HBase: Size Matters
Millions of Regions in HBase: Size MattersMillions of Regions in HBase: Size Matters
Millions of Regions in HBase: Size Matters
 
Flexible and Real-Time Stream Processing with Apache Flink
Flexible and Real-Time Stream Processing with Apache FlinkFlexible and Real-Time Stream Processing with Apache Flink
Flexible and Real-Time Stream Processing with Apache Flink
 
Unified Batch & Stream Processing with Apache Samza
Unified Batch & Stream Processing with Apache SamzaUnified Batch & Stream Processing with Apache Samza
Unified Batch & Stream Processing with Apache Samza
 
[Hadoop Meetup] Tensorflow on Apache Hadoop YARN - Sunil Govindan
[Hadoop Meetup] Tensorflow on Apache Hadoop YARN - Sunil Govindan[Hadoop Meetup] Tensorflow on Apache Hadoop YARN - Sunil Govindan
[Hadoop Meetup] Tensorflow on Apache Hadoop YARN - Sunil Govindan
 
Hive, Presto, and Spark on TPC-DS benchmark
Hive, Presto, and Spark on TPC-DS benchmarkHive, Presto, and Spark on TPC-DS benchmark
Hive, Presto, and Spark on TPC-DS benchmark
 
TeraCache: Efficient Caching Over Fast Storage Devices
TeraCache: Efficient Caching Over Fast Storage DevicesTeraCache: Efficient Caching Over Fast Storage Devices
TeraCache: Efficient Caching Over Fast Storage Devices
 
Scaling HDFS to Manage Billions of Files with Key-Value Stores
Scaling HDFS to Manage Billions of Files with Key-Value StoresScaling HDFS to Manage Billions of Files with Key-Value Stores
Scaling HDFS to Manage Billions of Files with Key-Value Stores
 
Apache Spark
Apache SparkApache Spark
Apache Spark
 
Tez Shuffle Handler: Shuffling at Scale with Apache Hadoop
Tez Shuffle Handler: Shuffling at Scale with Apache HadoopTez Shuffle Handler: Shuffling at Scale with Apache Hadoop
Tez Shuffle Handler: Shuffling at Scale with Apache Hadoop
 
Optimizing Hive Queries
Optimizing Hive QueriesOptimizing Hive Queries
Optimizing Hive Queries
 

Similar to Skew Mitigation For Facebook PetabyteScale Joins

Adaptive Query Execution: Speeding Up Spark SQL at Runtime
Adaptive Query Execution: Speeding Up Spark SQL at RuntimeAdaptive Query Execution: Speeding Up Spark SQL at Runtime
Adaptive Query Execution: Speeding Up Spark SQL at RuntimeDatabricks
 
Performant Streaming in Production: Preventing Common Pitfalls when Productio...
Performant Streaming in Production: Preventing Common Pitfalls when Productio...Performant Streaming in Production: Preventing Common Pitfalls when Productio...
Performant Streaming in Production: Preventing Common Pitfalls when Productio...Databricks
 
Using Delta Lake to Transform a Legacy Apache Spark to Support Complex Update...
Using Delta Lake to Transform a Legacy Apache Spark to Support Complex Update...Using Delta Lake to Transform a Legacy Apache Spark to Support Complex Update...
Using Delta Lake to Transform a Legacy Apache Spark to Support Complex Update...Databricks
 
High Performance, High Reliability Data Loading on ClickHouse
High Performance, High Reliability Data Loading on ClickHouseHigh Performance, High Reliability Data Loading on ClickHouse
High Performance, High Reliability Data Loading on ClickHouseAltinity Ltd
 
What’s New in the Upcoming Apache Spark 3.0
What’s New in the Upcoming Apache Spark 3.0What’s New in the Upcoming Apache Spark 3.0
What’s New in the Upcoming Apache Spark 3.0Databricks
 
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
 
Troubleshooting SQL Server
Troubleshooting SQL ServerTroubleshooting SQL Server
Troubleshooting SQL ServerStephen Rose
 
Healthcare Claim Reimbursement using Apache Spark
Healthcare Claim Reimbursement using Apache SparkHealthcare Claim Reimbursement using Apache Spark
Healthcare Claim Reimbursement using Apache SparkDatabricks
 
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 ClustersDatabricks
 
Journey to Stability: Petabyte Ceph Cluster in OpenStack Cloud
Journey to Stability: Petabyte Ceph Cluster in OpenStack CloudJourney to Stability: Petabyte Ceph Cluster in OpenStack Cloud
Journey to Stability: Petabyte Ceph Cluster in OpenStack CloudCeph Community
 
Journey to Stability: Petabyte Ceph Cluster in OpenStack Cloud
Journey to Stability: Petabyte Ceph Cluster in OpenStack CloudJourney to Stability: Petabyte Ceph Cluster in OpenStack Cloud
Journey to Stability: Petabyte Ceph Cluster in OpenStack CloudPatrick McGarry
 
Apache Spark 3.0: Overview of What’s New and Why Care
Apache Spark 3.0: Overview of What’s New and Why CareApache Spark 3.0: Overview of What’s New and Why Care
Apache Spark 3.0: Overview of What’s New and Why CareDatabricks
 
Aurora Serverless, 서버리스 RDB의 서막 - 트랙2, Community Day 2018 re:Invent 특집
Aurora Serverless, 서버리스 RDB의 서막 - 트랙2, Community Day 2018 re:Invent 특집Aurora Serverless, 서버리스 RDB의 서막 - 트랙2, Community Day 2018 re:Invent 특집
Aurora Serverless, 서버리스 RDB의 서막 - 트랙2, Community Day 2018 re:Invent 특집AWSKRUG - AWS한국사용자모임
 
Scylla Summit 2018: Make Scylla Fast Again! Find out how using Tools, Talent,...
Scylla Summit 2018: Make Scylla Fast Again! Find out how using Tools, Talent,...Scylla Summit 2018: Make Scylla Fast Again! Find out how using Tools, Talent,...
Scylla Summit 2018: Make Scylla Fast Again! Find out how using Tools, Talent,...ScyllaDB
 
Optimising Geospatial Queries with Dynamic File Pruning
Optimising Geospatial Queries with Dynamic File PruningOptimising Geospatial Queries with Dynamic File Pruning
Optimising Geospatial Queries with Dynamic File PruningDatabricks
 
Kafka to the Maxka - (Kafka Performance Tuning)
Kafka to the Maxka - (Kafka Performance Tuning)Kafka to the Maxka - (Kafka Performance Tuning)
Kafka to the Maxka - (Kafka Performance Tuning)DataWorks Summit
 
Tame the small files problem and optimize data layout for streaming ingestion...
Tame the small files problem and optimize data layout for streaming ingestion...Tame the small files problem and optimize data layout for streaming ingestion...
Tame the small files problem and optimize data layout for streaming ingestion...Flink Forward
 
Spark shuffle introduction
Spark shuffle introductionSpark shuffle introduction
Spark shuffle introductioncolorant
 
Maaz Anjum - IOUG Collaborate 2013 - An Insight into Space Realization on ODA...
Maaz Anjum - IOUG Collaborate 2013 - An Insight into Space Realization on ODA...Maaz Anjum - IOUG Collaborate 2013 - An Insight into Space Realization on ODA...
Maaz Anjum - IOUG Collaborate 2013 - An Insight into Space Realization on ODA...Maaz Anjum
 

Similar to Skew Mitigation For Facebook PetabyteScale Joins (20)

Adaptive Query Execution: Speeding Up Spark SQL at Runtime
Adaptive Query Execution: Speeding Up Spark SQL at RuntimeAdaptive Query Execution: Speeding Up Spark SQL at Runtime
Adaptive Query Execution: Speeding Up Spark SQL at Runtime
 
Performant Streaming in Production: Preventing Common Pitfalls when Productio...
Performant Streaming in Production: Preventing Common Pitfalls when Productio...Performant Streaming in Production: Preventing Common Pitfalls when Productio...
Performant Streaming in Production: Preventing Common Pitfalls when Productio...
 
Using Delta Lake to Transform a Legacy Apache Spark to Support Complex Update...
Using Delta Lake to Transform a Legacy Apache Spark to Support Complex Update...Using Delta Lake to Transform a Legacy Apache Spark to Support Complex Update...
Using Delta Lake to Transform a Legacy Apache Spark to Support Complex Update...
 
High Performance, High Reliability Data Loading on ClickHouse
High Performance, High Reliability Data Loading on ClickHouseHigh Performance, High Reliability Data Loading on ClickHouse
High Performance, High Reliability Data Loading on ClickHouse
 
What’s New in the Upcoming Apache Spark 3.0
What’s New in the Upcoming Apache Spark 3.0What’s New in the Upcoming Apache Spark 3.0
What’s New in the Upcoming Apache Spark 3.0
 
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
 
Troubleshooting SQL Server
Troubleshooting SQL ServerTroubleshooting SQL Server
Troubleshooting SQL Server
 
Healthcare Claim Reimbursement using Apache Spark
Healthcare Claim Reimbursement using Apache SparkHealthcare Claim Reimbursement using Apache Spark
Healthcare Claim Reimbursement using Apache Spark
 
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
 
Journey to Stability: Petabyte Ceph Cluster in OpenStack Cloud
Journey to Stability: Petabyte Ceph Cluster in OpenStack CloudJourney to Stability: Petabyte Ceph Cluster in OpenStack Cloud
Journey to Stability: Petabyte Ceph Cluster in OpenStack Cloud
 
Journey to Stability: Petabyte Ceph Cluster in OpenStack Cloud
Journey to Stability: Petabyte Ceph Cluster in OpenStack CloudJourney to Stability: Petabyte Ceph Cluster in OpenStack Cloud
Journey to Stability: Petabyte Ceph Cluster in OpenStack Cloud
 
Apache Spark 3.0: Overview of What’s New and Why Care
Apache Spark 3.0: Overview of What’s New and Why CareApache Spark 3.0: Overview of What’s New and Why Care
Apache Spark 3.0: Overview of What’s New and Why Care
 
Hive: Loading Data
Hive: Loading DataHive: Loading Data
Hive: Loading Data
 
Aurora Serverless, 서버리스 RDB의 서막 - 트랙2, Community Day 2018 re:Invent 특집
Aurora Serverless, 서버리스 RDB의 서막 - 트랙2, Community Day 2018 re:Invent 특집Aurora Serverless, 서버리스 RDB의 서막 - 트랙2, Community Day 2018 re:Invent 특집
Aurora Serverless, 서버리스 RDB의 서막 - 트랙2, Community Day 2018 re:Invent 특집
 
Scylla Summit 2018: Make Scylla Fast Again! Find out how using Tools, Talent,...
Scylla Summit 2018: Make Scylla Fast Again! Find out how using Tools, Talent,...Scylla Summit 2018: Make Scylla Fast Again! Find out how using Tools, Talent,...
Scylla Summit 2018: Make Scylla Fast Again! Find out how using Tools, Talent,...
 
Optimising Geospatial Queries with Dynamic File Pruning
Optimising Geospatial Queries with Dynamic File PruningOptimising Geospatial Queries with Dynamic File Pruning
Optimising Geospatial Queries with Dynamic File Pruning
 
Kafka to the Maxka - (Kafka Performance Tuning)
Kafka to the Maxka - (Kafka Performance Tuning)Kafka to the Maxka - (Kafka Performance Tuning)
Kafka to the Maxka - (Kafka Performance Tuning)
 
Tame the small files problem and optimize data layout for streaming ingestion...
Tame the small files problem and optimize data layout for streaming ingestion...Tame the small files problem and optimize data layout for streaming ingestion...
Tame the small files problem and optimize data layout for streaming ingestion...
 
Spark shuffle introduction
Spark shuffle introductionSpark shuffle introduction
Spark shuffle introduction
 
Maaz Anjum - IOUG Collaborate 2013 - An Insight into Space Realization on ODA...
Maaz Anjum - IOUG Collaborate 2013 - An Insight into Space Realization on ODA...Maaz Anjum - IOUG Collaborate 2013 - An Insight into Space Realization on ODA...
Maaz Anjum - IOUG Collaborate 2013 - An Insight into Space Realization on ODA...
 

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

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

why-transparency-and-traceability-are-essential-for-sustainable-supply-chains...
why-transparency-and-traceability-are-essential-for-sustainable-supply-chains...why-transparency-and-traceability-are-essential-for-sustainable-supply-chains...
why-transparency-and-traceability-are-essential-for-sustainable-supply-chains...Jack Cole
 
modul pembelajaran robotic Workshop _ by Slidesgo.pptx
modul pembelajaran robotic Workshop _ by Slidesgo.pptxmodul pembelajaran robotic Workshop _ by Slidesgo.pptx
modul pembelajaran robotic Workshop _ by Slidesgo.pptxaleedritatuxx
 
Networking Case Study prepared by teacher.pptx
Networking Case Study prepared by teacher.pptxNetworking Case Study prepared by teacher.pptx
Networking Case Study prepared by teacher.pptxHimangsuNath
 
Minimizing AI Hallucinations/Confabulations and the Path towards AGI with Exa...
Minimizing AI Hallucinations/Confabulations and the Path towards AGI with Exa...Minimizing AI Hallucinations/Confabulations and the Path towards AGI with Exa...
Minimizing AI Hallucinations/Confabulations and the Path towards AGI with Exa...Thomas Poetter
 
English-8-Q4-W3-Synthesizing-Essential-Information-From-Various-Sources-1.pdf
English-8-Q4-W3-Synthesizing-Essential-Information-From-Various-Sources-1.pdfEnglish-8-Q4-W3-Synthesizing-Essential-Information-From-Various-Sources-1.pdf
English-8-Q4-W3-Synthesizing-Essential-Information-From-Various-Sources-1.pdfblazblazml
 
Learn How Data Science Changes Our World
Learn How Data Science Changes Our WorldLearn How Data Science Changes Our World
Learn How Data Science Changes Our WorldEduminds Learning
 
NO1 Certified Black Magic Specialist Expert Amil baba in Lahore Islamabad Raw...
NO1 Certified Black Magic Specialist Expert Amil baba in Lahore Islamabad Raw...NO1 Certified Black Magic Specialist Expert Amil baba in Lahore Islamabad Raw...
NO1 Certified Black Magic Specialist Expert Amil baba in Lahore Islamabad Raw...Amil Baba Dawood bangali
 
Rithik Kumar Singh codealpha pythohn.pdf
Rithik Kumar Singh codealpha pythohn.pdfRithik Kumar Singh codealpha pythohn.pdf
Rithik Kumar Singh codealpha pythohn.pdfrahulyadav957181
 
Real-Time AI Streaming - AI Max Princeton
Real-Time AI  Streaming - AI Max PrincetonReal-Time AI  Streaming - AI Max Princeton
Real-Time AI Streaming - AI Max PrincetonTimothy Spann
 
FAIR, FAIRsharing, FAIR Cookbook and ELIXIR - Sansone SA - Boston 2024
FAIR, FAIRsharing, FAIR Cookbook and ELIXIR - Sansone SA - Boston 2024FAIR, FAIRsharing, FAIR Cookbook and ELIXIR - Sansone SA - Boston 2024
FAIR, FAIRsharing, FAIR Cookbook and ELIXIR - Sansone SA - Boston 2024Susanna-Assunta Sansone
 
Bank Loan Approval Analysis: A Comprehensive Data Analysis Project
Bank Loan Approval Analysis: A Comprehensive Data Analysis ProjectBank Loan Approval Analysis: A Comprehensive Data Analysis Project
Bank Loan Approval Analysis: A Comprehensive Data Analysis ProjectBoston Institute of Analytics
 
Data Analysis Project Presentation: Unveiling Your Ideal Customer, Bank Custo...
Data Analysis Project Presentation: Unveiling Your Ideal Customer, Bank Custo...Data Analysis Project Presentation: Unveiling Your Ideal Customer, Bank Custo...
Data Analysis Project Presentation: Unveiling Your Ideal Customer, Bank Custo...Boston Institute of Analytics
 
Decoding the Heart: Student Presentation on Heart Attack Prediction with Data...
Decoding the Heart: Student Presentation on Heart Attack Prediction with Data...Decoding the Heart: Student Presentation on Heart Attack Prediction with Data...
Decoding the Heart: Student Presentation on Heart Attack Prediction with Data...Boston Institute of Analytics
 
Principles and Practices of Data Visualization
Principles and Practices of Data VisualizationPrinciples and Practices of Data Visualization
Principles and Practices of Data VisualizationKianJazayeri1
 
What To Do For World Nature Conservation Day by Slidesgo.pptx
What To Do For World Nature Conservation Day by Slidesgo.pptxWhat To Do For World Nature Conservation Day by Slidesgo.pptx
What To Do For World Nature Conservation Day by Slidesgo.pptxSimranPal17
 
6 Tips for Interpretable Topic Models _ by Nicha Ruchirawat _ Towards Data Sc...
6 Tips for Interpretable Topic Models _ by Nicha Ruchirawat _ Towards Data Sc...6 Tips for Interpretable Topic Models _ by Nicha Ruchirawat _ Towards Data Sc...
6 Tips for Interpretable Topic Models _ by Nicha Ruchirawat _ Towards Data Sc...Dr Arash Najmaei ( Phd., MBA, BSc)
 
Data Analysis Project : Targeting the Right Customers, Presentation on Bank M...
Data Analysis Project : Targeting the Right Customers, Presentation on Bank M...Data Analysis Project : Targeting the Right Customers, Presentation on Bank M...
Data Analysis Project : Targeting the Right Customers, Presentation on Bank M...Boston Institute of Analytics
 
The Power of Data-Driven Storytelling_ Unveiling the Layers of Insight.pptx
The Power of Data-Driven Storytelling_ Unveiling the Layers of Insight.pptxThe Power of Data-Driven Storytelling_ Unveiling the Layers of Insight.pptx
The Power of Data-Driven Storytelling_ Unveiling the Layers of Insight.pptxTasha Penwell
 
Decoding Movie Sentiments: Analyzing Reviews with Data Analysis model
Decoding Movie Sentiments: Analyzing Reviews with Data Analysis modelDecoding Movie Sentiments: Analyzing Reviews with Data Analysis model
Decoding Movie Sentiments: Analyzing Reviews with Data Analysis modelBoston Institute of Analytics
 

Recently uploaded (20)

why-transparency-and-traceability-are-essential-for-sustainable-supply-chains...
why-transparency-and-traceability-are-essential-for-sustainable-supply-chains...why-transparency-and-traceability-are-essential-for-sustainable-supply-chains...
why-transparency-and-traceability-are-essential-for-sustainable-supply-chains...
 
modul pembelajaran robotic Workshop _ by Slidesgo.pptx
modul pembelajaran robotic Workshop _ by Slidesgo.pptxmodul pembelajaran robotic Workshop _ by Slidesgo.pptx
modul pembelajaran robotic Workshop _ by Slidesgo.pptx
 
Networking Case Study prepared by teacher.pptx
Networking Case Study prepared by teacher.pptxNetworking Case Study prepared by teacher.pptx
Networking Case Study prepared by teacher.pptx
 
Minimizing AI Hallucinations/Confabulations and the Path towards AGI with Exa...
Minimizing AI Hallucinations/Confabulations and the Path towards AGI with Exa...Minimizing AI Hallucinations/Confabulations and the Path towards AGI with Exa...
Minimizing AI Hallucinations/Confabulations and the Path towards AGI with Exa...
 
English-8-Q4-W3-Synthesizing-Essential-Information-From-Various-Sources-1.pdf
English-8-Q4-W3-Synthesizing-Essential-Information-From-Various-Sources-1.pdfEnglish-8-Q4-W3-Synthesizing-Essential-Information-From-Various-Sources-1.pdf
English-8-Q4-W3-Synthesizing-Essential-Information-From-Various-Sources-1.pdf
 
Learn How Data Science Changes Our World
Learn How Data Science Changes Our WorldLearn How Data Science Changes Our World
Learn How Data Science Changes Our World
 
NO1 Certified Black Magic Specialist Expert Amil baba in Lahore Islamabad Raw...
NO1 Certified Black Magic Specialist Expert Amil baba in Lahore Islamabad Raw...NO1 Certified Black Magic Specialist Expert Amil baba in Lahore Islamabad Raw...
NO1 Certified Black Magic Specialist Expert Amil baba in Lahore Islamabad Raw...
 
Rithik Kumar Singh codealpha pythohn.pdf
Rithik Kumar Singh codealpha pythohn.pdfRithik Kumar Singh codealpha pythohn.pdf
Rithik Kumar Singh codealpha pythohn.pdf
 
Real-Time AI Streaming - AI Max Princeton
Real-Time AI  Streaming - AI Max PrincetonReal-Time AI  Streaming - AI Max Princeton
Real-Time AI Streaming - AI Max Princeton
 
FAIR, FAIRsharing, FAIR Cookbook and ELIXIR - Sansone SA - Boston 2024
FAIR, FAIRsharing, FAIR Cookbook and ELIXIR - Sansone SA - Boston 2024FAIR, FAIRsharing, FAIR Cookbook and ELIXIR - Sansone SA - Boston 2024
FAIR, FAIRsharing, FAIR Cookbook and ELIXIR - Sansone SA - Boston 2024
 
Bank Loan Approval Analysis: A Comprehensive Data Analysis Project
Bank Loan Approval Analysis: A Comprehensive Data Analysis ProjectBank Loan Approval Analysis: A Comprehensive Data Analysis Project
Bank Loan Approval Analysis: A Comprehensive Data Analysis Project
 
Data Analysis Project: Stroke Prediction
Data Analysis Project: Stroke PredictionData Analysis Project: Stroke Prediction
Data Analysis Project: Stroke Prediction
 
Data Analysis Project Presentation: Unveiling Your Ideal Customer, Bank Custo...
Data Analysis Project Presentation: Unveiling Your Ideal Customer, Bank Custo...Data Analysis Project Presentation: Unveiling Your Ideal Customer, Bank Custo...
Data Analysis Project Presentation: Unveiling Your Ideal Customer, Bank Custo...
 
Decoding the Heart: Student Presentation on Heart Attack Prediction with Data...
Decoding the Heart: Student Presentation on Heart Attack Prediction with Data...Decoding the Heart: Student Presentation on Heart Attack Prediction with Data...
Decoding the Heart: Student Presentation on Heart Attack Prediction with Data...
 
Principles and Practices of Data Visualization
Principles and Practices of Data VisualizationPrinciples and Practices of Data Visualization
Principles and Practices of Data Visualization
 
What To Do For World Nature Conservation Day by Slidesgo.pptx
What To Do For World Nature Conservation Day by Slidesgo.pptxWhat To Do For World Nature Conservation Day by Slidesgo.pptx
What To Do For World Nature Conservation Day by Slidesgo.pptx
 
6 Tips for Interpretable Topic Models _ by Nicha Ruchirawat _ Towards Data Sc...
6 Tips for Interpretable Topic Models _ by Nicha Ruchirawat _ Towards Data Sc...6 Tips for Interpretable Topic Models _ by Nicha Ruchirawat _ Towards Data Sc...
6 Tips for Interpretable Topic Models _ by Nicha Ruchirawat _ Towards Data Sc...
 
Data Analysis Project : Targeting the Right Customers, Presentation on Bank M...
Data Analysis Project : Targeting the Right Customers, Presentation on Bank M...Data Analysis Project : Targeting the Right Customers, Presentation on Bank M...
Data Analysis Project : Targeting the Right Customers, Presentation on Bank M...
 
The Power of Data-Driven Storytelling_ Unveiling the Layers of Insight.pptx
The Power of Data-Driven Storytelling_ Unveiling the Layers of Insight.pptxThe Power of Data-Driven Storytelling_ Unveiling the Layers of Insight.pptx
The Power of Data-Driven Storytelling_ Unveiling the Layers of Insight.pptx
 
Decoding Movie Sentiments: Analyzing Reviews with Data Analysis model
Decoding Movie Sentiments: Analyzing Reviews with Data Analysis modelDecoding Movie Sentiments: Analyzing Reviews with Data Analysis model
Decoding Movie Sentiments: Analyzing Reviews with Data Analysis model
 

Skew Mitigation For Facebook PetabyteScale Joins

  • 1. Skew Mitigation For Facebook’s Petabyte-Scale Joins Suganthi Dewakar & Guanzhong (Tony) Xu
  • 2. Agenda Skew Join Journey ▪ Skew Hint ▪ Runtime Skew Mitigation ▪ Customized AQE Skew Mitigation Cosco + Skew Join ▪ Shuffle Recap ▪ Working with Original Cosco ▪ Splitting in File Boundaries
  • 3. What Is Data Skew ▪ Uneven distribution of partitioned data ▪ Affects aggregate operations e.g Group By and Join ▪ E.g. count number of people group by country ▪ Data skew at FB ▪ PB sized table Joins with TB sized skewed partitions ▪ Task latency ▪ Non-skewed partition: mins to few hours ▪ Skewed partition: hours to days ▪ Skewed pipelines ▪ Latency sensitive daily jobs ▪ Complex DAG with several upstream/downstream dependencies ▪ Can cause delay in hundreds of downsteam pipelines 0.01GB 1TB Percentile shuffle partition 1 partition 2 partition 3 (skewed) partition 4
  • 4. Data Skew In Join ▪ Join strategies in Spark ▪ SortMergeJoin/ ShuffleHashJoin ▪ Not skew resistant – requires input data to be partitioned ▪ BroadcastHashJoin ▪ Skew resistant – requires no data partitioning ▪ Requires one of side of join to be small ▪ Skew Join ▪ Hybrid join that combines above strategies
  • 5. Mitigating Data Skew In Joins Skew Hint Runtime skew mitigation Customized AQE skew mitigation 1 2 3 Optimizer rule-based solution Built on Spark 2.0 adaptive framework Based on Spark 3.0 AQE framework
  • 6. Mitigating Data Skew In Joins Skew Hint Runtime skew mitigation Customized AQE skew mitigation 1 2 3 Optimizer rule-based solution Built on Spark 2.0 adaptive framework Based on Spark 3.0 AQE framework
  • 7. Skew Hint ▪ User provides skew information as hints ▪ /*+ SKEWED_ON(column=value1,..) */ ▪ Split input into two parts ▪ Skewed keys ▪ Non-skewed keys ▪ Broadcast join skewed keys ▪ Shuffle/Sort join non-skewed keys ▪ Union both join outputs
  • 8. Skew Hint Scan table_A Scan table_B SortMergeJoin Exchange Exchange
  • 9. Skew Hint Union Scan table_A Scan table_B Filter [skewed keys] BroadcastHashJoin Filter [skewed keys] Scan table_A Scan table_B SortMergeJoin Filter [non-skewed keys] Filter [non-skewed keys] Exchange Exchange Skewed dataNon-skewed data
  • 10. Skew Hint Skew Hint Pros • Reduces runtime latency Cons • Requires prior knowledge of skewed keys • Double scan of data
  • 11. Mitigating Data Skew In Joins Skew Hint Runtime skew mitigation Customized AQE skew mitigation 1 2 3 Optimizer rule-based solution Built on Spark 2.0 adaptive framework Based on Spark 3.0 AQE framework
  • 12. Runtime skew mitigation § Spark 2.0 adaptive framework § Adds ExchangeCoordinator to all shuffle Exchange nodes § ExchangeCoordinator merges small partitions based on runtime stats § Runtime skewed partition detection § Collect MapOutputStatistics of a joins input stages § Partition is skewed iff size of partition is: § > min_threshold_config_value (e.g. 1GB) § > median_ratio X median_size(shuffle) (e.g. 10 times median size) § > pct_99_ratio X pct99_size(shuffle) (e.g. 3 times 99 percentile size) Built on Spark 2.0 adaptive framework
  • 13. Runtime skew mitigation § Split large partitions into smaller sub-partitions shuffle table_A shuffle table_B skewed partition split Built on Spark 2.0 adaptive framework table_A join table_B
  • 14. Built on Spark 2.0 adaptive framework Skew Hint Runtime Skew Mitigation Spark 2.0 adaptive framework Pros • Reduces runtime latency Cons • Requires prior knowledge of skewed keys • Double scan of data Pros • Reduces runtime latency • No prior skew key knowledge required • No double scan of data Cons Additional shuffles even when a join is not skewed • Between joins • Between joins and aggregates Runtime skew mitigation
  • 15. Mitigating Data Skew In Joins Skew Hint Runtime skew mitigation Customized AQE skew mitigation 1 2 3 Optimizer rule-based solution Built on Spark 2.0 adaptive framework Based on Spark 3.0 AQE framework
  • 16. AQE skew mitigation ▪ AQE recap ▪ New adaptive framework in OSS ▪ Features ▪ Switching join strategy ▪ Small shuffle partitions coalescing ▪ Optimizing skew join ▪ Runtime DAG change Spark 3.0 adaptive framework
  • 17. AQE skew mitigation Limitation ▪ Supports two table join only ▪ Single stage skewed join + reducer operation ▪ Skewed Join + Aggregate ▪ Skewed multi table joins ▪ Single stage skewed join + mapper operation ▪ Skewed join + union ▪ Skewed join + broadcast join
  • 18. Customized AQE skew mitigation Customization 1: Runtime Shuffle Insertion Lifecycle of AQE execution EnsureRequirements Execute Stats from previous stage No new Exchange beyond this point Optimize skew Create sub-stages Re-plan
  • 19. Customized AQE skew mitigation Customization 1: Runtime Shuffle Insertion Lifecycle of AQE execution EnsureRequirements Execute Stats from previous stage No new Exchange beyond this point Optimize skew Create sub-stages Re-plan If skewed, set outputPartitioning of join to Unknown Detect skew
  • 20. table_A (petabye) table_B Left Outer Join table_C Left Outer Join Customized AQE skew mitigation [Limitation] Customization 1: Runtime Shuffle Insertion table_D Left Outer Join
  • 21. table_A (petabye) table_B Left Outer Join table_C Left Outer Join Customized AQE skew mitigation [Limitation] Customization 1: Runtime Shuffle Insertion table_D Left Outer Join Exchange Exchange Shuffles PB sized data Shuffles PB sized data
  • 22. Customized AQE skew mitigation Customization 2: No Shuffle Multi Table Join shuffle table_A shuffle table_B skewed partition split shuffle table_C ▪ Split skewed partition in one table ▪ Replicate the corresponding partitions in the rest of the input tables shuffle table_Dshuffle table_A
  • 23. Skew Hint Runtime Skew Mitigation Spark 2.0 adaptive framework Customized AQE Skew Mitigation Spark 3.0 adaptive framework Pros • Reduces runtime latency Cons • Requires prior knowledge of skewed keys • Double scan of data Pros • Reduces runtime latency • No prior skew key knowledge required • No double scan of data Cons Additional shuffles even when a join is not skewed • Between joins • Between joins and aggregates Pros • Reduces runtime latency • No prior skew key knowledge required • No double scan of data • No unnecessary shuffles when a join is not skewed Customized AQE skew mitigation Customization 2: No Shuffle Multi Table Join
  • 24. Skew Join With Cosco
  • 25. ▪ Adaptive Skew Mitigation Splits in Mapper Boundaries ▪ Each sub-reducer is only required to read data from a subset of mappers. ▪ Only partial data is required ▪ E.g.: a skewed partition 0 is split that: sub-reducer 0 reads {mapper 1}, sub-reducer 1 reads {mapper 0, mapper 2}, sub-reducer 1 reads {mapper 0, mapper 2} Partition 0 The Problem File 0 File 1 File 3 Sub-reducer 0 Sub-reducer 1 Sub-reducer 2 Sub-reducer i… Sub-reducer i Mapper outputs are merged in files :(
  • 26. Cosco Shuffle Recap ▪ Why Cosco? ▪ Shuffle as a service: IO Efficiency ▪ Write-ahead buffer ▪ Groups shuffled output based on partition id instead of mapper id. ▪ Data deduplication is performed in reducer side ▪ Efficiency Wins: ▪ Solve the write amplification problem: avoid spills: 3X -> 1X ▪ Solve the small IO problem: average IO size: 200KB -> 2.5MB ▪ Previous Talk ▪ Cosco: An Efficient Facebook-Scale Shuffle Service
  • 27. DFS Cosco Shuffle ServiceCosco Shuffle Service Cosco Shuffle Recap Cosco Shuffle Service Partition 0 (file 2 buffer) Partition 0 (file 1 buffer) Mapper 0 Mapper 0 Mapper 0 Reducer 0 Reducer 1 File 1 File 0 File 2 File 0 File 1 Cosco Metadata Service
  • 28. Working with Mapper Boundaries ▪ Load all data but filter out uninterested ones ▪ Each sub-reducer still has to load all data from all file chunks of the partition. ▪ Filtering out records from uninterested mappers. ▪ IO inefficient Record (mapper-1) Record (mapper-3) Record (mapper-2) Record (mapper-2) Record (mapper-4) Record (mapper-1) … File 0 Read Drop Drop Drop Read Read Sub-reducer for mapper {1, 4}
  • 29. Skewed partition worst cases ▪ For the most heavily skewed partitions, it could be up to XXX TB and requires XX k splits, which means the skewed partition data has to be read XX k times as well. ▪ Very inefficient, most times are spent on disk IO. ▪ Compared to vanilla Spark shuffle, it no longer has any advantages in terms of IO.
  • 30. Why cannot we split in file boundaries?
  • 31. Terminology ▪ Record ▪ Unit of a data row ▪ Package ▪ Unit to be sent from mapper to shuffle service ▪ Chunk ▪ Unit to be flushed Record Record Record Record Package Package Package Package Chunk Multiple records to form a package Multiple packages to form a chunk
  • 32. ▪ Duplicated records ▪ A record could be duplicated in multiple files due to package resending which might be caused by various reasons. ▪ Cosco relies on reducer client to perform deduplication which requires it to read the entire partition data from a mapper. ▪ Simply splitting on file boundaries requires no overlap of mappers between file sets for different sub-reducers, which is not feasible. File 2 File 3 Restrictions Sub-reducer 0 File 0 File 1 Sub-reducer 0 is assigned to read file 0 & 1. But it has to read file 2 & 3 as well because they have mapper overlap with file 0 & 1.
  • 33. Duplication ▪ Mapper failure ▪ Each mapper is assigned a unique id. ▪ Each package is attached with its mapper’s unique id. ▪ Reducer consumes records from interested mappers based on those unique ids. ▪ It would not cause duplicated records in reducer if a mapper output is not fully read by a reducer. Mapper 0 (id: abc) Mapper 0 (id: xyz) Partition Package Id MapperId ... 0 3 abc 0 3 xyz ... Failed Sub-reducer 0 (read from `xyz`) Dropped Accepted
  • 34. Duplication ▪ Network issues ▪ A mapper fails to receive ack message after timeout, it will resend the package. ▪ We call such packages "suspected packages”: a suspected package is not necessary to be a duplicated packaged, but a duplicated package is always a suspected package. Shuffle service - 1 Shuffle service - 2Package-1 Package-2 Package-3 Mapper - 1 Ack Queue Chunk - i Records from package- {1,2,3} are flushed to chunks Chunk - j Records from package- {1,2,3} are duplicated to chunks Resend packages in ack queues
  • 36. Suspected Map ▪ Map<partition, Map<packageId, chunkId>> ▪ Each mapper keeps a map to track the resent packages due to connection. ▪ Whenever a resend happens, all the packages in the ACK queue are added to the map. Shuffle service - 1 Shuffle service - 2 Package-1 Pakcage-2 Package-3 Mapper - 1 ACK Queue Resend packages in ack queues Add to suspected package map packageId chunkId Package-x Chunk-xyz Package-1 null Package-2 null Package-3 null Partition - 1
  • 37. Suspected Map ▪ Map<partition, Map<packageId, chunkId>> ▪ An ACK message eventually returned contains a unique identifier of the chunk containing the package. ▪ Mapper keeps the mapping from those suspected packages to their authorized chunks. ▪ Once the mapper is done, reports the suspected map to Spark driver. ▪ Spark driver aggregate the suspected package info from different mappers. Shuffle service - 2 Pakcage-2 Package-3 Mapper - 1 ACK Queue Return ACK for Package-1 with chunkId 1. Remove package-1 from ACK queue. 2. Update the chunkId in suspected map packageId chunkId Package-x Chunk-xyz Package-1 Chunk-abc Package-2 null Package-3 null Partition - 1
  • 38. ▪ Each sub-reducer would be assigned a subset of files along with the corresponding suspected map of the partition. Reading subset of files on sub-reducer Chunk Set - 1 Chunk Set - 2 ... Skewed Partition Sub-reducer - 1 Sub-reducer - 2 ...
  • 39. ▪ Duplicated records would only be accepted if it is read from authorized chunk. Reading subset of files on sub-reducer Record {mapper-3,package-4,chunk-15} Chunk-15 Record {mapper-3,package-4,chunk-15} Chunk-4 Sub-reducer-2 Sub-reducer-7 Record Accepted Record Dropped {MapperId, PackageId} ChunkId ... ... {mapper-3, package-4} chunk-15 ... ...
  • 40. Chunk Lost ▪ Chunk lost failure requires restart of all sub-reducers accordingly: . ▪ Spark driver to restart mappers to regenerate data of that particular partition. ▪ Data in new chunks are non-deterministic. ▪ Spark driver broadcasts failures to all corresponding sub-reducer tasks and restarts them all. ▪ This is rare since chunks are RS-encoded
  • 41. Putting It All Together ▪ Motivation: ▪ To split a skewed partition into multiple non-overlapping file sets that each sub-reducer would be assigned one of them. ▪ Mapper: ▪ Keeps tracking of suspected packages along with authorized chunks. ▪ Report suspected package info to Spark driver when finishes. ▪ Driver: ▪ Aggregate suspect package info from all mappers and pass along to sub-reducers if necessary. ▪ Detects skewed partitions and split them based on file boundaries of each partition. ▪ Fail and restart all corresponding sub-reducer tasks of a partition if there is a chunk lost. ▪ Reducer: ▪ Fetches file list from driver along with suspected package info. ▪ Accept a record of a suspected package only if it is read from the authorized chunk. ▪ Data Correctness ▪ End-to-end checksum
  • 42. Summary Skew Join Journey ▪ Skew Hint ▪ Runtime Skew Mitigation ▪ Customized AQE Skew Mitigation Cosco + Skew Join ▪ Shuffle Recap ▪ Working with Original Cosco ▪ Splitting in File Boundaries
  • 43. Feedback Your feedback is important to us. Don’t forget to rate and review the sessions.