SlideShare ist ein Scribd-Unternehmen logo
1 von 36
Downloaden Sie, um offline zu lesen
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.

Weitere ähnliche Inhalte

Was ist angesagt?

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
 
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...Databricks
 
Apache Calcite Tutorial - BOSS 21
Apache Calcite Tutorial - BOSS 21Apache Calcite Tutorial - BOSS 21
Apache Calcite Tutorial - BOSS 21Stamatis Zampetakis
 
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
 
Storing 16 Bytes at Scale
Storing 16 Bytes at ScaleStoring 16 Bytes at Scale
Storing 16 Bytes at ScaleFabian Reinartz
 
Apache Iceberg: An Architectural Look Under the Covers
Apache Iceberg: An Architectural Look Under the CoversApache Iceberg: An Architectural Look Under the Covers
Apache Iceberg: An Architectural Look Under the CoversScyllaDB
 
Apache Calcite: One Frontend to Rule Them All
Apache Calcite: One Frontend to Rule Them AllApache Calcite: One Frontend to Rule Them All
Apache Calcite: One Frontend to Rule Them AllMichael Mior
 
The Parquet Format and Performance Optimization Opportunities
The Parquet Format and Performance Optimization OpportunitiesThe Parquet Format and Performance Optimization Opportunities
The Parquet Format and Performance Optimization OpportunitiesDatabricks
 
InfluxDB IOx Tech Talks: Query Engine Design and the Rust-Based DataFusion in...
InfluxDB IOx Tech Talks: Query Engine Design and the Rust-Based DataFusion in...InfluxDB IOx Tech Talks: Query Engine Design and the Rust-Based DataFusion in...
InfluxDB IOx Tech Talks: Query Engine Design and the Rust-Based DataFusion in...InfluxData
 
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
 
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
 
Everyday I'm Shuffling - Tips for Writing Better Spark Programs, Strata San J...
Everyday I'm Shuffling - Tips for Writing Better Spark Programs, Strata San J...Everyday I'm Shuffling - Tips for Writing Better Spark Programs, Strata San J...
Everyday I'm Shuffling - Tips for Writing Better Spark Programs, Strata San J...Databricks
 
Change Data Feed in Delta
Change Data Feed in DeltaChange Data Feed in Delta
Change Data Feed in DeltaDatabricks
 
Apache Calcite (a tutorial given at BOSS '21)
Apache Calcite (a tutorial given at BOSS '21)Apache Calcite (a tutorial given at BOSS '21)
Apache Calcite (a tutorial given at BOSS '21)Julian Hyde
 
Efficient Data Storage for Analytics with Apache Parquet 2.0
Efficient Data Storage for Analytics with Apache Parquet 2.0Efficient Data Storage for Analytics with Apache Parquet 2.0
Efficient Data Storage for Analytics with Apache Parquet 2.0Cloudera, Inc.
 
Spark DataFrames and ML Pipelines
Spark DataFrames and ML PipelinesSpark DataFrames and ML Pipelines
Spark DataFrames and ML PipelinesDatabricks
 
Deep Dive into Spark SQL with Advanced Performance Tuning with Xiao Li & Wenc...
Deep Dive into Spark SQL with Advanced Performance Tuning with Xiao Li & Wenc...Deep Dive into Spark SQL with Advanced Performance Tuning with Xiao Li & Wenc...
Deep Dive into Spark SQL with Advanced Performance Tuning with Xiao Li & Wenc...Databricks
 
Apache Arrow: Open Source Standard Becomes an Enterprise Necessity
Apache Arrow: Open Source Standard Becomes an Enterprise NecessityApache Arrow: Open Source Standard Becomes an Enterprise Necessity
Apache Arrow: Open Source Standard Becomes an Enterprise NecessityWes McKinney
 
Productizing Structured Streaming Jobs
Productizing Structured Streaming JobsProductizing Structured Streaming Jobs
Productizing Structured Streaming JobsDatabricks
 
Don’t optimize my queries, optimize my data!
Don’t optimize my queries, optimize my data!Don’t optimize my queries, optimize my data!
Don’t optimize my queries, optimize my data!Julian Hyde
 

Was ist angesagt? (20)

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
 
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...
 
Apache Calcite Tutorial - BOSS 21
Apache Calcite Tutorial - BOSS 21Apache Calcite Tutorial - BOSS 21
Apache Calcite Tutorial - BOSS 21
 
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
 
Storing 16 Bytes at Scale
Storing 16 Bytes at ScaleStoring 16 Bytes at Scale
Storing 16 Bytes at Scale
 
Apache Iceberg: An Architectural Look Under the Covers
Apache Iceberg: An Architectural Look Under the CoversApache Iceberg: An Architectural Look Under the Covers
Apache Iceberg: An Architectural Look Under the Covers
 
Apache Calcite: One Frontend to Rule Them All
Apache Calcite: One Frontend to Rule Them AllApache Calcite: One Frontend to Rule Them All
Apache Calcite: One Frontend to Rule Them All
 
The Parquet Format and Performance Optimization Opportunities
The Parquet Format and Performance Optimization OpportunitiesThe Parquet Format and Performance Optimization Opportunities
The Parquet Format and Performance Optimization Opportunities
 
InfluxDB IOx Tech Talks: Query Engine Design and the Rust-Based DataFusion in...
InfluxDB IOx Tech Talks: Query Engine Design and the Rust-Based DataFusion in...InfluxDB IOx Tech Talks: Query Engine Design and the Rust-Based DataFusion in...
InfluxDB IOx Tech Talks: Query Engine Design and the Rust-Based DataFusion in...
 
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
 
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
 
Everyday I'm Shuffling - Tips for Writing Better Spark Programs, Strata San J...
Everyday I'm Shuffling - Tips for Writing Better Spark Programs, Strata San J...Everyday I'm Shuffling - Tips for Writing Better Spark Programs, Strata San J...
Everyday I'm Shuffling - Tips for Writing Better Spark Programs, Strata San J...
 
Change Data Feed in Delta
Change Data Feed in DeltaChange Data Feed in Delta
Change Data Feed in Delta
 
Apache Calcite (a tutorial given at BOSS '21)
Apache Calcite (a tutorial given at BOSS '21)Apache Calcite (a tutorial given at BOSS '21)
Apache Calcite (a tutorial given at BOSS '21)
 
Efficient Data Storage for Analytics with Apache Parquet 2.0
Efficient Data Storage for Analytics with Apache Parquet 2.0Efficient Data Storage for Analytics with Apache Parquet 2.0
Efficient Data Storage for Analytics with Apache Parquet 2.0
 
Spark DataFrames and ML Pipelines
Spark DataFrames and ML PipelinesSpark DataFrames and ML Pipelines
Spark DataFrames and ML Pipelines
 
Deep Dive into Spark SQL with Advanced Performance Tuning with Xiao Li & Wenc...
Deep Dive into Spark SQL with Advanced Performance Tuning with Xiao Li & Wenc...Deep Dive into Spark SQL with Advanced Performance Tuning with Xiao Li & Wenc...
Deep Dive into Spark SQL with Advanced Performance Tuning with Xiao Li & Wenc...
 
Apache Arrow: Open Source Standard Becomes an Enterprise Necessity
Apache Arrow: Open Source Standard Becomes an Enterprise NecessityApache Arrow: Open Source Standard Becomes an Enterprise Necessity
Apache Arrow: Open Source Standard Becomes an Enterprise Necessity
 
Productizing Structured Streaming Jobs
Productizing Structured Streaming JobsProductizing Structured Streaming Jobs
Productizing Structured Streaming Jobs
 
Don’t optimize my queries, optimize my data!
Don’t optimize my queries, optimize my data!Don’t optimize my queries, optimize my data!
Don’t optimize my queries, optimize my data!
 

Ähnlich wie 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
 

Ähnlich wie 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
 

Mehr von 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 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
 
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
 
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 DetectionDatabricks
 
Jeeves Grows Up: An AI Chatbot for Performance and Quality
Jeeves Grows Up: An AI Chatbot for Performance and QualityJeeves Grows Up: An AI Chatbot for Performance and Quality
Jeeves Grows Up: An AI Chatbot for Performance and QualityDatabricks
 
Intuitive & Scalable Hyperparameter Tuning with Apache Spark + Fugue
Intuitive & Scalable Hyperparameter Tuning with Apache Spark + FugueIntuitive & Scalable Hyperparameter Tuning with Apache Spark + Fugue
Intuitive & Scalable Hyperparameter Tuning with Apache Spark + FugueDatabricks
 
Infrastructure Agnostic Machine Learning Workload Deployment
Infrastructure Agnostic Machine Learning Workload DeploymentInfrastructure Agnostic Machine Learning Workload Deployment
Infrastructure Agnostic Machine Learning Workload DeploymentDatabricks
 

Mehr von 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
 
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
 
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
 
Jeeves Grows Up: An AI Chatbot for Performance and Quality
Jeeves Grows Up: An AI Chatbot for Performance and QualityJeeves Grows Up: An AI Chatbot for Performance and Quality
Jeeves Grows Up: An AI Chatbot for Performance and Quality
 
Intuitive & Scalable Hyperparameter Tuning with Apache Spark + Fugue
Intuitive & Scalable Hyperparameter Tuning with Apache Spark + FugueIntuitive & Scalable Hyperparameter Tuning with Apache Spark + Fugue
Intuitive & Scalable Hyperparameter Tuning with Apache Spark + Fugue
 
Infrastructure Agnostic Machine Learning Workload Deployment
Infrastructure Agnostic Machine Learning Workload DeploymentInfrastructure Agnostic Machine Learning Workload Deployment
Infrastructure Agnostic Machine Learning Workload Deployment
 

Kürzlich hochgeladen

Defining Constituents, Data Vizzes and Telling a Data Story
Defining Constituents, Data Vizzes and Telling a Data StoryDefining Constituents, Data Vizzes and Telling a Data Story
Defining Constituents, Data Vizzes and Telling a Data StoryJeremy Anderson
 
9654467111 Call Girls In Munirka Hotel And Home Service
9654467111 Call Girls In Munirka Hotel And Home Service9654467111 Call Girls In Munirka Hotel And Home Service
9654467111 Call Girls In Munirka Hotel And Home ServiceSapana Sha
 
原版1:1定制南十字星大学毕业证(SCU毕业证)#文凭成绩单#真实留信学历认证永久存档
原版1:1定制南十字星大学毕业证(SCU毕业证)#文凭成绩单#真实留信学历认证永久存档原版1:1定制南十字星大学毕业证(SCU毕业证)#文凭成绩单#真实留信学历认证永久存档
原版1:1定制南十字星大学毕业证(SCU毕业证)#文凭成绩单#真实留信学历认证永久存档208367051
 
Advanced Machine Learning for Business Professionals
Advanced Machine Learning for Business ProfessionalsAdvanced Machine Learning for Business Professionals
Advanced Machine Learning for Business ProfessionalsVICTOR MAESTRE RAMIREZ
 
Easter Eggs From Star Wars and in cars 1 and 2
Easter Eggs From Star Wars and in cars 1 and 2Easter Eggs From Star Wars and in cars 1 and 2
Easter Eggs From Star Wars and in cars 1 and 217djon017
 
9711147426✨Call In girls Gurgaon Sector 31. SCO 25 escort service
9711147426✨Call In girls Gurgaon Sector 31. SCO 25 escort service9711147426✨Call In girls Gurgaon Sector 31. SCO 25 escort service
9711147426✨Call In girls Gurgaon Sector 31. SCO 25 escort servicejennyeacort
 
Beautiful Sapna Vip Call Girls Hauz Khas 9711199012 Call /Whatsapps
Beautiful Sapna Vip  Call Girls Hauz Khas 9711199012 Call /WhatsappsBeautiful Sapna Vip  Call Girls Hauz Khas 9711199012 Call /Whatsapps
Beautiful Sapna Vip Call Girls Hauz Khas 9711199012 Call /Whatsappssapnasaifi408
 
Kantar AI Summit- Under Embargo till Wednesday, 24th April 2024, 4 PM, IST.pdf
Kantar AI Summit- Under Embargo till Wednesday, 24th April 2024, 4 PM, IST.pdfKantar AI Summit- Under Embargo till Wednesday, 24th April 2024, 4 PM, IST.pdf
Kantar AI Summit- Under Embargo till Wednesday, 24th April 2024, 4 PM, IST.pdfSocial Samosa
 
办理学位证纽约大学毕业证(NYU毕业证书)原版一比一
办理学位证纽约大学毕业证(NYU毕业证书)原版一比一办理学位证纽约大学毕业证(NYU毕业证书)原版一比一
办理学位证纽约大学毕业证(NYU毕业证书)原版一比一fhwihughh
 
GA4 Without Cookies [Measure Camp AMS]
GA4 Without Cookies [Measure Camp AMS]GA4 Without Cookies [Measure Camp AMS]
GA4 Without Cookies [Measure Camp AMS]📊 Markus Baersch
 
Identifying Appropriate Test Statistics Involving Population Mean
Identifying Appropriate Test Statistics Involving Population MeanIdentifying Appropriate Test Statistics Involving Population Mean
Identifying Appropriate Test Statistics Involving Population MeanMYRABACSAFRA2
 
Top 5 Best Data Analytics Courses In Queens
Top 5 Best Data Analytics Courses In QueensTop 5 Best Data Analytics Courses In Queens
Top 5 Best Data Analytics Courses In Queensdataanalyticsqueen03
 
PKS-TGC-1084-630 - Stage 1 Proposal.pptx
PKS-TGC-1084-630 - Stage 1 Proposal.pptxPKS-TGC-1084-630 - Stage 1 Proposal.pptx
PKS-TGC-1084-630 - Stage 1 Proposal.pptxPramod Kumar Srivastava
 
Predicting Salary Using Data Science: A Comprehensive Analysis.pdf
Predicting Salary Using Data Science: A Comprehensive Analysis.pdfPredicting Salary Using Data Science: A Comprehensive Analysis.pdf
Predicting Salary Using Data Science: A Comprehensive Analysis.pdfBoston Institute of Analytics
 
NLP Data Science Project Presentation:Predicting Heart Disease with NLP Data ...
NLP Data Science Project Presentation:Predicting Heart Disease with NLP Data ...NLP Data Science Project Presentation:Predicting Heart Disease with NLP Data ...
NLP Data Science Project Presentation:Predicting Heart Disease with NLP Data ...Boston Institute of Analytics
 
Call Us ➥97111√47426🤳Call Girls in Aerocity (Delhi NCR)
Call Us ➥97111√47426🤳Call Girls in Aerocity (Delhi NCR)Call Us ➥97111√47426🤳Call Girls in Aerocity (Delhi NCR)
Call Us ➥97111√47426🤳Call Girls in Aerocity (Delhi NCR)jennyeacort
 
Multiple time frame trading analysis -brianshannon.pdf
Multiple time frame trading analysis -brianshannon.pdfMultiple time frame trading analysis -brianshannon.pdf
Multiple time frame trading analysis -brianshannon.pdfchwongval
 
Building on a FAIRly Strong Foundation to Connect Academic Research to Transl...
Building on a FAIRly Strong Foundation to Connect Academic Research to Transl...Building on a FAIRly Strong Foundation to Connect Academic Research to Transl...
Building on a FAIRly Strong Foundation to Connect Academic Research to Transl...Jack DiGiovanna
 
20240419 - Measurecamp Amsterdam - SAM.pdf
20240419 - Measurecamp Amsterdam - SAM.pdf20240419 - Measurecamp Amsterdam - SAM.pdf
20240419 - Measurecamp Amsterdam - SAM.pdfHuman37
 

Kürzlich hochgeladen (20)

Defining Constituents, Data Vizzes and Telling a Data Story
Defining Constituents, Data Vizzes and Telling a Data StoryDefining Constituents, Data Vizzes and Telling a Data Story
Defining Constituents, Data Vizzes and Telling a Data Story
 
9654467111 Call Girls In Munirka Hotel And Home Service
9654467111 Call Girls In Munirka Hotel And Home Service9654467111 Call Girls In Munirka Hotel And Home Service
9654467111 Call Girls In Munirka Hotel And Home Service
 
原版1:1定制南十字星大学毕业证(SCU毕业证)#文凭成绩单#真实留信学历认证永久存档
原版1:1定制南十字星大学毕业证(SCU毕业证)#文凭成绩单#真实留信学历认证永久存档原版1:1定制南十字星大学毕业证(SCU毕业证)#文凭成绩单#真实留信学历认证永久存档
原版1:1定制南十字星大学毕业证(SCU毕业证)#文凭成绩单#真实留信学历认证永久存档
 
Deep Generative Learning for All - The Gen AI Hype (Spring 2024)
Deep Generative Learning for All - The Gen AI Hype (Spring 2024)Deep Generative Learning for All - The Gen AI Hype (Spring 2024)
Deep Generative Learning for All - The Gen AI Hype (Spring 2024)
 
Advanced Machine Learning for Business Professionals
Advanced Machine Learning for Business ProfessionalsAdvanced Machine Learning for Business Professionals
Advanced Machine Learning for Business Professionals
 
Easter Eggs From Star Wars and in cars 1 and 2
Easter Eggs From Star Wars and in cars 1 and 2Easter Eggs From Star Wars and in cars 1 and 2
Easter Eggs From Star Wars and in cars 1 and 2
 
9711147426✨Call In girls Gurgaon Sector 31. SCO 25 escort service
9711147426✨Call In girls Gurgaon Sector 31. SCO 25 escort service9711147426✨Call In girls Gurgaon Sector 31. SCO 25 escort service
9711147426✨Call In girls Gurgaon Sector 31. SCO 25 escort service
 
Beautiful Sapna Vip Call Girls Hauz Khas 9711199012 Call /Whatsapps
Beautiful Sapna Vip  Call Girls Hauz Khas 9711199012 Call /WhatsappsBeautiful Sapna Vip  Call Girls Hauz Khas 9711199012 Call /Whatsapps
Beautiful Sapna Vip Call Girls Hauz Khas 9711199012 Call /Whatsapps
 
Kantar AI Summit- Under Embargo till Wednesday, 24th April 2024, 4 PM, IST.pdf
Kantar AI Summit- Under Embargo till Wednesday, 24th April 2024, 4 PM, IST.pdfKantar AI Summit- Under Embargo till Wednesday, 24th April 2024, 4 PM, IST.pdf
Kantar AI Summit- Under Embargo till Wednesday, 24th April 2024, 4 PM, IST.pdf
 
办理学位证纽约大学毕业证(NYU毕业证书)原版一比一
办理学位证纽约大学毕业证(NYU毕业证书)原版一比一办理学位证纽约大学毕业证(NYU毕业证书)原版一比一
办理学位证纽约大学毕业证(NYU毕业证书)原版一比一
 
GA4 Without Cookies [Measure Camp AMS]
GA4 Without Cookies [Measure Camp AMS]GA4 Without Cookies [Measure Camp AMS]
GA4 Without Cookies [Measure Camp AMS]
 
Identifying Appropriate Test Statistics Involving Population Mean
Identifying Appropriate Test Statistics Involving Population MeanIdentifying Appropriate Test Statistics Involving Population Mean
Identifying Appropriate Test Statistics Involving Population Mean
 
Top 5 Best Data Analytics Courses In Queens
Top 5 Best Data Analytics Courses In QueensTop 5 Best Data Analytics Courses In Queens
Top 5 Best Data Analytics Courses In Queens
 
PKS-TGC-1084-630 - Stage 1 Proposal.pptx
PKS-TGC-1084-630 - Stage 1 Proposal.pptxPKS-TGC-1084-630 - Stage 1 Proposal.pptx
PKS-TGC-1084-630 - Stage 1 Proposal.pptx
 
Predicting Salary Using Data Science: A Comprehensive Analysis.pdf
Predicting Salary Using Data Science: A Comprehensive Analysis.pdfPredicting Salary Using Data Science: A Comprehensive Analysis.pdf
Predicting Salary Using Data Science: A Comprehensive Analysis.pdf
 
NLP Data Science Project Presentation:Predicting Heart Disease with NLP Data ...
NLP Data Science Project Presentation:Predicting Heart Disease with NLP Data ...NLP Data Science Project Presentation:Predicting Heart Disease with NLP Data ...
NLP Data Science Project Presentation:Predicting Heart Disease with NLP Data ...
 
Call Us ➥97111√47426🤳Call Girls in Aerocity (Delhi NCR)
Call Us ➥97111√47426🤳Call Girls in Aerocity (Delhi NCR)Call Us ➥97111√47426🤳Call Girls in Aerocity (Delhi NCR)
Call Us ➥97111√47426🤳Call Girls in Aerocity (Delhi NCR)
 
Multiple time frame trading analysis -brianshannon.pdf
Multiple time frame trading analysis -brianshannon.pdfMultiple time frame trading analysis -brianshannon.pdf
Multiple time frame trading analysis -brianshannon.pdf
 
Building on a FAIRly Strong Foundation to Connect Academic Research to Transl...
Building on a FAIRly Strong Foundation to Connect Academic Research to Transl...Building on a FAIRly Strong Foundation to Connect Academic Research to Transl...
Building on a FAIRly Strong Foundation to Connect Academic Research to Transl...
 
20240419 - Measurecamp Amsterdam - SAM.pdf
20240419 - Measurecamp Amsterdam - SAM.pdf20240419 - Measurecamp Amsterdam - SAM.pdf
20240419 - Measurecamp Amsterdam - SAM.pdf
 

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.