SlideShare uma empresa Scribd logo
1 de 36
Baixar para ler offline
Migrating ETL
workflow to Spark at
scale in Pinterest
Daniel Dai, Zirui Li
Pinterest Inc
About Us
• Daniel Dai
• Tech Lead at Pinterest
• PMC member for Apache Hive and Pig
• Zirui Li
• Software Engineer at Pinterest Spark Platform Team
• Focus on building Pinterest in-house Spark platform & functionalities
Agenda
▪ Spark @ Pinterest
▪ Cascading/Scalding to
Spark Conversion
▪ Technical Challenges
▪ Migration Process
▪ Result and Future Plan
Agenda
• Spark @ Pinterest
• Cascading/Scalding to Spark Conversion
• Technical Challenges
• Migration Process
• Result and Future Plan
We Are on Cloud
• We use AWS
• However, we build our
own clusters
• Avoid vendor lockdown
• Timely support by our own team
• We store everything on
S3
• Cost less than HDFS
• HDFS is for temporary storage
S3
EC2
HDFS
Yarn
EC2
HDFS
Yarn
EC2
HDFS
Yarn
Spark Clusters
• We have a couple of Spark clusters
• From several hundred nodes to 1000+ nodes
• Spark only cluster and mixed use cluster
• Cross cluster routing
• R5D instance type for Spark only cluster
• Faster local disk
• High memory to cpu ratio
Spark Versions and Use Cases
• We are running Spark 2.4
• With quite a few internal fixes
• Will migrate to 3.1 this year
• Use cases
• Production use cases
• SparkSQL, PySpark, Spark Native via airflow
• Adhoc use case
• SparkSQL via Querybook, PySpark via Jupyter
Migration Plan
• 40% workloads are already
Spark
• The number is 12% one year ago
• Migration in progress
• Hive to SparkSQL
• Cacading/Scalding to Spark
• Hadoop streaming to Spark pipe
Hive
Cascading/Scalding
Hadoop Streaming
Where are we?
Migration Plan
• Still half workloads are on
Cascading/Scalding
• ETL use cases
• Spark Future
• Query engine: Presto/SparkSQL
• ETL: Spark native
• Machine learning: PySpark
Agenda
• Spark in Pinterest
• Cascading/Scalding to Spark Conversion
• Technical Challenges
• Migration Process
• Result and Future Plan
Cascading
• Simple DAG
• Only 6 different pipes
• Most logic in UDF
• Each – UDF in map
• Every – UDF in reduce
• Java API
Source
Each
GroupBy
Every
Sink
Pattern 1
Source
Each
CoGroup
Every
Sink
Pattern 2
Source
Each
Scalding
• Rich set of operators on top of Cascading
• Operators are very similar to Spark RDD
• Scala API
Migration Path
+
▪ UDF
interface is
private
▪ SQL easy to
migrate to
any engine
Recommend if there’s not
many UDFs
SparkSQL
−
PySpark
▪ Suboptimal
performanc
e, especially
for Python
UDF
▪ Rich Python
libraries
available to
use
+ −
Recommended for Machine
Learning only
+
Native Spark
▪ most structured path to enjoin
rich spark syntax
▪ Work for almost all
Cascading/Scalding
applications
Default & Recommended for
general cases
Spark API
▪ Newer & Recommended API
RDD
Spark Dataframe/Dataset
▪ Most inputs are thrift sequence files
▪ Encode/Decode thrift object to/from
dataframe is slow
Recommended only for non-thrift-
sequence file
▪ More Flexible on handling thrift object
serialization / deserialization
▪ Semantically close to Scalding
▪ Older API
▪ Less performant than Dataframe
Default choice for the conversion
+
−
+
−
Rewrite the
application manually
Reuse most of
Cascading/Scalding
library code
▪ However, avoid
Cascading
specific structure
Automatic tool to help
result validation &
performance tuning
Approach
Translate Cascading
• DAG is usually simple
• Most Cascading pipe has one-to-one mapping to Spark transformation
// val processedInput: RDD[(String, Token)]
// val tokenFreq: RDD[(String, Double)]
val tokenFreqVar = spark.sparkContext.broadcast(tokenFreq.collectAsMap())
val joined = processedInput.map {
t => (t._1, (t._2, tokenFreqVar.value.get(t._1)))
}
Cascading Pipe Spark RDD Operator Note
Each Map side UDF
Every Reduce side UDF
Merge union
CoGroup join/leftOuterJoin/right
OuterJoin/fullOuterJoin
GroupBy GroupBy/GroupByKey secondary sort might be needed
HashJoin Broadcast join no native support in RDD, simulate via broadcast variable
• Complexity is in UDF
UDF Translation
Semantic Difference
Multi-threading
UDF initialization
and cleanup
▪ Do both filtering &
transformation
▪ Java
▪ map + filter
▪ Scala
▪ Multi-thread model
▪ Worst case set
executor-cores=1
▪ Single-thread model
▪ Class with initialization &
cleanup
▪ No init / cleanup hook
▪ mapPartitions to
simulate
Cascading UDF Spark
VS
.mapPartitions{iter =>
// Expensive initialization block
// init block
while (iter.hasNext()) {
val event = iter.next
process(event)
}
// cleanup block
}
Translate Scalding
• Most operator has 1 to 1 mapping to RDD operator
• UDF can be used in Spark without change
Scalding Operator Spark RDD Operator Note
map map
flatMap flatMap
filter filter
filterNot filter Spark does not have filterNot, use filter with negative condition
groupBy groupBy
group groupByKey
groupAll groupBy(t=>1)
...
Agenda
• Spark in Pinterest
• Cascading/Scalding to Spark Conversion
• Technical Challenges
• Migration Process
• Result and Future Plan
Secondary Sort
• Use “repartitionAndSortWithinPartitions” in Spark
• There’s gap in semantics: Use GroupSortedIterator to fill the gap
output = new GroupBy(output, new Fields("user_id"), new Fields("sec_key"));
group key sort key
(2, 2), "apple"
(1, 3), "facebook"
(1, 1), "pinterest"
(1, 2), "twitter"
(3, 2), "google"
input
iterator for key 1:
(1, 1), "pinterest"
(1, 2), "twitter"
(1, 3), "facebook"
iterator for key 2:
(2, 2), "apple"
iterator for key 3:
(3, 2), "google"
Cascading
(1, 1), "pinterest"
(1, 2), "twitter"
(1, 3), "facebook"
(2, 2), "apple"
(3, 2), "google"
Spark
Accumulators
• Spark accumulator is not
accurate
• Stage retry
• Same code run multiple times in different
stage
• Solution
• Deduplicate with stage+partition
• persist
val sc = new SparkContext(conf);
val inputRecords = sc.longAccumulator("Input")
val a = sc.textFile("studenttab10k");
val b = a.map(line => line.split("t"));
val c = b.map { t =>
inputRecords.add(1L)
(t(0), t(1).toInt, t(2).toDouble)
};
val sumScore = c.map(t => t._3).sum()
// c.persist()
c.map { t =>
(t._1, t._3/sumScore)
}.saveAsTextFile("output")
Accumulator Continue
• Retrieve the Accumulator of
the Earliest Stage
• Exception: user intentionally
use the same accumulator in
different stages
NUM_OUTPUT_TOKENS
Stage 14: 168006868318
Stage 21: 336013736636
val sc = new SparkContext(conf);
val inputRecords = sc.longAccumulator("Input")
val input1 = sc.textFile("input1");
val input1_processed = input1.map { t =>
inputRecords.add(1L)
(t(0), (t(1).toInt, t(2).toDouble))
};
val input2 = sc.textFile("input2");
val input2_processed = input2.map { t =>
inputRecords.add(1L)
(t(0), (t(1).toInt, t(2).toDouble))
};
input1_processed.join(input2_processed)
.saveAsTextFile("output")
Accumulator Tab in Spark UI
• SPARK-35197
Profiling
• Visualize frame graph using Nebula
• Realtime
• Ability to segment into stage/task
• Focus on only useful threads
OutputCommitter
• Issue with OutputCommitter
• slow metadata operation
• 503 errors
• Netflix s3committer
• Wrapper for Spark RDD
• s3committer only support old API
Agenda
• Spark @ Pinterest
• Cascading/Scalding to Spark Conversion
• Technical Challenges
• Migration Process
• Result and Future Plan
Automatic Migration Service (AMS)
• A tool to automate majority of migration process
Data Validation
Row counts
Checksum
Comparison
Create a table around
output
SparkSQL UDF
CountAndChecksumUdaf
Doesn’t work for
double/float
Doesn’t work for array if
order is different
−
Input depends
on current
timestamp
There's
random
number
generator in
the code
Rounding
differences
which result
differences in
filter condition
test
Unstable top
result if there's
a tie
Source of Uncertainty
Performance Tuning
Collect runtime
memory/vcore usage
Tuning passed if
criterias meet:
▪ Runtime reduced
▪ Vcore*sec
reduced 20%+
▪ Memory increase
less than 100%
Retry with tuned
memory / vcore if
necessary
Balancing Performance
• Trade-offs
• More executors
• Better performance, but cost more
• Use more cores per executor
• Save on memory, but cost more on cpu
• Use dynamic allocation usually save cost
• Skew won’t cost more with dynamic allocation
• Control parallelism
• spark.default.parallelism for RDD
• spark.sql.shuffle.partitions for dataframe/dataset/SparkSQL
▪ Automatically pick Spark over
Cascading/Scalding during runtime if
condition meets
▪ Data Validation Pass
▪ Performance Optimization Pass
▪ Automatically handle failure with handlers if
applicable
▪ Configuration incorrectness
▪ OutOfMemory
▪ ...
▪ Manual troubleshooting is needed for other
uncaught failures
Failure handling
Automatic Migration
Automatic Migration & Failure Handling
Agenda
• Spark @ Pinterest
• Cascading/Scalding to Spark Conversion
• Technical Challenges
• Migration Process
• Result and Future Plan
Result
• 40% performance improvement
• 47% cost saving on cpu
• Use 33% more memory
Future Plan
• Manual conversion for application still evolving
• Spark backend for legacy application
Feedback
Your feedback is important to us.
Don’t forget to rate and review the sessions.

Mais conteúdo relacionado

Mais procurados

Spark + Parquet In Depth: Spark Summit East Talk by Emily Curtin and Robbie S...
Spark + Parquet In Depth: Spark Summit East Talk by Emily Curtin and Robbie S...Spark + Parquet In Depth: Spark Summit East Talk by Emily Curtin and Robbie S...
Spark + Parquet In Depth: Spark Summit East Talk by Emily Curtin and Robbie S...Spark Summit
 
Apache Iceberg Presentation for the St. Louis Big Data IDEA
Apache Iceberg Presentation for the St. Louis Big Data IDEAApache Iceberg Presentation for the St. Louis Big Data IDEA
Apache Iceberg Presentation for the St. Louis Big Data IDEAAdam Doyle
 
Modularized ETL Writing with Apache Spark
Modularized ETL Writing with Apache SparkModularized ETL Writing with Apache Spark
Modularized ETL Writing with Apache SparkDatabricks
 
A Thorough Comparison of Delta Lake, Iceberg and Hudi
A Thorough Comparison of Delta Lake, Iceberg and HudiA Thorough Comparison of Delta Lake, Iceberg and Hudi
A Thorough Comparison of Delta Lake, Iceberg and HudiDatabricks
 
Apache Iceberg - A Table Format for Hige Analytic Datasets
Apache Iceberg - A Table Format for Hige Analytic DatasetsApache Iceberg - A Table Format for Hige Analytic Datasets
Apache Iceberg - A Table Format for Hige Analytic DatasetsAlluxio, Inc.
 
Radical Speed for SQL Queries on Databricks: Photon Under the Hood
Radical Speed for SQL Queries on Databricks: Photon Under the HoodRadical Speed for SQL Queries on Databricks: Photon Under the Hood
Radical Speed for SQL Queries on Databricks: Photon Under the HoodDatabricks
 
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 SQLDatabricks
 
Performance Analysis of Apache Spark and Presto in Cloud Environments
Performance Analysis of Apache Spark and Presto in Cloud EnvironmentsPerformance Analysis of Apache Spark and Presto in Cloud Environments
Performance Analysis of Apache Spark and Presto in Cloud EnvironmentsDatabricks
 
Parquet performance tuning: the missing guide
Parquet performance tuning: the missing guideParquet performance tuning: the missing guide
Parquet performance tuning: the missing guideRyan Blue
 
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
 
Deep Dive: Memory Management in Apache Spark
Deep Dive: Memory Management in Apache SparkDeep Dive: Memory Management in Apache Spark
Deep Dive: Memory Management in Apache SparkDatabricks
 
Apache Spark Core—Deep Dive—Proper Optimization
Apache Spark Core—Deep Dive—Proper OptimizationApache Spark Core—Deep Dive—Proper Optimization
Apache Spark Core—Deep Dive—Proper OptimizationDatabricks
 
Learn Apache Spark: A Comprehensive Guide
Learn Apache Spark: A Comprehensive GuideLearn Apache Spark: A Comprehensive Guide
Learn Apache Spark: A Comprehensive GuideWhizlabs
 
Intro to Delta Lake
Intro to Delta LakeIntro to Delta Lake
Intro to Delta LakeDatabricks
 
Change Data Feed in Delta
Change Data Feed in DeltaChange Data Feed in Delta
Change Data Feed in DeltaDatabricks
 
A Deep Dive into Spark SQL's Catalyst Optimizer with Yin Huai
A Deep Dive into Spark SQL's Catalyst Optimizer with Yin HuaiA Deep Dive into Spark SQL's Catalyst Optimizer with Yin Huai
A Deep Dive into Spark SQL's Catalyst Optimizer with Yin HuaiDatabricks
 
Apache Spark on K8S Best Practice and Performance in the Cloud
Apache Spark on K8S Best Practice and Performance in the CloudApache Spark on K8S Best Practice and Performance in the Cloud
Apache Spark on K8S Best Practice and Performance in the CloudDatabricks
 
Memory Management in Apache Spark
Memory Management in Apache SparkMemory Management in Apache Spark
Memory Management in Apache SparkDatabricks
 
Making Apache Spark Better with Delta Lake
Making Apache Spark Better with Delta LakeMaking Apache Spark Better with Delta Lake
Making Apache Spark Better with Delta LakeDatabricks
 
Common Strategies for Improving Performance on Your Delta Lakehouse
Common Strategies for Improving Performance on Your Delta LakehouseCommon Strategies for Improving Performance on Your Delta Lakehouse
Common Strategies for Improving Performance on Your Delta LakehouseDatabricks
 

Mais procurados (20)

Spark + Parquet In Depth: Spark Summit East Talk by Emily Curtin and Robbie S...
Spark + Parquet In Depth: Spark Summit East Talk by Emily Curtin and Robbie S...Spark + Parquet In Depth: Spark Summit East Talk by Emily Curtin and Robbie S...
Spark + Parquet In Depth: Spark Summit East Talk by Emily Curtin and Robbie S...
 
Apache Iceberg Presentation for the St. Louis Big Data IDEA
Apache Iceberg Presentation for the St. Louis Big Data IDEAApache Iceberg Presentation for the St. Louis Big Data IDEA
Apache Iceberg Presentation for the St. Louis Big Data IDEA
 
Modularized ETL Writing with Apache Spark
Modularized ETL Writing with Apache SparkModularized ETL Writing with Apache Spark
Modularized ETL Writing with Apache Spark
 
A Thorough Comparison of Delta Lake, Iceberg and Hudi
A Thorough Comparison of Delta Lake, Iceberg and HudiA Thorough Comparison of Delta Lake, Iceberg and Hudi
A Thorough Comparison of Delta Lake, Iceberg and Hudi
 
Apache Iceberg - A Table Format for Hige Analytic Datasets
Apache Iceberg - A Table Format for Hige Analytic DatasetsApache Iceberg - A Table Format for Hige Analytic Datasets
Apache Iceberg - A Table Format for Hige Analytic Datasets
 
Radical Speed for SQL Queries on Databricks: Photon Under the Hood
Radical Speed for SQL Queries on Databricks: Photon Under the HoodRadical Speed for SQL Queries on Databricks: Photon Under the Hood
Radical Speed for SQL Queries on Databricks: Photon Under the Hood
 
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
 
Performance Analysis of Apache Spark and Presto in Cloud Environments
Performance Analysis of Apache Spark and Presto in Cloud EnvironmentsPerformance Analysis of Apache Spark and Presto in Cloud Environments
Performance Analysis of Apache Spark and Presto in Cloud Environments
 
Parquet performance tuning: the missing guide
Parquet performance tuning: the missing guideParquet performance tuning: the missing guide
Parquet performance tuning: the missing guide
 
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...
 
Deep Dive: Memory Management in Apache Spark
Deep Dive: Memory Management in Apache SparkDeep Dive: Memory Management in Apache Spark
Deep Dive: Memory Management in Apache Spark
 
Apache Spark Core—Deep Dive—Proper Optimization
Apache Spark Core—Deep Dive—Proper OptimizationApache Spark Core—Deep Dive—Proper Optimization
Apache Spark Core—Deep Dive—Proper Optimization
 
Learn Apache Spark: A Comprehensive Guide
Learn Apache Spark: A Comprehensive GuideLearn Apache Spark: A Comprehensive Guide
Learn Apache Spark: A Comprehensive Guide
 
Intro to Delta Lake
Intro to Delta LakeIntro to Delta Lake
Intro to Delta Lake
 
Change Data Feed in Delta
Change Data Feed in DeltaChange Data Feed in Delta
Change Data Feed in Delta
 
A Deep Dive into Spark SQL's Catalyst Optimizer with Yin Huai
A Deep Dive into Spark SQL's Catalyst Optimizer with Yin HuaiA Deep Dive into Spark SQL's Catalyst Optimizer with Yin Huai
A Deep Dive into Spark SQL's Catalyst Optimizer with Yin Huai
 
Apache Spark on K8S Best Practice and Performance in the Cloud
Apache Spark on K8S Best Practice and Performance in the CloudApache Spark on K8S Best Practice and Performance in the Cloud
Apache Spark on K8S Best Practice and Performance in the Cloud
 
Memory Management in Apache Spark
Memory Management in Apache SparkMemory Management in Apache Spark
Memory Management in Apache Spark
 
Making Apache Spark Better with Delta Lake
Making Apache Spark Better with Delta LakeMaking Apache Spark Better with Delta Lake
Making Apache Spark Better with Delta Lake
 
Common Strategies for Improving Performance on Your Delta Lakehouse
Common Strategies for Improving Performance on Your Delta LakehouseCommon Strategies for Improving Performance on Your Delta Lakehouse
Common Strategies for Improving Performance on Your Delta Lakehouse
 

Semelhante a Migrating ETL workflow to Spark at scale in Pinterest

Global Big Data Conference Sept 2014 AWS Kinesis Spark Streaming Approximatio...
Global Big Data Conference Sept 2014 AWS Kinesis Spark Streaming Approximatio...Global Big Data Conference Sept 2014 AWS Kinesis Spark Streaming Approximatio...
Global Big Data Conference Sept 2014 AWS Kinesis Spark Streaming Approximatio...Chris Fregly
 
Tuning and Monitoring Deep Learning on Apache Spark
Tuning and Monitoring Deep Learning on Apache SparkTuning and Monitoring Deep Learning on Apache Spark
Tuning and Monitoring Deep Learning on Apache SparkDatabricks
 
Introduction to Apache Spark
Introduction to Apache Spark Introduction to Apache Spark
Introduction to Apache Spark Juan Pedro Moreno
 
Apache Spark Fundamentals
Apache Spark FundamentalsApache Spark Fundamentals
Apache Spark FundamentalsZahra Eskandari
 
Productionizing Spark and the Spark Job Server
Productionizing Spark and the Spark Job ServerProductionizing Spark and the Spark Job Server
Productionizing Spark and the Spark Job ServerEvan Chan
 
700 Updatable Queries Per Second: Spark as a Real-Time Web Service
700 Updatable Queries Per Second: Spark as a Real-Time Web Service700 Updatable Queries Per Second: Spark as a Real-Time Web Service
700 Updatable Queries Per Second: Spark as a Real-Time Web ServiceEvan Chan
 
700 Queries Per Second with Updates: Spark As A Real-Time Web Service
700 Queries Per Second with Updates: Spark As A Real-Time Web Service700 Queries Per Second with Updates: Spark As A Real-Time Web Service
700 Queries Per Second with Updates: Spark As A Real-Time Web ServiceSpark Summit
 
Apache Spark for Beginners
Apache Spark for BeginnersApache Spark for Beginners
Apache Spark for BeginnersAnirudh
 
Productionizing Spark and the REST Job Server- Evan Chan
Productionizing Spark and the REST Job Server- Evan ChanProductionizing Spark and the REST Job Server- Evan Chan
Productionizing Spark and the REST Job Server- Evan ChanSpark Summit
 
Spark Overview and Performance Issues
Spark Overview and Performance IssuesSpark Overview and Performance Issues
Spark Overview and Performance IssuesAntonios Katsarakis
 
Seattle Spark Meetup Mobius CSharp API
Seattle Spark Meetup Mobius CSharp APISeattle Spark Meetup Mobius CSharp API
Seattle Spark Meetup Mobius CSharp APIshareddatamsft
 
Real Time Analytics with Dse
Real Time Analytics with DseReal Time Analytics with Dse
Real Time Analytics with DseDataStax Academy
 
East Bay Java User Group Oct 2014 Spark Streaming Kinesis Machine Learning
 East Bay Java User Group Oct 2014 Spark Streaming Kinesis Machine Learning East Bay Java User Group Oct 2014 Spark Streaming Kinesis Machine Learning
East Bay Java User Group Oct 2014 Spark Streaming Kinesis Machine LearningChris Fregly
 
Paris Data Geek - Spark Streaming
Paris Data Geek - Spark Streaming Paris Data Geek - Spark Streaming
Paris Data Geek - Spark Streaming Djamel Zouaoui
 
Large Scale Data Analytics with Spark and Cassandra on the DSE Platform
Large Scale Data Analytics with Spark and Cassandra on the DSE PlatformLarge Scale Data Analytics with Spark and Cassandra on the DSE Platform
Large Scale Data Analytics with Spark and Cassandra on the DSE PlatformDataStax Academy
 

Semelhante a Migrating ETL workflow to Spark at scale in Pinterest (20)

Global Big Data Conference Sept 2014 AWS Kinesis Spark Streaming Approximatio...
Global Big Data Conference Sept 2014 AWS Kinesis Spark Streaming Approximatio...Global Big Data Conference Sept 2014 AWS Kinesis Spark Streaming Approximatio...
Global Big Data Conference Sept 2014 AWS Kinesis Spark Streaming Approximatio...
 
Tuning and Monitoring Deep Learning on Apache Spark
Tuning and Monitoring Deep Learning on Apache SparkTuning and Monitoring Deep Learning on Apache Spark
Tuning and Monitoring Deep Learning on Apache Spark
 
Introduction to Apache Spark
Introduction to Apache Spark Introduction to Apache Spark
Introduction to Apache Spark
 
Apache Spark Core
Apache Spark CoreApache Spark Core
Apache Spark Core
 
Apache Spark Fundamentals
Apache Spark FundamentalsApache Spark Fundamentals
Apache Spark Fundamentals
 
Intro to Apache Spark
Intro to Apache SparkIntro to Apache Spark
Intro to Apache Spark
 
Intro to Apache Spark
Intro to Apache SparkIntro to Apache Spark
Intro to Apache Spark
 
Productionizing Spark and the Spark Job Server
Productionizing Spark and the Spark Job ServerProductionizing Spark and the Spark Job Server
Productionizing Spark and the Spark Job Server
 
700 Updatable Queries Per Second: Spark as a Real-Time Web Service
700 Updatable Queries Per Second: Spark as a Real-Time Web Service700 Updatable Queries Per Second: Spark as a Real-Time Web Service
700 Updatable Queries Per Second: Spark as a Real-Time Web Service
 
700 Queries Per Second with Updates: Spark As A Real-Time Web Service
700 Queries Per Second with Updates: Spark As A Real-Time Web Service700 Queries Per Second with Updates: Spark As A Real-Time Web Service
700 Queries Per Second with Updates: Spark As A Real-Time Web Service
 
Apache Spark for Beginners
Apache Spark for BeginnersApache Spark for Beginners
Apache Spark for Beginners
 
Productionizing Spark and the REST Job Server- Evan Chan
Productionizing Spark and the REST Job Server- Evan ChanProductionizing Spark and the REST Job Server- Evan Chan
Productionizing Spark and the REST Job Server- Evan Chan
 
Spark Overview and Performance Issues
Spark Overview and Performance IssuesSpark Overview and Performance Issues
Spark Overview and Performance Issues
 
Apache Spark on HDinsight Training
Apache Spark on HDinsight TrainingApache Spark on HDinsight Training
Apache Spark on HDinsight Training
 
Seattle Spark Meetup Mobius CSharp API
Seattle Spark Meetup Mobius CSharp APISeattle Spark Meetup Mobius CSharp API
Seattle Spark Meetup Mobius CSharp API
 
Real Time Analytics with Dse
Real Time Analytics with DseReal Time Analytics with Dse
Real Time Analytics with Dse
 
East Bay Java User Group Oct 2014 Spark Streaming Kinesis Machine Learning
 East Bay Java User Group Oct 2014 Spark Streaming Kinesis Machine Learning East Bay Java User Group Oct 2014 Spark Streaming Kinesis Machine Learning
East Bay Java User Group Oct 2014 Spark Streaming Kinesis Machine Learning
 
Paris Data Geek - Spark Streaming
Paris Data Geek - Spark Streaming Paris Data Geek - Spark Streaming
Paris Data Geek - Spark Streaming
 
Large Scale Data Analytics with Spark and Cassandra on the DSE Platform
Large Scale Data Analytics with Spark and Cassandra on the DSE PlatformLarge Scale Data Analytics with Spark and Cassandra on the DSE Platform
Large Scale Data Analytics with Spark and Cassandra on the DSE Platform
 
Spark Worshop
Spark WorshopSpark Worshop
Spark Worshop
 

Mais de 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
 

Mais de 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
 

Último

FESE Capital Markets Fact Sheet 2024 Q1.pdf
FESE Capital Markets Fact Sheet 2024 Q1.pdfFESE Capital Markets Fact Sheet 2024 Q1.pdf
FESE Capital Markets Fact Sheet 2024 Q1.pdfMarinCaroMartnezBerg
 
Smarteg dropshipping via API with DroFx.pptx
Smarteg dropshipping via API with DroFx.pptxSmarteg dropshipping via API with DroFx.pptx
Smarteg dropshipping via API with DroFx.pptxolyaivanovalion
 
Log Analysis using OSSEC sasoasasasas.pptx
Log Analysis using OSSEC sasoasasasas.pptxLog Analysis using OSSEC sasoasasasas.pptx
Log Analysis using OSSEC sasoasasasas.pptxJohnnyPlasten
 
Ravak dropshipping via API with DroFx.pptx
Ravak dropshipping via API with DroFx.pptxRavak dropshipping via API with DroFx.pptx
Ravak dropshipping via API with DroFx.pptxolyaivanovalion
 
Low Rate Call Girls Bhilai Anika 8250192130 Independent Escort Service Bhilai
Low Rate Call Girls Bhilai Anika 8250192130 Independent Escort Service BhilaiLow Rate Call Girls Bhilai Anika 8250192130 Independent Escort Service Bhilai
Low Rate Call Girls Bhilai Anika 8250192130 Independent Escort Service BhilaiSuhani Kapoor
 
Midocean dropshipping via API with DroFx
Midocean dropshipping via API with DroFxMidocean dropshipping via API with DroFx
Midocean dropshipping via API with DroFxolyaivanovalion
 
Call me @ 9892124323 Cheap Rate Call Girls in Vashi with Real Photo 100% Secure
Call me @ 9892124323  Cheap Rate Call Girls in Vashi with Real Photo 100% SecureCall me @ 9892124323  Cheap Rate Call Girls in Vashi with Real Photo 100% Secure
Call me @ 9892124323 Cheap Rate Call Girls in Vashi with Real Photo 100% SecurePooja Nehwal
 
BabyOno dropshipping via API with DroFx.pptx
BabyOno dropshipping via API with DroFx.pptxBabyOno dropshipping via API with DroFx.pptx
BabyOno dropshipping via API with DroFx.pptxolyaivanovalion
 
VIP Call Girls Service Miyapur Hyderabad Call +91-8250192130
VIP Call Girls Service Miyapur Hyderabad Call +91-8250192130VIP Call Girls Service Miyapur Hyderabad Call +91-8250192130
VIP Call Girls Service Miyapur Hyderabad Call +91-8250192130Suhani Kapoor
 
RA-11058_IRR-COMPRESS Do 198 series of 1998
RA-11058_IRR-COMPRESS Do 198 series of 1998RA-11058_IRR-COMPRESS Do 198 series of 1998
RA-11058_IRR-COMPRESS Do 198 series of 1998YohFuh
 
Industrialised data - the key to AI success.pdf
Industrialised data - the key to AI success.pdfIndustrialised data - the key to AI success.pdf
Industrialised data - the key to AI success.pdfLars Albertsson
 
Unveiling Insights: The Role of a Data Analyst
Unveiling Insights: The Role of a Data AnalystUnveiling Insights: The Role of a Data Analyst
Unveiling Insights: The Role of a Data AnalystSamantha Rae Coolbeth
 
Schema on read is obsolete. Welcome metaprogramming..pdf
Schema on read is obsolete. Welcome metaprogramming..pdfSchema on read is obsolete. Welcome metaprogramming..pdf
Schema on read is obsolete. Welcome metaprogramming..pdfLars Albertsson
 
Halmar dropshipping via API with DroFx
Halmar  dropshipping  via API with DroFxHalmar  dropshipping  via API with DroFx
Halmar dropshipping via API with DroFxolyaivanovalion
 
Generative AI on Enterprise Cloud with NiFi and Milvus
Generative AI on Enterprise Cloud with NiFi and MilvusGenerative AI on Enterprise Cloud with NiFi and Milvus
Generative AI on Enterprise Cloud with NiFi and MilvusTimothy Spann
 
Al Barsha Escorts $#$ O565212860 $#$ Escort Service In Al Barsha
Al Barsha Escorts $#$ O565212860 $#$ Escort Service In Al BarshaAl Barsha Escorts $#$ O565212860 $#$ Escort Service In Al Barsha
Al Barsha Escorts $#$ O565212860 $#$ Escort Service In Al BarshaAroojKhan71
 
Carero dropshipping via API with DroFx.pptx
Carero dropshipping via API with DroFx.pptxCarero dropshipping via API with DroFx.pptx
Carero dropshipping via API with DroFx.pptxolyaivanovalion
 
定制英国白金汉大学毕业证(UCB毕业证书) 成绩单原版一比一
定制英国白金汉大学毕业证(UCB毕业证书)																			成绩单原版一比一定制英国白金汉大学毕业证(UCB毕业证书)																			成绩单原版一比一
定制英国白金汉大学毕业证(UCB毕业证书) 成绩单原版一比一ffjhghh
 
CebaBaby dropshipping via API with DroFX.pptx
CebaBaby dropshipping via API with DroFX.pptxCebaBaby dropshipping via API with DroFX.pptx
CebaBaby dropshipping via API with DroFX.pptxolyaivanovalion
 

Último (20)

FESE Capital Markets Fact Sheet 2024 Q1.pdf
FESE Capital Markets Fact Sheet 2024 Q1.pdfFESE Capital Markets Fact Sheet 2024 Q1.pdf
FESE Capital Markets Fact Sheet 2024 Q1.pdf
 
Smarteg dropshipping via API with DroFx.pptx
Smarteg dropshipping via API with DroFx.pptxSmarteg dropshipping via API with DroFx.pptx
Smarteg dropshipping via API with DroFx.pptx
 
Delhi 99530 vip 56974 Genuine Escort Service Call Girls in Kishangarh
Delhi 99530 vip 56974 Genuine Escort Service Call Girls in  KishangarhDelhi 99530 vip 56974 Genuine Escort Service Call Girls in  Kishangarh
Delhi 99530 vip 56974 Genuine Escort Service Call Girls in Kishangarh
 
Log Analysis using OSSEC sasoasasasas.pptx
Log Analysis using OSSEC sasoasasasas.pptxLog Analysis using OSSEC sasoasasasas.pptx
Log Analysis using OSSEC sasoasasasas.pptx
 
Ravak dropshipping via API with DroFx.pptx
Ravak dropshipping via API with DroFx.pptxRavak dropshipping via API with DroFx.pptx
Ravak dropshipping via API with DroFx.pptx
 
Low Rate Call Girls Bhilai Anika 8250192130 Independent Escort Service Bhilai
Low Rate Call Girls Bhilai Anika 8250192130 Independent Escort Service BhilaiLow Rate Call Girls Bhilai Anika 8250192130 Independent Escort Service Bhilai
Low Rate Call Girls Bhilai Anika 8250192130 Independent Escort Service Bhilai
 
Midocean dropshipping via API with DroFx
Midocean dropshipping via API with DroFxMidocean dropshipping via API with DroFx
Midocean dropshipping via API with DroFx
 
Call me @ 9892124323 Cheap Rate Call Girls in Vashi with Real Photo 100% Secure
Call me @ 9892124323  Cheap Rate Call Girls in Vashi with Real Photo 100% SecureCall me @ 9892124323  Cheap Rate Call Girls in Vashi with Real Photo 100% Secure
Call me @ 9892124323 Cheap Rate Call Girls in Vashi with Real Photo 100% Secure
 
BabyOno dropshipping via API with DroFx.pptx
BabyOno dropshipping via API with DroFx.pptxBabyOno dropshipping via API with DroFx.pptx
BabyOno dropshipping via API with DroFx.pptx
 
VIP Call Girls Service Miyapur Hyderabad Call +91-8250192130
VIP Call Girls Service Miyapur Hyderabad Call +91-8250192130VIP Call Girls Service Miyapur Hyderabad Call +91-8250192130
VIP Call Girls Service Miyapur Hyderabad Call +91-8250192130
 
RA-11058_IRR-COMPRESS Do 198 series of 1998
RA-11058_IRR-COMPRESS Do 198 series of 1998RA-11058_IRR-COMPRESS Do 198 series of 1998
RA-11058_IRR-COMPRESS Do 198 series of 1998
 
Industrialised data - the key to AI success.pdf
Industrialised data - the key to AI success.pdfIndustrialised data - the key to AI success.pdf
Industrialised data - the key to AI success.pdf
 
Unveiling Insights: The Role of a Data Analyst
Unveiling Insights: The Role of a Data AnalystUnveiling Insights: The Role of a Data Analyst
Unveiling Insights: The Role of a Data Analyst
 
Schema on read is obsolete. Welcome metaprogramming..pdf
Schema on read is obsolete. Welcome metaprogramming..pdfSchema on read is obsolete. Welcome metaprogramming..pdf
Schema on read is obsolete. Welcome metaprogramming..pdf
 
Halmar dropshipping via API with DroFx
Halmar  dropshipping  via API with DroFxHalmar  dropshipping  via API with DroFx
Halmar dropshipping via API with DroFx
 
Generative AI on Enterprise Cloud with NiFi and Milvus
Generative AI on Enterprise Cloud with NiFi and MilvusGenerative AI on Enterprise Cloud with NiFi and Milvus
Generative AI on Enterprise Cloud with NiFi and Milvus
 
Al Barsha Escorts $#$ O565212860 $#$ Escort Service In Al Barsha
Al Barsha Escorts $#$ O565212860 $#$ Escort Service In Al BarshaAl Barsha Escorts $#$ O565212860 $#$ Escort Service In Al Barsha
Al Barsha Escorts $#$ O565212860 $#$ Escort Service In Al Barsha
 
Carero dropshipping via API with DroFx.pptx
Carero dropshipping via API with DroFx.pptxCarero dropshipping via API with DroFx.pptx
Carero dropshipping via API with DroFx.pptx
 
定制英国白金汉大学毕业证(UCB毕业证书) 成绩单原版一比一
定制英国白金汉大学毕业证(UCB毕业证书)																			成绩单原版一比一定制英国白金汉大学毕业证(UCB毕业证书)																			成绩单原版一比一
定制英国白金汉大学毕业证(UCB毕业证书) 成绩单原版一比一
 
CebaBaby dropshipping via API with DroFX.pptx
CebaBaby dropshipping via API with DroFX.pptxCebaBaby dropshipping via API with DroFX.pptx
CebaBaby dropshipping via API with DroFX.pptx
 

Migrating ETL workflow to Spark at scale in Pinterest

  • 1. Migrating ETL workflow to Spark at scale in Pinterest Daniel Dai, Zirui Li Pinterest Inc
  • 2. About Us • Daniel Dai • Tech Lead at Pinterest • PMC member for Apache Hive and Pig • Zirui Li • Software Engineer at Pinterest Spark Platform Team • Focus on building Pinterest in-house Spark platform & functionalities
  • 3. Agenda ▪ Spark @ Pinterest ▪ Cascading/Scalding to Spark Conversion ▪ Technical Challenges ▪ Migration Process ▪ Result and Future Plan
  • 4. Agenda • Spark @ Pinterest • Cascading/Scalding to Spark Conversion • Technical Challenges • Migration Process • Result and Future Plan
  • 5. We Are on Cloud • We use AWS • However, we build our own clusters • Avoid vendor lockdown • Timely support by our own team • We store everything on S3 • Cost less than HDFS • HDFS is for temporary storage S3 EC2 HDFS Yarn EC2 HDFS Yarn EC2 HDFS Yarn
  • 6. Spark Clusters • We have a couple of Spark clusters • From several hundred nodes to 1000+ nodes • Spark only cluster and mixed use cluster • Cross cluster routing • R5D instance type for Spark only cluster • Faster local disk • High memory to cpu ratio
  • 7. Spark Versions and Use Cases • We are running Spark 2.4 • With quite a few internal fixes • Will migrate to 3.1 this year • Use cases • Production use cases • SparkSQL, PySpark, Spark Native via airflow • Adhoc use case • SparkSQL via Querybook, PySpark via Jupyter
  • 8. Migration Plan • 40% workloads are already Spark • The number is 12% one year ago • Migration in progress • Hive to SparkSQL • Cacading/Scalding to Spark • Hadoop streaming to Spark pipe Hive Cascading/Scalding Hadoop Streaming Where are we?
  • 9. Migration Plan • Still half workloads are on Cascading/Scalding • ETL use cases • Spark Future • Query engine: Presto/SparkSQL • ETL: Spark native • Machine learning: PySpark
  • 10. Agenda • Spark in Pinterest • Cascading/Scalding to Spark Conversion • Technical Challenges • Migration Process • Result and Future Plan
  • 11. Cascading • Simple DAG • Only 6 different pipes • Most logic in UDF • Each – UDF in map • Every – UDF in reduce • Java API Source Each GroupBy Every Sink Pattern 1 Source Each CoGroup Every Sink Pattern 2 Source Each
  • 12. Scalding • Rich set of operators on top of Cascading • Operators are very similar to Spark RDD • Scala API
  • 13. Migration Path + ▪ UDF interface is private ▪ SQL easy to migrate to any engine Recommend if there’s not many UDFs SparkSQL − PySpark ▪ Suboptimal performanc e, especially for Python UDF ▪ Rich Python libraries available to use + − Recommended for Machine Learning only + Native Spark ▪ most structured path to enjoin rich spark syntax ▪ Work for almost all Cascading/Scalding applications Default & Recommended for general cases
  • 14. Spark API ▪ Newer & Recommended API RDD Spark Dataframe/Dataset ▪ Most inputs are thrift sequence files ▪ Encode/Decode thrift object to/from dataframe is slow Recommended only for non-thrift- sequence file ▪ More Flexible on handling thrift object serialization / deserialization ▪ Semantically close to Scalding ▪ Older API ▪ Less performant than Dataframe Default choice for the conversion + − + −
  • 15. Rewrite the application manually Reuse most of Cascading/Scalding library code ▪ However, avoid Cascading specific structure Automatic tool to help result validation & performance tuning Approach
  • 16. Translate Cascading • DAG is usually simple • Most Cascading pipe has one-to-one mapping to Spark transformation // val processedInput: RDD[(String, Token)] // val tokenFreq: RDD[(String, Double)] val tokenFreqVar = spark.sparkContext.broadcast(tokenFreq.collectAsMap()) val joined = processedInput.map { t => (t._1, (t._2, tokenFreqVar.value.get(t._1))) } Cascading Pipe Spark RDD Operator Note Each Map side UDF Every Reduce side UDF Merge union CoGroup join/leftOuterJoin/right OuterJoin/fullOuterJoin GroupBy GroupBy/GroupByKey secondary sort might be needed HashJoin Broadcast join no native support in RDD, simulate via broadcast variable • Complexity is in UDF
  • 17. UDF Translation Semantic Difference Multi-threading UDF initialization and cleanup ▪ Do both filtering & transformation ▪ Java ▪ map + filter ▪ Scala ▪ Multi-thread model ▪ Worst case set executor-cores=1 ▪ Single-thread model ▪ Class with initialization & cleanup ▪ No init / cleanup hook ▪ mapPartitions to simulate Cascading UDF Spark VS .mapPartitions{iter => // Expensive initialization block // init block while (iter.hasNext()) { val event = iter.next process(event) } // cleanup block }
  • 18. Translate Scalding • Most operator has 1 to 1 mapping to RDD operator • UDF can be used in Spark without change Scalding Operator Spark RDD Operator Note map map flatMap flatMap filter filter filterNot filter Spark does not have filterNot, use filter with negative condition groupBy groupBy group groupByKey groupAll groupBy(t=>1) ...
  • 19. Agenda • Spark in Pinterest • Cascading/Scalding to Spark Conversion • Technical Challenges • Migration Process • Result and Future Plan
  • 20. Secondary Sort • Use “repartitionAndSortWithinPartitions” in Spark • There’s gap in semantics: Use GroupSortedIterator to fill the gap output = new GroupBy(output, new Fields("user_id"), new Fields("sec_key")); group key sort key (2, 2), "apple" (1, 3), "facebook" (1, 1), "pinterest" (1, 2), "twitter" (3, 2), "google" input iterator for key 1: (1, 1), "pinterest" (1, 2), "twitter" (1, 3), "facebook" iterator for key 2: (2, 2), "apple" iterator for key 3: (3, 2), "google" Cascading (1, 1), "pinterest" (1, 2), "twitter" (1, 3), "facebook" (2, 2), "apple" (3, 2), "google" Spark
  • 21. Accumulators • Spark accumulator is not accurate • Stage retry • Same code run multiple times in different stage • Solution • Deduplicate with stage+partition • persist val sc = new SparkContext(conf); val inputRecords = sc.longAccumulator("Input") val a = sc.textFile("studenttab10k"); val b = a.map(line => line.split("t")); val c = b.map { t => inputRecords.add(1L) (t(0), t(1).toInt, t(2).toDouble) }; val sumScore = c.map(t => t._3).sum() // c.persist() c.map { t => (t._1, t._3/sumScore) }.saveAsTextFile("output")
  • 22. Accumulator Continue • Retrieve the Accumulator of the Earliest Stage • Exception: user intentionally use the same accumulator in different stages NUM_OUTPUT_TOKENS Stage 14: 168006868318 Stage 21: 336013736636 val sc = new SparkContext(conf); val inputRecords = sc.longAccumulator("Input") val input1 = sc.textFile("input1"); val input1_processed = input1.map { t => inputRecords.add(1L) (t(0), (t(1).toInt, t(2).toDouble)) }; val input2 = sc.textFile("input2"); val input2_processed = input2.map { t => inputRecords.add(1L) (t(0), (t(1).toInt, t(2).toDouble)) }; input1_processed.join(input2_processed) .saveAsTextFile("output")
  • 23. Accumulator Tab in Spark UI • SPARK-35197
  • 24. Profiling • Visualize frame graph using Nebula • Realtime • Ability to segment into stage/task • Focus on only useful threads
  • 25. OutputCommitter • Issue with OutputCommitter • slow metadata operation • 503 errors • Netflix s3committer • Wrapper for Spark RDD • s3committer only support old API
  • 26. Agenda • Spark @ Pinterest • Cascading/Scalding to Spark Conversion • Technical Challenges • Migration Process • Result and Future Plan
  • 27. Automatic Migration Service (AMS) • A tool to automate majority of migration process
  • 28. Data Validation Row counts Checksum Comparison Create a table around output SparkSQL UDF CountAndChecksumUdaf Doesn’t work for double/float Doesn’t work for array if order is different −
  • 29. Input depends on current timestamp There's random number generator in the code Rounding differences which result differences in filter condition test Unstable top result if there's a tie Source of Uncertainty
  • 30. Performance Tuning Collect runtime memory/vcore usage Tuning passed if criterias meet: ▪ Runtime reduced ▪ Vcore*sec reduced 20%+ ▪ Memory increase less than 100% Retry with tuned memory / vcore if necessary
  • 31. Balancing Performance • Trade-offs • More executors • Better performance, but cost more • Use more cores per executor • Save on memory, but cost more on cpu • Use dynamic allocation usually save cost • Skew won’t cost more with dynamic allocation • Control parallelism • spark.default.parallelism for RDD • spark.sql.shuffle.partitions for dataframe/dataset/SparkSQL
  • 32. ▪ Automatically pick Spark over Cascading/Scalding during runtime if condition meets ▪ Data Validation Pass ▪ Performance Optimization Pass ▪ Automatically handle failure with handlers if applicable ▪ Configuration incorrectness ▪ OutOfMemory ▪ ... ▪ Manual troubleshooting is needed for other uncaught failures Failure handling Automatic Migration Automatic Migration & Failure Handling
  • 33. Agenda • Spark @ Pinterest • Cascading/Scalding to Spark Conversion • Technical Challenges • Migration Process • Result and Future Plan
  • 34. Result • 40% performance improvement • 47% cost saving on cpu • Use 33% more memory
  • 35. Future Plan • Manual conversion for application still evolving • Spark backend for legacy application
  • 36. Feedback Your feedback is important to us. Don’t forget to rate and review the sessions.