SlideShare a Scribd company logo
1 of 44
Download to read offline
WIFI SSID:Spark+AISummit | Password: UnifiedDataAnalytics
Christopher Crosbie, Google
Ben Sidhom, Google
Improving Spark
Downscaling
#UnifiedDataAnalytics #SparkAISummit
Open
Source
Google
Cloud Products
Google
Research
2000 2010
GFS
Map
Reduce
Dremel
Flume
Java MillwheelPubSub
2020
BigTable
BigQuery Pub/Sub Dataflow Bigtable MLDataproc
Long History of Solving Data
Problems
Tensorflow
Apache Airflow
Cloud ML Engine
Cloud Dataflow
Cloud Data Fusion
Cloud Composer
Who are we and what is Cloud Dataproc?
Google Cloud Platform’s
fully-managed Apache Spark
and Apache Hadoop service
Rapid cluster creation
Familiar open source tools
Customizable machines
Ephemeral clusters on-demand
Tightly Integrated
with other Google Cloud
Platform services
Cloud Dataproc: Open source solutions with GCP
Taking the best of open source And opening up access to the best of GCP
Webhcat
BigQuery
Cloud
Datastore
Cloud
Bigtable
Compute
Engine
Kubernetes
Engine
Cloud
Dataflow
Cloud
Dataproc
Cloud
Functions
Cloud Machine
Learning
Engine
Cloud
Pub/Sub
Key
Management
Service
Cloud
Spanner
Cloud SQL BQ Transfer
Service
Cloud
Translation API
Cloud Vision
API
Cloud
Storage
Jobs are “fire and forget”
No need to manually intervene
when a cluster is over or under
capacity
Choose balance between
standard and preemptible workers
Save resources (quota & cost) at
any point in time
Dataproc Autoscaling GA
Complicating Spark Downscaling
Without autoscaling
Submit job
Monitor resource usage
Adjust cluster size
With autoscaling
Submit jobs
Based on the difference between
YARN pending and available
memory
If more memory is needed then
scale up
If there is excess memory then
scale down
Obey VM limits and scale based
on scale factor
Autoscaling policies: fine grained control
Is there too much or too little
YARN memory?
Do nothing
Is the cluster at the maximum
# of nodes?
Do not autoscale
Determine type and scale of
nodes to modify
Autoscale cluster
Yes No
Yes No
YARN Infrastructure
Complexities
Finding processed data
(shuffle files, cached RDDs, etc)
Optimizing costs
Spark Autoscaling Challenges
YARN
YARN-based managed Spark
Dataproc Cluster
HDFS
Persistent Disk
Cluster bucket
Cloud Storage Compute engine nodes
Dataproc Image
Apache Spark
Apache Hadoop
Apache Hive
...
Clients
Cloud Dataproc API
Clusters
...
Jobs
Clients
(SSH)
Dataproc Agent
User Data
Cloud Storage
YARN pain points
Management is difficult
Clusters are complicated and have to use more components than are
required for a job or model. This also requires hard-to-find experts.
Complicated OSS software stack
Version and dependency management is hard. Have to understand how to
tune multiple components for efficiency.
Isolation is hard
I have to think about my jobs to size clusters, and isolating jobs requires
additional steps.
Multiple k8s
options
Moving the OSS ecosystem
to Kubernetes offers
customers a range of options
depending on their needs and
core expertise.
DIY k8s Dataproc
k8s Dataproc +
Vendor components
Runs OSS on k8s? Yes - self-managed
Yes - managed k8s
clusters
Yes - managed k8s
clusters
SLAs GKE only Dataproc cluster
Dataproc cluster
and component
OSS components Community only Google optimized
Google optimized +
vendor optimized
In-depth component
support
No No Yes
Integrated
management
No Yes Yes
Integrated security No Yes Yes
Hybrid/cross-cloud
support
No Yes Yes
How we are making this happen
• Kubernetes Operators - Application control
plane for complex applications
– The language of Kubernetes allows
extending its vocabulary through
Custom Resource Definition (CRD)
– Kubernetes Operator is an app-specific
control plane running in the cluster
• CRD: app-specific vocabulary
• CR: instance of CRD
• CR Controller: interpreter and
reconciliation loop for CRs
– The cluster can now speak the
app-specific words through the
Kubernetes API
Control Plane
(Master)
MyApp API
Data Plane
(Nodes)
CRUD MyApp ...
Kubernetes
MyApp Control Plane
Kubernetes API
● Integrates with BigQuery,
Google’s Serverless Data
Warehouse
● Provides Google Cloud Storage
as replacement for HDFS
● Ships logs to Stackdriver
Monitoring
○ via Prometheus server
with the Stackdriver
sidecar
● Contains sparkctl, a command
line tool that simplifies client-local
application dependencies in a
Kubernetes environment.
https://github.com/GoogleCloudPlatform/spark-on-k8s-operator
Deployment options
1. Deploy unified resource management
Get away deal from two separate cluster management interfaces to manage
open source component. Offers one central view for easy management.
2. Isolate Spark jobs and resources
Remove the headaches of version and dependency management; instead,
move models and ETL pipelines from dev to production without added work.
Build resilient infrastructure
Don’t worry about sizing and building clusters, manipulating Docker files, or
messing around with Kubernetes networking configurations. It just works.
Key benefits for autoscaling
Helpful but does not solve our
core problem…..
Finding the
processed data
What exactly is a shuffle & why do we care? Rob Wynne
A Brief History of Spark Shuffle
● Shuffle files to local storage on the executors
● Executors responsible for serving the files
● Loss of an executor meant loss of the shuffle files
● Result: poor auto-scaling
○ Pathological loop: scale down, lose work, re-compute, trigger scale up…
● Depended on driver GC event to clean up shuffle files
22#UnifiedDataAnalytics #SparkAISummit
Today: Dynamic allocation and “external” shuffle
● Executors no longer need to serve data
● “External” shuffle is not exactly external
○ Only executors can be released
○ Can scale up & down executors but not the machines
● Still depends on driver GC event to clean up shuffle files
23#UnifiedDataAnalytics #SparkAISummit
Spark’s shuffle code today
private[spark] trait ShuffleManager {
def registerShuffle[K, V, C](shuffleId: Int, numMaps: Int, dependency: ShuffleDependency[K, V,
C]): ShuffleHandle
def getWriter[K, V](handle: ShuffleHandle, mapId: Int, context: TaskContext, metrics:
ShuffleWriteMetricsReporter): ShuffleWriter[K, V]
def getReader[K, C](handle: ShuffleHandle, startPartition: Int, endPartition: Int, context:
TaskContext, metrics: ShuffleReadMetricsReporter): ShuffleReader[K, C]
def unregisterShuffle(shuffleId: Int): Boolean
def shuffleBlockResolver: ShuffleBlockResolver
def stop(): Unit
}
24#UnifiedDataAnalytics #SparkAISummit
Continued..
/**
* Obtained inside a map task to write out records to the shuffle system.
*/
private[spark] abstract class ShuffleWriter[K, V] {
/** Write a sequence of records to this task's output */
@throws[IOException]
def write(records: Iterator[Product2[K, V]]): Unit
/** Close this writer, passing along whether the map completed */
def stop(success: Boolean): Option[MapStatus]
}
25#UnifiedDataAnalytics #SparkAISummit
Continued..
/** Write a bunch of records to this task's output */
override def write(records: Iterator[Product2[K, V]]): Unit = {
sorter = if (dep.mapSideCombine) {
new ExternalSorter[K, V, C](
context, dep.aggregator, Some(dep.partitioner), dep.keyOrdering, dep.serializer)
} else {
// In this case we pass neither an aggregator nor an ordering to the sorter, because we
don't
// care whether the keys get sorted in each partition; that will be done on the reduce
side
// if the operation being run is sortByKey.
new ExternalSorter[K, V, V](
context, aggregator = None, Some(dep.partitioner), ordering = None, dep.serializer)
}
sorter.insertAll(records)
...
26#UnifiedDataAnalytics #SparkAISummit
Continued..
// Don't bother including the time to open the merged output file in the shuffle write time,
// because it just opens a single file, so is typically too fast to measure accurately
// (see SPARK-3570).
val output = shuffleBlockResolver.getDataFile(dep.shuffleId, mapId)
val tmp = Utils.tempFileWith(output)
try {
val blockId = ShuffleBlockId(dep.shuffleId, mapId,
IndexShuffleBlockResolver.NOOP_REDUCE_ID)
val partitionLengths = sorter.writePartitionedFile(blockId, tmp)
shuffleBlockResolver.writeIndexFileAndCommit(dep.shuffleId, mapId, partitionLengths, tmp)
mapStatus = MapStatus(blockManager.shuffleServerId, partitionLengths)
} finally {
if (tmp.exists() && !tmp.delete()) {
logError(s"Error while deleting temp file ${tmp.getAbsolutePath}")
}
}
}
27#UnifiedDataAnalytics #SparkAISummit
Continued..
/ Note: Changes to the format in this file should be kept in sync with
//
org.apache.spark.network.shuffle.ExternalShuffleBlockResolver#getSortBasedShuffleBlockData().
private[spark] class IndexShuffleBlockResolver(
conf: SparkConf,
_blockManager: BlockManager = null)
extends ShuffleBlockResolver
………..
28#UnifiedDataAnalytics #SparkAISummit
Problems with This
● Rapid downscaling infeasible
○ Scaling down entire nodes hard
● Preemptible VMs & Spot Instances
29#UnifiedDataAnalytics #SparkAISummit
Optimizing Costs
Preemptible
VMs and
Spot
instances
PVMs Up to 80% cheaper for
short-lived instances. Can be pulled
at any time. Guaranteed to be
removed at least once in 24 hours.
Spot is based on Vickrey auction.
Stage 1 Stage 2Shuffle
How can we fix this?
Make intermediate shuffle data external to both the executor and the
machine itself
33#UnifiedDataAnalytics #SparkAISummit
Where we started
class HcfsShuffleWriter[K, V, C] extends ShuffleWriter[K, V] {
override def write(records: Iterator[Product2[K, V]]): Unit = {
val sorter = new ExternalSorter[K, V, C/V](...)
sorter.insertAll(records)
val partitionIter = sorter.partitionedIter
val hcfsStream = …
val countingStream = new CountingOutputStream(hcfsStream)
val framedOutput = new FramingOutputStream(countingStream)
try {
for ((partition, iter) <- partitionIter) {
// Write partition to external storage
}
} finally {
framedOutput.closeUnderlying()
}
}
34#UnifiedDataAnalytics #SparkAISummit
Alpha: HDFS not quite ready for prime time
● RPC overhead to HDFS or persistent storage
● Especially poor performance with misaligned partition/block sizes
○ HDFS/GCS/etc different expectations of block size
● Loss of implicit in-memory page cache
● Possibly slowness in cleaning up shuffle files
● Namenode contention when reading shuffle files (HDFS)
○ Added index caching layer to mitigate this
● Additional metadata tracking
36#UnifiedDataAnalytics #SparkAISummit
Object Storage?
Apache Crail (Incubating) is a high-performance distributed data store designed for fast sharing
of ephemeral data in distributed data processing workloads
● Fast
● Heterogeneous
● Modular
What about Google Cloud Bigtable?
Consistent low latency, high
throughput, and scalable
wide-column database service.
Back to basics - NFS
● Shuffle to Elastifile
○ Cloud based NFS service (scales horizontally)
○ Tailored to random access patterns, small files
○ NFS looks like local FS, but is not. Must be careful when dealing with
commit semantics and speculative execution.
● Still a performance hit but factors better than HDFS
41#UnifiedDataAnalytics #SparkAISummit
Goal: OSS Disaggregated Shuffle
Architecture
Kubernetes Cluster
Spark Driver Pod
Shuffle
Offload
(WIP)
Executor
Virtual Machine Group
Elastifile
Cloud (object)
Storage
Use the cloud to
fix the cloud?
DON’T FORGET TO RATE
AND REVIEW THE SESSIONS
SEARCH SPARK + AI SUMMIT

More Related Content

What's hot

What's hot (20)

Apache Kafka Best Practices
Apache Kafka Best PracticesApache Kafka Best Practices
Apache Kafka Best Practices
 
Kafka Streams State Stores Being Persistent
Kafka Streams State Stores Being PersistentKafka Streams State Stores Being Persistent
Kafka Streams State Stores Being Persistent
 
Tuning Apache Spark for Large-Scale Workloads Gaoxiang Liu and Sital Kedia
Tuning Apache Spark for Large-Scale Workloads Gaoxiang Liu and Sital KediaTuning Apache Spark for Large-Scale Workloads Gaoxiang Liu and Sital Kedia
Tuning Apache Spark for Large-Scale Workloads Gaoxiang Liu and Sital Kedia
 
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
 
Running Apache Spark on Kubernetes: Best Practices and Pitfalls
Running Apache Spark on Kubernetes: Best Practices and PitfallsRunning Apache Spark on Kubernetes: Best Practices and Pitfalls
Running Apache Spark on Kubernetes: Best Practices and Pitfalls
 
Introduction to Kafka Streams
Introduction to Kafka StreamsIntroduction to Kafka Streams
Introduction to Kafka Streams
 
Introduction to Apache Kafka
Introduction to Apache KafkaIntroduction to Apache Kafka
Introduction to Apache Kafka
 
Top 5 mistakes when writing Spark applications
Top 5 mistakes when writing Spark applicationsTop 5 mistakes when writing Spark applications
Top 5 mistakes when writing Spark applications
 
Apache Kafka Introduction
Apache Kafka IntroductionApache Kafka Introduction
Apache Kafka Introduction
 
The Patterns of Distributed Logging and Containers
The Patterns of Distributed Logging and ContainersThe Patterns of Distributed Logging and Containers
The Patterns of Distributed Logging and Containers
 
Lessons from the Field: Applying Best Practices to Your Apache Spark Applicat...
Lessons from the Field: Applying Best Practices to Your Apache Spark Applicat...Lessons from the Field: Applying Best Practices to Your Apache Spark Applicat...
Lessons from the Field: Applying Best Practices to Your Apache Spark Applicat...
 
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
 
Cassandra Introduction & Features
Cassandra Introduction & FeaturesCassandra Introduction & Features
Cassandra Introduction & Features
 
Performance Tuning RocksDB for Kafka Streams’ State Stores
Performance Tuning RocksDB for Kafka Streams’ State StoresPerformance Tuning RocksDB for Kafka Streams’ State Stores
Performance Tuning RocksDB for Kafka Streams’ State Stores
 
Dr. Elephant for Monitoring and Tuning Apache Spark Jobs on Hadoop with Carl ...
Dr. Elephant for Monitoring and Tuning Apache Spark Jobs on Hadoop with Carl ...Dr. Elephant for Monitoring and Tuning Apache Spark Jobs on Hadoop with Carl ...
Dr. Elephant for Monitoring and Tuning Apache Spark Jobs on Hadoop with Carl ...
 
Spark SQL Deep Dive @ Melbourne Spark Meetup
Spark SQL Deep Dive @ Melbourne Spark MeetupSpark SQL Deep Dive @ Melbourne Spark Meetup
Spark SQL Deep Dive @ Melbourne Spark Meetup
 
ksqlDB: A Stream-Relational Database System
ksqlDB: A Stream-Relational Database SystemksqlDB: A Stream-Relational Database System
ksqlDB: A Stream-Relational Database System
 
Apache Kafka
Apache KafkaApache Kafka
Apache Kafka
 
Stream processing with Apache Flink (Timo Walther - Ververica)
Stream processing with Apache Flink (Timo Walther - Ververica)Stream processing with Apache Flink (Timo Walther - Ververica)
Stream processing with Apache Flink (Timo Walther - Ververica)
 
XStream: stream processing platform at facebook
XStream:  stream processing platform at facebookXStream:  stream processing platform at facebook
XStream: stream processing platform at facebook
 

Similar to Improving Apache Spark Downscaling

SF Big Analytics_20190612: Scaling Apache Spark on Kubernetes at Lyft
SF Big Analytics_20190612: Scaling Apache Spark on Kubernetes at LyftSF Big Analytics_20190612: Scaling Apache Spark on Kubernetes at Lyft
SF Big Analytics_20190612: Scaling Apache Spark on Kubernetes at Lyft
Chester Chen
 
Apache Druid Auto Scale-out/in for Streaming Data Ingestion on Kubernetes
Apache Druid Auto Scale-out/in for Streaming Data Ingestion on KubernetesApache Druid Auto Scale-out/in for Streaming Data Ingestion on Kubernetes
Apache Druid Auto Scale-out/in for Streaming Data Ingestion on Kubernetes
DataWorks Summit
 

Similar to Improving Apache Spark Downscaling (20)

Kerberizing Spark: Spark Summit East talk by Abel Rincon and Jorge Lopez-Malla
Kerberizing Spark: Spark Summit East talk by Abel Rincon and Jorge Lopez-MallaKerberizing Spark: Spark Summit East talk by Abel Rincon and Jorge Lopez-Malla
Kerberizing Spark: Spark Summit East talk by Abel Rincon and Jorge Lopez-Malla
 
DevEx | there’s no place like k3s
DevEx | there’s no place like k3sDevEx | there’s no place like k3s
DevEx | there’s no place like k3s
 
SF Big Analytics_20190612: Scaling Apache Spark on Kubernetes at Lyft
SF Big Analytics_20190612: Scaling Apache Spark on Kubernetes at LyftSF Big Analytics_20190612: Scaling Apache Spark on Kubernetes at Lyft
SF Big Analytics_20190612: Scaling Apache Spark on Kubernetes at Lyft
 
Apache Druid Auto Scale-out/in for Streaming Data Ingestion on Kubernetes
Apache Druid Auto Scale-out/in for Streaming Data Ingestion on KubernetesApache Druid Auto Scale-out/in for Streaming Data Ingestion on Kubernetes
Apache Druid Auto Scale-out/in for Streaming Data Ingestion on Kubernetes
 
Scalable Clusters On Demand
Scalable Clusters On DemandScalable Clusters On Demand
Scalable Clusters On Demand
 
Data Engineer's Lunch #82: Automating Apache Cassandra Operations with Apache...
Data Engineer's Lunch #82: Automating Apache Cassandra Operations with Apache...Data Engineer's Lunch #82: Automating Apache Cassandra Operations with Apache...
Data Engineer's Lunch #82: Automating Apache Cassandra Operations with Apache...
 
Powerful Google developer tools for immediate impact! (2023-24 C)
Powerful Google developer tools for immediate impact! (2023-24 C)Powerful Google developer tools for immediate impact! (2023-24 C)
Powerful Google developer tools for immediate impact! (2023-24 C)
 
Flink Forward SF 2017: James Malone - Make The Cloud Work For You
Flink Forward SF 2017: James Malone - Make The Cloud Work For YouFlink Forward SF 2017: James Malone - Make The Cloud Work For You
Flink Forward SF 2017: James Malone - Make The Cloud Work For You
 
Volodymyr Lyubinets "Introduction to big data processing with Apache Spark"
Volodymyr Lyubinets "Introduction to big data processing with Apache Spark"Volodymyr Lyubinets "Introduction to big data processing with Apache Spark"
Volodymyr Lyubinets "Introduction to big data processing with Apache Spark"
 
Free GitOps Workshop + Intro to Kubernetes & GitOps
Free GitOps Workshop + Intro to Kubernetes & GitOpsFree GitOps Workshop + Intro to Kubernetes & GitOps
Free GitOps Workshop + Intro to Kubernetes & GitOps
 
Handout3o
Handout3oHandout3o
Handout3o
 
Vitalii Bondarenko HDinsight: spark. advanced in memory big-data analytics wi...
Vitalii Bondarenko HDinsight: spark. advanced in memory big-data analytics wi...Vitalii Bondarenko HDinsight: spark. advanced in memory big-data analytics wi...
Vitalii Bondarenko HDinsight: spark. advanced in memory big-data analytics wi...
 
SE2016 BigData Vitalii Bondarenko "HD insight spark. Advanced in-memory Big D...
SE2016 BigData Vitalii Bondarenko "HD insight spark. Advanced in-memory Big D...SE2016 BigData Vitalii Bondarenko "HD insight spark. Advanced in-memory Big D...
SE2016 BigData Vitalii Bondarenko "HD insight spark. Advanced in-memory Big D...
 
Running Airflow Workflows as ETL Processes on Hadoop
Running Airflow Workflows as ETL Processes on HadoopRunning Airflow Workflows as ETL Processes on Hadoop
Running Airflow Workflows as ETL Processes on Hadoop
 
Google Cloud Dataflow
Google Cloud DataflowGoogle Cloud Dataflow
Google Cloud Dataflow
 
[20200720]cloud native develoment - Nelson Lin
[20200720]cloud native develoment - Nelson Lin[20200720]cloud native develoment - Nelson Lin
[20200720]cloud native develoment - Nelson Lin
 
Hybrid Cloud, Kubeflow and Tensorflow Extended [TFX]
Hybrid Cloud, Kubeflow and Tensorflow Extended [TFX]Hybrid Cloud, Kubeflow and Tensorflow Extended [TFX]
Hybrid Cloud, Kubeflow and Tensorflow Extended [TFX]
 
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
 
ETL with SPARK - First Spark London meetup
ETL with SPARK - First Spark London meetupETL with SPARK - First Spark London meetup
ETL with SPARK - First Spark London meetup
 
How to Puppetize Google Cloud Platform - PuppetConf 2014
How to Puppetize Google Cloud Platform - PuppetConf 2014How to Puppetize Google Cloud Platform - PuppetConf 2014
How to Puppetize Google Cloud Platform - PuppetConf 2014
 

More from Databricks

Democratizing Data Quality Through a Centralized Platform
Democratizing Data Quality Through a Centralized PlatformDemocratizing Data Quality Through a Centralized Platform
Democratizing Data Quality Through a Centralized Platform
Databricks
 
Stage Level Scheduling Improving Big Data and AI Integration
Stage Level Scheduling Improving Big Data and AI IntegrationStage Level Scheduling Improving Big Data and AI Integration
Stage Level Scheduling Improving Big Data and AI Integration
Databricks
 
Simplify Data Conversion from Spark to TensorFlow and PyTorch
Simplify Data Conversion from Spark to TensorFlow and PyTorchSimplify Data Conversion from Spark to TensorFlow and PyTorch
Simplify Data Conversion from Spark to TensorFlow and PyTorch
Databricks
 
Raven: End-to-end Optimization of ML Prediction Queries
Raven: End-to-end Optimization of ML Prediction QueriesRaven: End-to-end Optimization of ML Prediction Queries
Raven: End-to-end Optimization of ML Prediction Queries
Databricks
 
Processing Large Datasets for ADAS Applications using Apache Spark
Processing Large Datasets for ADAS Applications using Apache SparkProcessing Large Datasets for ADAS Applications using Apache Spark
Processing Large Datasets for ADAS Applications using Apache Spark
Databricks
 

More from Databricks (20)

DW Migration Webinar-March 2022.pptx
DW Migration Webinar-March 2022.pptxDW Migration Webinar-March 2022.pptx
DW Migration Webinar-March 2022.pptx
 
Data Lakehouse Symposium | Day 1 | Part 1
Data Lakehouse Symposium | Day 1 | Part 1Data Lakehouse Symposium | Day 1 | Part 1
Data Lakehouse Symposium | Day 1 | Part 1
 
Data Lakehouse Symposium | Day 1 | Part 2
Data Lakehouse Symposium | Day 1 | Part 2Data Lakehouse Symposium | Day 1 | Part 2
Data Lakehouse Symposium | Day 1 | Part 2
 
Data Lakehouse Symposium | Day 2
Data Lakehouse Symposium | Day 2Data Lakehouse Symposium | Day 2
Data Lakehouse Symposium | Day 2
 
Data Lakehouse Symposium | Day 4
Data Lakehouse Symposium | Day 4Data Lakehouse Symposium | Day 4
Data Lakehouse Symposium | Day 4
 
5 Critical Steps to Clean Your Data Swamp When Migrating Off of Hadoop
5 Critical Steps to Clean Your Data Swamp When Migrating Off of Hadoop5 Critical Steps to Clean Your Data Swamp When Migrating Off of Hadoop
5 Critical Steps to Clean Your Data Swamp When Migrating Off of Hadoop
 
Democratizing Data Quality Through a Centralized Platform
Democratizing Data Quality Through a Centralized PlatformDemocratizing Data Quality Through a Centralized Platform
Democratizing Data Quality Through a Centralized Platform
 
Learn to Use Databricks for Data Science
Learn to Use Databricks for Data ScienceLearn to Use Databricks for Data Science
Learn to Use Databricks for Data Science
 
Why APM Is Not the Same As ML Monitoring
Why APM Is Not the Same As ML MonitoringWhy APM Is Not the Same As ML Monitoring
Why APM Is Not the Same As ML Monitoring
 
The Function, the Context, and the Data—Enabling ML Ops at Stitch Fix
The Function, the Context, and the Data—Enabling ML Ops at Stitch FixThe Function, the Context, and the Data—Enabling ML Ops at Stitch Fix
The Function, the Context, and the Data—Enabling ML Ops at Stitch Fix
 
Stage Level Scheduling Improving Big Data and AI Integration
Stage Level Scheduling Improving Big Data and AI IntegrationStage Level Scheduling Improving Big Data and AI Integration
Stage Level Scheduling Improving Big Data and AI Integration
 
Simplify Data Conversion from Spark to TensorFlow and PyTorch
Simplify Data Conversion from Spark to TensorFlow and PyTorchSimplify Data Conversion from Spark to TensorFlow and PyTorch
Simplify Data Conversion from Spark to TensorFlow and PyTorch
 
Scaling 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
 
Machine Learning CI/CD for Email Attack Detection
Machine Learning CI/CD for Email Attack DetectionMachine Learning CI/CD for Email Attack Detection
Machine Learning CI/CD for Email Attack Detection
 

Recently uploaded

In Riyadh ((+919101817206)) Cytotec kit @ Abortion Pills Saudi Arabia
In Riyadh ((+919101817206)) Cytotec kit @ Abortion Pills Saudi ArabiaIn Riyadh ((+919101817206)) Cytotec kit @ Abortion Pills Saudi Arabia
In Riyadh ((+919101817206)) Cytotec kit @ Abortion Pills Saudi Arabia
ahmedjiabur940
 
Top profile Call Girls In dimapur [ 7014168258 ] Call Me For Genuine Models W...
Top profile Call Girls In dimapur [ 7014168258 ] Call Me For Genuine Models W...Top profile Call Girls In dimapur [ 7014168258 ] Call Me For Genuine Models W...
Top profile Call Girls In dimapur [ 7014168258 ] Call Me For Genuine Models W...
gajnagarg
 
Top profile Call Girls In Vadodara [ 7014168258 ] Call Me For Genuine Models ...
Top profile Call Girls In Vadodara [ 7014168258 ] Call Me For Genuine Models ...Top profile Call Girls In Vadodara [ 7014168258 ] Call Me For Genuine Models ...
Top profile Call Girls In Vadodara [ 7014168258 ] Call Me For Genuine Models ...
gajnagarg
 
Top profile Call Girls In Tumkur [ 7014168258 ] Call Me For Genuine Models We...
Top profile Call Girls In Tumkur [ 7014168258 ] Call Me For Genuine Models We...Top profile Call Girls In Tumkur [ 7014168258 ] Call Me For Genuine Models We...
Top profile Call Girls In Tumkur [ 7014168258 ] Call Me For Genuine Models We...
nirzagarg
 
一比一原版(曼大毕业证书)曼尼托巴大学毕业证成绩单留信学历认证一手价格
一比一原版(曼大毕业证书)曼尼托巴大学毕业证成绩单留信学历认证一手价格一比一原版(曼大毕业证书)曼尼托巴大学毕业证成绩单留信学历认证一手价格
一比一原版(曼大毕业证书)曼尼托巴大学毕业证成绩单留信学历认证一手价格
q6pzkpark
 
怎样办理旧金山城市学院毕业证(CCSF毕业证书)成绩单学校原版复制
怎样办理旧金山城市学院毕业证(CCSF毕业证书)成绩单学校原版复制怎样办理旧金山城市学院毕业证(CCSF毕业证书)成绩单学校原版复制
怎样办理旧金山城市学院毕业证(CCSF毕业证书)成绩单学校原版复制
vexqp
 
Cytotec in Jeddah+966572737505) get unwanted pregnancy kit Riyadh
Cytotec in Jeddah+966572737505) get unwanted pregnancy kit RiyadhCytotec in Jeddah+966572737505) get unwanted pregnancy kit Riyadh
Cytotec in Jeddah+966572737505) get unwanted pregnancy kit Riyadh
Abortion pills in Riyadh +966572737505 get cytotec
 
Reconciling Conflicting Data Curation Actions: Transparency Through Argument...
Reconciling Conflicting Data Curation Actions:  Transparency Through Argument...Reconciling Conflicting Data Curation Actions:  Transparency Through Argument...
Reconciling Conflicting Data Curation Actions: Transparency Through Argument...
Bertram Ludäscher
 
Top profile Call Girls In Purnia [ 7014168258 ] Call Me For Genuine Models We...
Top profile Call Girls In Purnia [ 7014168258 ] Call Me For Genuine Models We...Top profile Call Girls In Purnia [ 7014168258 ] Call Me For Genuine Models We...
Top profile Call Girls In Purnia [ 7014168258 ] Call Me For Genuine Models We...
nirzagarg
 
PLE-statistics document for primary schs
PLE-statistics document for primary schsPLE-statistics document for primary schs
PLE-statistics document for primary schs
cnajjemba
 
Gartner's Data Analytics Maturity Model.pptx
Gartner's Data Analytics Maturity Model.pptxGartner's Data Analytics Maturity Model.pptx
Gartner's Data Analytics Maturity Model.pptx
chadhar227
 
Top profile Call Girls In Bihar Sharif [ 7014168258 ] Call Me For Genuine Mod...
Top profile Call Girls In Bihar Sharif [ 7014168258 ] Call Me For Genuine Mod...Top profile Call Girls In Bihar Sharif [ 7014168258 ] Call Me For Genuine Mod...
Top profile Call Girls In Bihar Sharif [ 7014168258 ] Call Me For Genuine Mod...
nirzagarg
 
如何办理英国诺森比亚大学毕业证(NU毕业证书)成绩单原件一模一样
如何办理英国诺森比亚大学毕业证(NU毕业证书)成绩单原件一模一样如何办理英国诺森比亚大学毕业证(NU毕业证书)成绩单原件一模一样
如何办理英国诺森比亚大学毕业证(NU毕业证书)成绩单原件一模一样
wsppdmt
 
怎样办理纽约州立大学宾汉姆顿分校毕业证(SUNY-Bin毕业证书)成绩单学校原版复制
怎样办理纽约州立大学宾汉姆顿分校毕业证(SUNY-Bin毕业证书)成绩单学校原版复制怎样办理纽约州立大学宾汉姆顿分校毕业证(SUNY-Bin毕业证书)成绩单学校原版复制
怎样办理纽约州立大学宾汉姆顿分校毕业证(SUNY-Bin毕业证书)成绩单学校原版复制
vexqp
 
Top profile Call Girls In Chandrapur [ 7014168258 ] Call Me For Genuine Model...
Top profile Call Girls In Chandrapur [ 7014168258 ] Call Me For Genuine Model...Top profile Call Girls In Chandrapur [ 7014168258 ] Call Me For Genuine Model...
Top profile Call Girls In Chandrapur [ 7014168258 ] Call Me For Genuine Model...
gajnagarg
 

Recently uploaded (20)

In Riyadh ((+919101817206)) Cytotec kit @ Abortion Pills Saudi Arabia
In Riyadh ((+919101817206)) Cytotec kit @ Abortion Pills Saudi ArabiaIn Riyadh ((+919101817206)) Cytotec kit @ Abortion Pills Saudi Arabia
In Riyadh ((+919101817206)) Cytotec kit @ Abortion Pills Saudi Arabia
 
Top profile Call Girls In dimapur [ 7014168258 ] Call Me For Genuine Models W...
Top profile Call Girls In dimapur [ 7014168258 ] Call Me For Genuine Models W...Top profile Call Girls In dimapur [ 7014168258 ] Call Me For Genuine Models W...
Top profile Call Girls In dimapur [ 7014168258 ] Call Me For Genuine Models W...
 
Top profile Call Girls In Vadodara [ 7014168258 ] Call Me For Genuine Models ...
Top profile Call Girls In Vadodara [ 7014168258 ] Call Me For Genuine Models ...Top profile Call Girls In Vadodara [ 7014168258 ] Call Me For Genuine Models ...
Top profile Call Girls In Vadodara [ 7014168258 ] Call Me For Genuine Models ...
 
Top profile Call Girls In Tumkur [ 7014168258 ] Call Me For Genuine Models We...
Top profile Call Girls In Tumkur [ 7014168258 ] Call Me For Genuine Models We...Top profile Call Girls In Tumkur [ 7014168258 ] Call Me For Genuine Models We...
Top profile Call Girls In Tumkur [ 7014168258 ] Call Me For Genuine Models We...
 
一比一原版(曼大毕业证书)曼尼托巴大学毕业证成绩单留信学历认证一手价格
一比一原版(曼大毕业证书)曼尼托巴大学毕业证成绩单留信学历认证一手价格一比一原版(曼大毕业证书)曼尼托巴大学毕业证成绩单留信学历认证一手价格
一比一原版(曼大毕业证书)曼尼托巴大学毕业证成绩单留信学历认证一手价格
 
怎样办理旧金山城市学院毕业证(CCSF毕业证书)成绩单学校原版复制
怎样办理旧金山城市学院毕业证(CCSF毕业证书)成绩单学校原版复制怎样办理旧金山城市学院毕业证(CCSF毕业证书)成绩单学校原版复制
怎样办理旧金山城市学院毕业证(CCSF毕业证书)成绩单学校原版复制
 
Cytotec in Jeddah+966572737505) get unwanted pregnancy kit Riyadh
Cytotec in Jeddah+966572737505) get unwanted pregnancy kit RiyadhCytotec in Jeddah+966572737505) get unwanted pregnancy kit Riyadh
Cytotec in Jeddah+966572737505) get unwanted pregnancy kit Riyadh
 
Reconciling Conflicting Data Curation Actions: Transparency Through Argument...
Reconciling Conflicting Data Curation Actions:  Transparency Through Argument...Reconciling Conflicting Data Curation Actions:  Transparency Through Argument...
Reconciling Conflicting Data Curation Actions: Transparency Through Argument...
 
Top profile Call Girls In Purnia [ 7014168258 ] Call Me For Genuine Models We...
Top profile Call Girls In Purnia [ 7014168258 ] Call Me For Genuine Models We...Top profile Call Girls In Purnia [ 7014168258 ] Call Me For Genuine Models We...
Top profile Call Girls In Purnia [ 7014168258 ] Call Me For Genuine Models We...
 
Dubai Call Girls Peeing O525547819 Call Girls Dubai
Dubai Call Girls Peeing O525547819 Call Girls DubaiDubai Call Girls Peeing O525547819 Call Girls Dubai
Dubai Call Girls Peeing O525547819 Call Girls Dubai
 
PLE-statistics document for primary schs
PLE-statistics document for primary schsPLE-statistics document for primary schs
PLE-statistics document for primary schs
 
The-boAt-Story-Navigating-the-Waves-of-Innovation.pptx
The-boAt-Story-Navigating-the-Waves-of-Innovation.pptxThe-boAt-Story-Navigating-the-Waves-of-Innovation.pptx
The-boAt-Story-Navigating-the-Waves-of-Innovation.pptx
 
Gartner's Data Analytics Maturity Model.pptx
Gartner's Data Analytics Maturity Model.pptxGartner's Data Analytics Maturity Model.pptx
Gartner's Data Analytics Maturity Model.pptx
 
Top profile Call Girls In Bihar Sharif [ 7014168258 ] Call Me For Genuine Mod...
Top profile Call Girls In Bihar Sharif [ 7014168258 ] Call Me For Genuine Mod...Top profile Call Girls In Bihar Sharif [ 7014168258 ] Call Me For Genuine Mod...
Top profile Call Girls In Bihar Sharif [ 7014168258 ] Call Me For Genuine Mod...
 
SAC 25 Final National, Regional & Local Angel Group Investing Insights 2024 0...
SAC 25 Final National, Regional & Local Angel Group Investing Insights 2024 0...SAC 25 Final National, Regional & Local Angel Group Investing Insights 2024 0...
SAC 25 Final National, Regional & Local Angel Group Investing Insights 2024 0...
 
如何办理英国诺森比亚大学毕业证(NU毕业证书)成绩单原件一模一样
如何办理英国诺森比亚大学毕业证(NU毕业证书)成绩单原件一模一样如何办理英国诺森比亚大学毕业证(NU毕业证书)成绩单原件一模一样
如何办理英国诺森比亚大学毕业证(NU毕业证书)成绩单原件一模一样
 
怎样办理纽约州立大学宾汉姆顿分校毕业证(SUNY-Bin毕业证书)成绩单学校原版复制
怎样办理纽约州立大学宾汉姆顿分校毕业证(SUNY-Bin毕业证书)成绩单学校原版复制怎样办理纽约州立大学宾汉姆顿分校毕业证(SUNY-Bin毕业证书)成绩单学校原版复制
怎样办理纽约州立大学宾汉姆顿分校毕业证(SUNY-Bin毕业证书)成绩单学校原版复制
 
Top profile Call Girls In Chandrapur [ 7014168258 ] Call Me For Genuine Model...
Top profile Call Girls In Chandrapur [ 7014168258 ] Call Me For Genuine Model...Top profile Call Girls In Chandrapur [ 7014168258 ] Call Me For Genuine Model...
Top profile Call Girls In Chandrapur [ 7014168258 ] Call Me For Genuine Model...
 
Discover Why Less is More in B2B Research
Discover Why Less is More in B2B ResearchDiscover Why Less is More in B2B Research
Discover Why Less is More in B2B Research
 
DATA SUMMIT 24 Building Real-Time Pipelines With FLaNK
DATA SUMMIT 24  Building Real-Time Pipelines With FLaNKDATA SUMMIT 24  Building Real-Time Pipelines With FLaNK
DATA SUMMIT 24 Building Real-Time Pipelines With FLaNK
 

Improving Apache Spark Downscaling

  • 1. WIFI SSID:Spark+AISummit | Password: UnifiedDataAnalytics
  • 2. Christopher Crosbie, Google Ben Sidhom, Google Improving Spark Downscaling #UnifiedDataAnalytics #SparkAISummit
  • 3. Open Source Google Cloud Products Google Research 2000 2010 GFS Map Reduce Dremel Flume Java MillwheelPubSub 2020 BigTable BigQuery Pub/Sub Dataflow Bigtable MLDataproc Long History of Solving Data Problems Tensorflow
  • 4. Apache Airflow Cloud ML Engine Cloud Dataflow Cloud Data Fusion Cloud Composer
  • 5. Who are we and what is Cloud Dataproc? Google Cloud Platform’s fully-managed Apache Spark and Apache Hadoop service Rapid cluster creation Familiar open source tools Customizable machines Ephemeral clusters on-demand Tightly Integrated with other Google Cloud Platform services
  • 6. Cloud Dataproc: Open source solutions with GCP Taking the best of open source And opening up access to the best of GCP Webhcat BigQuery Cloud Datastore Cloud Bigtable Compute Engine Kubernetes Engine Cloud Dataflow Cloud Dataproc Cloud Functions Cloud Machine Learning Engine Cloud Pub/Sub Key Management Service Cloud Spanner Cloud SQL BQ Transfer Service Cloud Translation API Cloud Vision API Cloud Storage
  • 7. Jobs are “fire and forget” No need to manually intervene when a cluster is over or under capacity Choose balance between standard and preemptible workers Save resources (quota & cost) at any point in time Dataproc Autoscaling GA Complicating Spark Downscaling Without autoscaling Submit job Monitor resource usage Adjust cluster size With autoscaling Submit jobs
  • 8. Based on the difference between YARN pending and available memory If more memory is needed then scale up If there is excess memory then scale down Obey VM limits and scale based on scale factor Autoscaling policies: fine grained control Is there too much or too little YARN memory? Do nothing Is the cluster at the maximum # of nodes? Do not autoscale Determine type and scale of nodes to modify Autoscale cluster Yes No Yes No
  • 9. YARN Infrastructure Complexities Finding processed data (shuffle files, cached RDDs, etc) Optimizing costs Spark Autoscaling Challenges
  • 10. YARN
  • 11. YARN-based managed Spark Dataproc Cluster HDFS Persistent Disk Cluster bucket Cloud Storage Compute engine nodes Dataproc Image Apache Spark Apache Hadoop Apache Hive ... Clients Cloud Dataproc API Clusters ... Jobs Clients (SSH) Dataproc Agent User Data Cloud Storage
  • 12. YARN pain points Management is difficult Clusters are complicated and have to use more components than are required for a job or model. This also requires hard-to-find experts. Complicated OSS software stack Version and dependency management is hard. Have to understand how to tune multiple components for efficiency. Isolation is hard I have to think about my jobs to size clusters, and isolating jobs requires additional steps.
  • 13.
  • 14. Multiple k8s options Moving the OSS ecosystem to Kubernetes offers customers a range of options depending on their needs and core expertise. DIY k8s Dataproc k8s Dataproc + Vendor components Runs OSS on k8s? Yes - self-managed Yes - managed k8s clusters Yes - managed k8s clusters SLAs GKE only Dataproc cluster Dataproc cluster and component OSS components Community only Google optimized Google optimized + vendor optimized In-depth component support No No Yes Integrated management No Yes Yes Integrated security No Yes Yes Hybrid/cross-cloud support No Yes Yes
  • 15. How we are making this happen • Kubernetes Operators - Application control plane for complex applications – The language of Kubernetes allows extending its vocabulary through Custom Resource Definition (CRD) – Kubernetes Operator is an app-specific control plane running in the cluster • CRD: app-specific vocabulary • CR: instance of CRD • CR Controller: interpreter and reconciliation loop for CRs – The cluster can now speak the app-specific words through the Kubernetes API Control Plane (Master) MyApp API Data Plane (Nodes) CRUD MyApp ... Kubernetes MyApp Control Plane Kubernetes API
  • 16. ● Integrates with BigQuery, Google’s Serverless Data Warehouse ● Provides Google Cloud Storage as replacement for HDFS ● Ships logs to Stackdriver Monitoring ○ via Prometheus server with the Stackdriver sidecar ● Contains sparkctl, a command line tool that simplifies client-local application dependencies in a Kubernetes environment. https://github.com/GoogleCloudPlatform/spark-on-k8s-operator
  • 18. 1. Deploy unified resource management Get away deal from two separate cluster management interfaces to manage open source component. Offers one central view for easy management. 2. Isolate Spark jobs and resources Remove the headaches of version and dependency management; instead, move models and ETL pipelines from dev to production without added work. Build resilient infrastructure Don’t worry about sizing and building clusters, manipulating Docker files, or messing around with Kubernetes networking configurations. It just works. Key benefits for autoscaling
  • 19. Helpful but does not solve our core problem…..
  • 21. What exactly is a shuffle & why do we care? Rob Wynne
  • 22. A Brief History of Spark Shuffle ● Shuffle files to local storage on the executors ● Executors responsible for serving the files ● Loss of an executor meant loss of the shuffle files ● Result: poor auto-scaling ○ Pathological loop: scale down, lose work, re-compute, trigger scale up… ● Depended on driver GC event to clean up shuffle files 22#UnifiedDataAnalytics #SparkAISummit
  • 23. Today: Dynamic allocation and “external” shuffle ● Executors no longer need to serve data ● “External” shuffle is not exactly external ○ Only executors can be released ○ Can scale up & down executors but not the machines ● Still depends on driver GC event to clean up shuffle files 23#UnifiedDataAnalytics #SparkAISummit
  • 24. Spark’s shuffle code today private[spark] trait ShuffleManager { def registerShuffle[K, V, C](shuffleId: Int, numMaps: Int, dependency: ShuffleDependency[K, V, C]): ShuffleHandle def getWriter[K, V](handle: ShuffleHandle, mapId: Int, context: TaskContext, metrics: ShuffleWriteMetricsReporter): ShuffleWriter[K, V] def getReader[K, C](handle: ShuffleHandle, startPartition: Int, endPartition: Int, context: TaskContext, metrics: ShuffleReadMetricsReporter): ShuffleReader[K, C] def unregisterShuffle(shuffleId: Int): Boolean def shuffleBlockResolver: ShuffleBlockResolver def stop(): Unit } 24#UnifiedDataAnalytics #SparkAISummit
  • 25. Continued.. /** * Obtained inside a map task to write out records to the shuffle system. */ private[spark] abstract class ShuffleWriter[K, V] { /** Write a sequence of records to this task's output */ @throws[IOException] def write(records: Iterator[Product2[K, V]]): Unit /** Close this writer, passing along whether the map completed */ def stop(success: Boolean): Option[MapStatus] } 25#UnifiedDataAnalytics #SparkAISummit
  • 26. Continued.. /** Write a bunch of records to this task's output */ override def write(records: Iterator[Product2[K, V]]): Unit = { sorter = if (dep.mapSideCombine) { new ExternalSorter[K, V, C]( context, dep.aggregator, Some(dep.partitioner), dep.keyOrdering, dep.serializer) } else { // In this case we pass neither an aggregator nor an ordering to the sorter, because we don't // care whether the keys get sorted in each partition; that will be done on the reduce side // if the operation being run is sortByKey. new ExternalSorter[K, V, V]( context, aggregator = None, Some(dep.partitioner), ordering = None, dep.serializer) } sorter.insertAll(records) ... 26#UnifiedDataAnalytics #SparkAISummit
  • 27. Continued.. // Don't bother including the time to open the merged output file in the shuffle write time, // because it just opens a single file, so is typically too fast to measure accurately // (see SPARK-3570). val output = shuffleBlockResolver.getDataFile(dep.shuffleId, mapId) val tmp = Utils.tempFileWith(output) try { val blockId = ShuffleBlockId(dep.shuffleId, mapId, IndexShuffleBlockResolver.NOOP_REDUCE_ID) val partitionLengths = sorter.writePartitionedFile(blockId, tmp) shuffleBlockResolver.writeIndexFileAndCommit(dep.shuffleId, mapId, partitionLengths, tmp) mapStatus = MapStatus(blockManager.shuffleServerId, partitionLengths) } finally { if (tmp.exists() && !tmp.delete()) { logError(s"Error while deleting temp file ${tmp.getAbsolutePath}") } } } 27#UnifiedDataAnalytics #SparkAISummit
  • 28. Continued.. / Note: Changes to the format in this file should be kept in sync with // org.apache.spark.network.shuffle.ExternalShuffleBlockResolver#getSortBasedShuffleBlockData(). private[spark] class IndexShuffleBlockResolver( conf: SparkConf, _blockManager: BlockManager = null) extends ShuffleBlockResolver ……….. 28#UnifiedDataAnalytics #SparkAISummit
  • 29. Problems with This ● Rapid downscaling infeasible ○ Scaling down entire nodes hard ● Preemptible VMs & Spot Instances 29#UnifiedDataAnalytics #SparkAISummit
  • 31. Preemptible VMs and Spot instances PVMs Up to 80% cheaper for short-lived instances. Can be pulled at any time. Guaranteed to be removed at least once in 24 hours. Spot is based on Vickrey auction.
  • 32. Stage 1 Stage 2Shuffle
  • 33. How can we fix this? Make intermediate shuffle data external to both the executor and the machine itself 33#UnifiedDataAnalytics #SparkAISummit
  • 34. Where we started class HcfsShuffleWriter[K, V, C] extends ShuffleWriter[K, V] { override def write(records: Iterator[Product2[K, V]]): Unit = { val sorter = new ExternalSorter[K, V, C/V](...) sorter.insertAll(records) val partitionIter = sorter.partitionedIter val hcfsStream = … val countingStream = new CountingOutputStream(hcfsStream) val framedOutput = new FramingOutputStream(countingStream) try { for ((partition, iter) <- partitionIter) { // Write partition to external storage } } finally { framedOutput.closeUnderlying() } } 34#UnifiedDataAnalytics #SparkAISummit
  • 35.
  • 36. Alpha: HDFS not quite ready for prime time ● RPC overhead to HDFS or persistent storage ● Especially poor performance with misaligned partition/block sizes ○ HDFS/GCS/etc different expectations of block size ● Loss of implicit in-memory page cache ● Possibly slowness in cleaning up shuffle files ● Namenode contention when reading shuffle files (HDFS) ○ Added index caching layer to mitigate this ● Additional metadata tracking 36#UnifiedDataAnalytics #SparkAISummit
  • 38.
  • 39. Apache Crail (Incubating) is a high-performance distributed data store designed for fast sharing of ephemeral data in distributed data processing workloads ● Fast ● Heterogeneous ● Modular
  • 40. What about Google Cloud Bigtable? Consistent low latency, high throughput, and scalable wide-column database service.
  • 41. Back to basics - NFS ● Shuffle to Elastifile ○ Cloud based NFS service (scales horizontally) ○ Tailored to random access patterns, small files ○ NFS looks like local FS, but is not. Must be careful when dealing with commit semantics and speculative execution. ● Still a performance hit but factors better than HDFS 41#UnifiedDataAnalytics #SparkAISummit
  • 42. Goal: OSS Disaggregated Shuffle Architecture Kubernetes Cluster Spark Driver Pod Shuffle Offload (WIP) Executor Virtual Machine Group Elastifile Cloud (object) Storage
  • 43. Use the cloud to fix the cloud?
  • 44. DON’T FORGET TO RATE AND REVIEW THE SESSIONS SEARCH SPARK + AI SUMMIT