SlideShare une entreprise Scribd logo
1  sur  60
Télécharger pour lire hors ligne
Runtime Internals
连城
lian@databricks.com
cheng.lian@ciilabs.org
What is
• A fast and general engine for large-scale data
processing
• An open source implementation of Resilient
Distributed Datasets (RDD)
• Has an advanced DAG execution engine that
supports cyclic data flow and in-memory
computing
Why
• Fast
– Run machine learning like iterative programs up to
100x faster than Hadoop MapReduce in memory,
or 10x faster on disk
– Run HiveQL compatible queries 100x faster than
Hive (with Shark/Spark SQL)
Why
• Easy to use
– Fluent Scala/Java/Python API
– Interactive shell
– 2-5x less code (than Hadoop MapReduce)
Why
• Easy to use
– Fluent Scala/Java/Python API
– Interactive shell
– 2-5x less code (than Hadoop MapReduce)
sc.textFile("hdfs://...")
.flatMap(_.split(" "))
.map(_ -> 1)
.reduceByKey(_ + _)
.collectAsMap()
Why
• Easy to use
– Fluent Scala/Java/Python API
– Interactive shell
– 2-5x less code (than Hadoop MapReduce)
sc.textFile("hdfs://...")
.flatMap(_.split(" "))
.map(_ -> 1)
.reduceByKey(_ + _)
.collectAsMap()
Can you write down
WordCount
in 30 seconds with
Hadoop MapReduce?
Why
• Unified big data pipeline for:
– Batch/Interactive (Spark Core vs MR/Tez)
– SQL (Shark/Spark SQL vs Hive)
– Streaming (Spark Streaming vs Storm)
– Machine learning (MLlib vs Mahout)
– Graph (GraphX vs Giraph)
A Bigger Picture...
An Even Bigger Picture...
• Components of the Spark stack focus on big
data analysis and are compatible with existing
Hadoop storage systems
• Users don’t need to suffer expensive ETL cost
to use the Spark stack
“One Stack To Rule Them All”
• Well, mostly :-)
• And don't forget Shark/Spark SQL vs Hive
Resilient Distributed Datasets
Resilient Distributed Datasets
• Conceptually, RDDs can be roughly viewed as
partitioned, locality aware distributed vectors
• An RDD…
– either points to a direct data source
– or applies some transformation to its parent RDD(s)
to generate new data elements
– Computation can be represented by lazy evaluated
lineage DAGs composed by connected RDDs
Resilient Distributed Datasets
• Frequently used RDDs can be materialized and
cached in-memory to accelerate computation
• Spark scheduler takes data locality into
account
The In-Memory Magic
• “In fact, one study [1] analyzed the access
patterns in the Hive warehouses at Facebook
and discovered that for the vast majority (96%)
of jobs, the entire inputs could fit into a
fraction of the cluster’s total memory.”
[1] G. Ananthanarayanan, A. Ghodsi, S. Shenker, and I. Stoica. Disk-locality in
datacenter computing considered irrelevant. In HotOS ’11, 2011.
The In-Memory Magic
• Without cache
– Elements are accessed in an iterator-based
streaming style
– One element a time, no bulk copy
– Space complexity is almost O(1) when there’s only
narrow dependencies
The In-Memory Magic
• With cache
– One block per RDD partition
– LRU cache eviction
– Locality aware
– Evicted blocks can be recomputed in parallel with
the help of RDD lineage DAG
Play with Error Logs
sc.textFile("hdfs://<input>")
.filter(_.startsWith("ERROR"))
.map(_.split(" ")(1))
.saveAsTextFile("hdfs://<output>")
Step by Step
The RDD.saveAsTextFile() action triggers a job. Tasks are started on
scheduled executors.
The task pulls data lazily from the final RDD (the MappedRDD here).
Step by Step
A record (text line) is pulled out from HDFS...
Step by Step
... into a HadoopRDD
Step by Step
Then filtered with the FilteredRDD
Step by Step
Oops, not a match
Step by Step
Another record is pulled out...
Step by Step
Filtered again...
Step by Step
Passed to the MappedRDD for next transformation
Step by Step
Transformed by the MappedRDD (the error message is extracted)
Step by Step
Saves the message to HDFS
Step by Step
A Synthesized Iterator
new Iterator[String] {
private var head: String = _
private var headDefined: Boolean = false
def hasNext: Boolean = headDefined || {
do {
try head = readOneLineFromHDFS(...) // (1) read from HDFS
catch {
case _: EOFException => return false
}
} while (!head.startsWith("ERROR")) // (2) filter closure
true
}
def next: String = if (hasNext) {
headDefined = false
head.split(" ")(1) // (3) map closure
} else {
throw new NoSuchElementException("...")
}
}
Constant Space
Complexity!
If Cached...
val cached = sc
.textFile("hdfs://<input>")
.filter(_.startsWith("ERROR"))
.cache()
cached
.map(_.split(" ")(1))
.saveAsTextFile("hdfs://<output>")
If Cached...
All filtered error messages are cached in memory before being passed to the
next RDD. LRU cache eviction is applied when memory is insufficient
But… I Don’t Have Enough
Memory To Cache All Data
Don’t Worry
• In most cases, you only need to cache a fraction of
hot data extracted/transformed from the whole input
dataset
• Spark performance downgrades linearly & gracefully
when memory decreases
How Does The Job Get
Distributed?
Spark Cluster Overview
Driver, Cluster Manager,
Worker, Executor, ...
So... Where THE HELL
Does My Code Run?
Well... It Depends
Driver and SparkContext
import org.apache.spark.SparkContext
import org.apache.spark.SparkContext._
val sc = new SparkContext(...)
• A SparkContext initializes the application driver,
the latter then registers the application to the cluster
manager, and gets a list of executors
• Since then, the driver takes full responsibilities
WordCount Revisited
val lines = sc.textFile("input")
val words = lines.flatMap(_.split(" "))
val ones = words.map(_ -> 1)
val counts = ones.reduceByKey(_ + _)
val result = counts.collectAsMap()
• RDD lineage DAG is built on driver side with:
– Data source RDD(s)
– Transformation RDD(s), which are created by
transformations
WordCount Revisited
val lines = sc.textFile("input")
val words = lines.flatMap(_.split(" "))
val ones = words.map(_ -> 1)
val counts = ones.reduceByKey(_ + _)
val result = counts.collectAsMap()
• Once an action is triggered on driver side, a
job is submitted to the DAG scheduler of the
driver
WordCount Revisited
• DAG scheduler cuts the DAG into stages and
turns each partition of a stage into a single task.
• DAG scheduler decides what to run
WordCount Revisited
• Tasks are then scheduled to executors by
driver side task scheduler according to
resource and locality constraints
• Task scheduler decides where to run
WordCount Revisited
val lines = sc.textFile("input")
val words = lines.flatMap(_.split(" "))
val ones = words.map(_ -> 1)
val counts = ones.reduceByKey(_ + _)
val result = counts.collectAsMap()
• Within a task, the lineage DAG of
corresponding stage is serialized together with
closures of transformations, then sent to and
executed on scheduled executors
WordCount Revisited
val lines = sc.textFile("input")
val words = lines.flatMap(_.split(" "))
val ones = words.map(_ -> 1)
val counts = ones.reduceByKey(_ + _)
val result = counts.collectAsMap()
• The reduceByKey transformation introduces
in a shuffle
• Shuffle outputs are written to local FS on the
mapper side, then downloaded by reducers
WordCount Revisited
val lines = sc.textFile("input")
val words = lines.flatMap(_.split(" "))
val ones = words.map(_ -> 1)
val counts = ones.reduceByKey(_ + _)
val result = counts.collectAsMap()
• ReduceByKey automatically combines values
within a single partition locally on the mapper
side and then reduce them globally on the
reducer side.
WordCount Revisited
val lines = sc.textFile("input")
val words = lines.flatMap(_.split(" "))
val ones = words.map(_ -> 1)
val counts = ones.reduceByKey(_ + _)
val result = counts.collectAsMap()
• At last, results of the action are sent back to
the driver, then the job finishes.
What About Parallelism?
How To Control Parallelism?
• Can be specified in a number of ways
– RDD partition number
• sc.textFile("input", minSplits = 10)
• sc.parallelize(1 to 10000, numSlices = 10)
– Mapper side parallelism
• Usually inherited from parent RDD(s)
– Reducer side parallelism
• rdd.reduceByKey(_ + _, numPartitions = 10)
• rdd.reduceByKey(partitioner = p, _ + _)
• …
How To Control Parallelism?
• “Zoom in/out”
– RDD.repartition(numPartitions: Int)
– RDD.coalesce(
numPartitions: Int,
shuffle: Boolean)
A Trap of Partitions
sc.textFile("input", minSplits = 2)
.map { line =>
val Array(key, value) = line.split(",")
key.toInt -> value.toInt
}
.reduceByKey(_ + _)
.saveAsText("output")
• Run in local mode
• 60+GB input file
A Trap of Partitions
sc.textFile("input", minSplits = 2)
.map { line =>
val Array(key, value) = line.split(",")
key.toInt -> value.toInt
}
.reduceByKey(_ + _)
.saveAsText("output") Hmm...
Pretty cute?
A Trap of Partitions
sc.textFile("input", minSplits = 2)
.map { line =>
val Array(key, value) = line.split(",")
key.toInt -> value.toInt
}
.reduceByKey(_ + _)
.saveAsText("output")
• Actually, this may
generate 4,000,000
temporary files...
• Why 4M? (Hint: 2K2)
!!!
A Trap of Partitions
sc.textFile("input", minSplits = 2)
.map { line =>
val Array(key, value) = line.split(",")
key.toInt -> value.toInt
}
.reduceByKey(_ + _)
.saveAsText("output")
• 1 partition per HDFS split
• Similar to mapper task number in Hadoop
MapReduce, the minSplits parameter is
only a hint for split number
A Trap of Partitions
sc.textFile("input", minSplits = 2)
.map { line =>
val Array(key, value) = line.split(",")
key.toInt -> value.toInt
}
.reduceByKey(_ + _)
.saveAsText("output")
• Actual split number is also controlled by some
other properties like minSplitSize and default
FS block size
A Trap of Partitions
sc.textFile("input", minSplits = 2)
.map { line =>
val Array(key, value) = line.split(",")
key.toInt -> value.toInt
}
.reduceByKey(_ + _)
.saveAsText("output")
• In this case, the final split size equals to local
FS block size, which is 32MB by default, and
60GB / 32MB ≈ 2K
• ReduceByKey generates 2K2 shuffle outputs
A Trap of Partitions
sc.textFile("input", minSplits = 2).coalesce(2)
.map { line =>
val Array(key, value) = line.split(",")
key.toInt -> value.toInt
}
.reduceByKey(_ + _)
.saveAsText("output")
• Use RDD.coalesce() to control partition
number precisely.
Summary
• In-memory magic
– Iterator-based streaming style data access
– Usually you don’t need to cache all dataset
– When memory is not enough, Spark performance
downgrades linearly & gracefully
Summary
• Distributed job
– Locality aware
– Resource aware
• Parallelism
– Reasonable default behavior
– Can be finely tuned
References
• Resilient distributed datasets: a fault-tolerant
abstraction for in-memory cluster computing
• An Architecture for Fast and General Data
Processing on Large Clusters
• Spark Internals (video, slides)
• Shark: SQL and Rich Analytics at Scale
• Spark Summit 2013
Q&A

Contenu connexe

Tendances

Apache Spark RDDs
Apache Spark RDDsApache Spark RDDs
Apache Spark RDDsDean Chen
 
Transformations and actions a visual guide training
Transformations and actions a visual guide trainingTransformations and actions a visual guide training
Transformations and actions a visual guide trainingSpark Summit
 
Introduction to Spark Internals
Introduction to Spark InternalsIntroduction to Spark Internals
Introduction to Spark InternalsPietro Michiardi
 
Spark & Spark Streaming Internals - Nov 15 (1)
Spark & Spark Streaming Internals - Nov 15 (1)Spark & Spark Streaming Internals - Nov 15 (1)
Spark & Spark Streaming Internals - Nov 15 (1)Akhil Das
 
Introduction to Apache Spark
Introduction to Apache SparkIntroduction to Apache Spark
Introduction to Apache SparkDatio Big Data
 
Apache Spark Internals
Apache Spark InternalsApache Spark Internals
Apache Spark InternalsKnoldus Inc.
 
Deep Dive with Spark Streaming - Tathagata Das - Spark Meetup 2013-06-17
Deep Dive with Spark Streaming - Tathagata  Das - Spark Meetup 2013-06-17Deep Dive with Spark Streaming - Tathagata  Das - Spark Meetup 2013-06-17
Deep Dive with Spark Streaming - Tathagata Das - Spark Meetup 2013-06-17spark-project
 
Apache Spark Introduction and Resilient Distributed Dataset basics and deep dive
Apache Spark Introduction and Resilient Distributed Dataset basics and deep diveApache Spark Introduction and Resilient Distributed Dataset basics and deep dive
Apache Spark Introduction and Resilient Distributed Dataset basics and deep diveSachin Aggarwal
 
20130912 YTC_Reynold Xin_Spark and Shark
20130912 YTC_Reynold Xin_Spark and Shark20130912 YTC_Reynold Xin_Spark and Shark
20130912 YTC_Reynold Xin_Spark and SharkYahooTechConference
 
Debugging & Tuning in Spark
Debugging & Tuning in SparkDebugging & Tuning in Spark
Debugging & Tuning in SparkShiao-An Yuan
 
Apache Spark Introduction
Apache Spark IntroductionApache Spark Introduction
Apache Spark Introductionsudhakara st
 
Spark overview
Spark overviewSpark overview
Spark overviewLisa Hua
 
A deeper-understanding-of-spark-internals
A deeper-understanding-of-spark-internalsA deeper-understanding-of-spark-internals
A deeper-understanding-of-spark-internalsCheng Min Chi
 

Tendances (20)

Apache Spark with Scala
Apache Spark with ScalaApache Spark with Scala
Apache Spark with Scala
 
Apache Spark RDDs
Apache Spark RDDsApache Spark RDDs
Apache Spark RDDs
 
Apache Spark RDD 101
Apache Spark RDD 101Apache Spark RDD 101
Apache Spark RDD 101
 
Transformations and actions a visual guide training
Transformations and actions a visual guide trainingTransformations and actions a visual guide training
Transformations and actions a visual guide training
 
Introduction to Spark Internals
Introduction to Spark InternalsIntroduction to Spark Internals
Introduction to Spark Internals
 
Spark & Spark Streaming Internals - Nov 15 (1)
Spark & Spark Streaming Internals - Nov 15 (1)Spark & Spark Streaming Internals - Nov 15 (1)
Spark & Spark Streaming Internals - Nov 15 (1)
 
Apache Spark
Apache Spark Apache Spark
Apache Spark
 
Apache Spark & Streaming
Apache Spark & StreamingApache Spark & Streaming
Apache Spark & Streaming
 
Apache spark core
Apache spark coreApache spark core
Apache spark core
 
Introduction to Apache Spark
Introduction to Apache SparkIntroduction to Apache Spark
Introduction to Apache Spark
 
Apache Spark Internals
Apache Spark InternalsApache Spark Internals
Apache Spark Internals
 
Deep Dive with Spark Streaming - Tathagata Das - Spark Meetup 2013-06-17
Deep Dive with Spark Streaming - Tathagata  Das - Spark Meetup 2013-06-17Deep Dive with Spark Streaming - Tathagata  Das - Spark Meetup 2013-06-17
Deep Dive with Spark Streaming - Tathagata Das - Spark Meetup 2013-06-17
 
Apache Spark Introduction and Resilient Distributed Dataset basics and deep dive
Apache Spark Introduction and Resilient Distributed Dataset basics and deep diveApache Spark Introduction and Resilient Distributed Dataset basics and deep dive
Apache Spark Introduction and Resilient Distributed Dataset basics and deep dive
 
20130912 YTC_Reynold Xin_Spark and Shark
20130912 YTC_Reynold Xin_Spark and Shark20130912 YTC_Reynold Xin_Spark and Shark
20130912 YTC_Reynold Xin_Spark and Shark
 
Debugging & Tuning in Spark
Debugging & Tuning in SparkDebugging & Tuning in Spark
Debugging & Tuning in Spark
 
Apache Spark Introduction
Apache Spark IntroductionApache Spark Introduction
Apache Spark Introduction
 
Spark overview
Spark overviewSpark overview
Spark overview
 
A deeper-understanding-of-spark-internals
A deeper-understanding-of-spark-internalsA deeper-understanding-of-spark-internals
A deeper-understanding-of-spark-internals
 
Intro to Apache Spark
Intro to Apache SparkIntro to Apache Spark
Intro to Apache Spark
 
Apache Spark Architecture
Apache Spark ArchitectureApache Spark Architecture
Apache Spark Architecture
 

En vedette

Spark Internals - Hadoop Source Code Reading #16 in Japan
Spark Internals - Hadoop Source Code Reading #16 in JapanSpark Internals - Hadoop Source Code Reading #16 in Japan
Spark Internals - Hadoop Source Code Reading #16 in JapanTaro L. Saito
 
Python and Bigdata - An Introduction to Spark (PySpark)
Python and Bigdata -  An Introduction to Spark (PySpark)Python and Bigdata -  An Introduction to Spark (PySpark)
Python and Bigdata - An Introduction to Spark (PySpark)hiteshnd
 
A deeper-understanding-of-spark-internals-aaron-davidson
A deeper-understanding-of-spark-internals-aaron-davidsonA deeper-understanding-of-spark-internals-aaron-davidson
A deeper-understanding-of-spark-internals-aaron-davidsonCheng Min Chi
 
Effective Programming In Scala
Effective Programming In ScalaEffective Programming In Scala
Effective Programming In ScalaHarsh Sharma
 
Type Parameterization
Type ParameterizationType Parameterization
Type ParameterizationKnoldus Inc.
 
Simple Scala DSLs
Simple Scala DSLsSimple Scala DSLs
Simple Scala DSLslinxbetter
 
A Deeper Understanding of Spark Internals (Hadoop Conference Japan 2014)
A Deeper Understanding of Spark Internals (Hadoop Conference Japan 2014)A Deeper Understanding of Spark Internals (Hadoop Conference Japan 2014)
A Deeper Understanding of Spark Internals (Hadoop Conference Japan 2014)Hadoop / Spark Conference Japan
 
So various polymorphism in Scala
So various polymorphism in ScalaSo various polymorphism in Scala
So various polymorphism in Scalab0ris_1
 
Unikernels: in search of a killer app and a killer ecosystem
Unikernels: in search of a killer app and a killer ecosystemUnikernels: in search of a killer app and a killer ecosystem
Unikernels: in search of a killer app and a killer ecosystemrhatr
 
Spark fundamentals i (bd095 en) version #1: updated: april 2015
Spark fundamentals i (bd095 en) version #1: updated: april 2015Spark fundamentals i (bd095 en) version #1: updated: april 2015
Spark fundamentals i (bd095 en) version #1: updated: april 2015Ashutosh Sonaliya
 
Real-World Scala Design Patterns
Real-World Scala Design PatternsReal-World Scala Design Patterns
Real-World Scala Design PatternsNLJUG
 
Full stack analytics with Hadoop 2
Full stack analytics with Hadoop 2Full stack analytics with Hadoop 2
Full stack analytics with Hadoop 2Gabriele Modena
 
Type Checking Scala Spark Datasets: Dataset Transforms
Type Checking Scala Spark Datasets: Dataset TransformsType Checking Scala Spark Datasets: Dataset Transforms
Type Checking Scala Spark Datasets: Dataset TransformsJohn Nestor
 
Scala Implicits - Not to be feared
Scala Implicits - Not to be fearedScala Implicits - Not to be feared
Scala Implicits - Not to be fearedDerek Wyatt
 
臺灣高中數學講義 - 第一冊 - 數與式
臺灣高中數學講義 - 第一冊 - 數與式臺灣高中數學講義 - 第一冊 - 數與式
臺灣高中數學講義 - 第一冊 - 數與式Xuan-Chao Huang
 
New Analytics Toolbox DevNexus 2015
New Analytics Toolbox DevNexus 2015New Analytics Toolbox DevNexus 2015
New Analytics Toolbox DevNexus 2015Robbie Strickland
 
Think Like Spark: Some Spark Concepts and a Use Case
Think Like Spark: Some Spark Concepts and a Use CaseThink Like Spark: Some Spark Concepts and a Use Case
Think Like Spark: Some Spark Concepts and a Use CaseRachel Warren
 

En vedette (20)

Spark Internals - Hadoop Source Code Reading #16 in Japan
Spark Internals - Hadoop Source Code Reading #16 in JapanSpark Internals - Hadoop Source Code Reading #16 in Japan
Spark Internals - Hadoop Source Code Reading #16 in Japan
 
Python and Bigdata - An Introduction to Spark (PySpark)
Python and Bigdata -  An Introduction to Spark (PySpark)Python and Bigdata -  An Introduction to Spark (PySpark)
Python and Bigdata - An Introduction to Spark (PySpark)
 
A deeper-understanding-of-spark-internals-aaron-davidson
A deeper-understanding-of-spark-internals-aaron-davidsonA deeper-understanding-of-spark-internals-aaron-davidson
A deeper-understanding-of-spark-internals-aaron-davidson
 
Effective Programming In Scala
Effective Programming In ScalaEffective Programming In Scala
Effective Programming In Scala
 
Type Parameterization
Type ParameterizationType Parameterization
Type Parameterization
 
Scala collections
Scala collectionsScala collections
Scala collections
 
Simple Scala DSLs
Simple Scala DSLsSimple Scala DSLs
Simple Scala DSLs
 
A Deeper Understanding of Spark Internals (Hadoop Conference Japan 2014)
A Deeper Understanding of Spark Internals (Hadoop Conference Japan 2014)A Deeper Understanding of Spark Internals (Hadoop Conference Japan 2014)
A Deeper Understanding of Spark Internals (Hadoop Conference Japan 2014)
 
So various polymorphism in Scala
So various polymorphism in ScalaSo various polymorphism in Scala
So various polymorphism in Scala
 
Scala’s implicits
Scala’s implicitsScala’s implicits
Scala’s implicits
 
PySaprk
PySaprkPySaprk
PySaprk
 
Unikernels: in search of a killer app and a killer ecosystem
Unikernels: in search of a killer app and a killer ecosystemUnikernels: in search of a killer app and a killer ecosystem
Unikernels: in search of a killer app and a killer ecosystem
 
Spark fundamentals i (bd095 en) version #1: updated: april 2015
Spark fundamentals i (bd095 en) version #1: updated: april 2015Spark fundamentals i (bd095 en) version #1: updated: april 2015
Spark fundamentals i (bd095 en) version #1: updated: april 2015
 
Real-World Scala Design Patterns
Real-World Scala Design PatternsReal-World Scala Design Patterns
Real-World Scala Design Patterns
 
Full stack analytics with Hadoop 2
Full stack analytics with Hadoop 2Full stack analytics with Hadoop 2
Full stack analytics with Hadoop 2
 
Type Checking Scala Spark Datasets: Dataset Transforms
Type Checking Scala Spark Datasets: Dataset TransformsType Checking Scala Spark Datasets: Dataset Transforms
Type Checking Scala Spark Datasets: Dataset Transforms
 
Scala Implicits - Not to be feared
Scala Implicits - Not to be fearedScala Implicits - Not to be feared
Scala Implicits - Not to be feared
 
臺灣高中數學講義 - 第一冊 - 數與式
臺灣高中數學講義 - 第一冊 - 數與式臺灣高中數學講義 - 第一冊 - 數與式
臺灣高中數學講義 - 第一冊 - 數與式
 
New Analytics Toolbox DevNexus 2015
New Analytics Toolbox DevNexus 2015New Analytics Toolbox DevNexus 2015
New Analytics Toolbox DevNexus 2015
 
Think Like Spark: Some Spark Concepts and a Use Case
Think Like Spark: Some Spark Concepts and a Use CaseThink Like Spark: Some Spark Concepts and a Use Case
Think Like Spark: Some Spark Concepts and a Use Case
 

Similaire à DTCC '14 Spark Runtime Internals

Sumedh Wale's presentation
Sumedh Wale's presentationSumedh Wale's presentation
Sumedh Wale's presentationpunesparkmeetup
 
HadoopThe Hadoop Java Software Framework
HadoopThe Hadoop Java Software FrameworkHadoopThe Hadoop Java Software Framework
HadoopThe Hadoop Java Software FrameworkThoughtWorks
 
Dive into spark2
Dive into spark2Dive into spark2
Dive into spark2Gal Marder
 
OCF.tw's talk about "Introduction to spark"
OCF.tw's talk about "Introduction to spark"OCF.tw's talk about "Introduction to spark"
OCF.tw's talk about "Introduction to spark"Giivee The
 
Bigdata processing with Spark - part II
Bigdata processing with Spark - part IIBigdata processing with Spark - part II
Bigdata processing with Spark - part IIArjen de Vries
 
Scala Meetup Hamburg - Spark
Scala Meetup Hamburg - SparkScala Meetup Hamburg - Spark
Scala Meetup Hamburg - SparkIvan Morozov
 
Apache Spark: What? Why? When?
Apache Spark: What? Why? When?Apache Spark: What? Why? When?
Apache Spark: What? Why? When?Massimo Schenone
 
Leveraging Hadoop in your PostgreSQL Environment
Leveraging Hadoop in your PostgreSQL EnvironmentLeveraging Hadoop in your PostgreSQL Environment
Leveraging Hadoop in your PostgreSQL EnvironmentJim Mlodgenski
 
Apache Spark - San Diego Big Data Meetup Jan 14th 2015
Apache Spark - San Diego Big Data Meetup Jan 14th 2015Apache Spark - San Diego Big Data Meetup Jan 14th 2015
Apache Spark - San Diego Big Data Meetup Jan 14th 2015cdmaxime
 
Spark 计算模型
Spark 计算模型Spark 计算模型
Spark 计算模型wang xing
 
Why Functional Programming Is Important in Big Data Era
Why Functional Programming Is Important in Big Data EraWhy Functional Programming Is Important in Big Data Era
Why Functional Programming Is Important in Big Data EraHandaru Sakti
 
TriHUG talk on Spark and Shark
TriHUG talk on Spark and SharkTriHUG talk on Spark and Shark
TriHUG talk on Spark and Sharktrihug
 

Similaire à DTCC '14 Spark Runtime Internals (20)

Sumedh Wale's presentation
Sumedh Wale's presentationSumedh Wale's presentation
Sumedh Wale's presentation
 
Scala and spark
Scala and sparkScala and spark
Scala and spark
 
HadoopThe Hadoop Java Software Framework
HadoopThe Hadoop Java Software FrameworkHadoopThe Hadoop Java Software Framework
HadoopThe Hadoop Java Software Framework
 
Zaharia spark-scala-days-2012
Zaharia spark-scala-days-2012Zaharia spark-scala-days-2012
Zaharia spark-scala-days-2012
 
spark-tutorial.pptx
spark-tutorial.pptxspark-tutorial.pptx
spark-tutorial.pptx
 
Strata spark streaming2
Strata spark streaming2Strata spark streaming2
Strata spark streaming2
 
Spark tutorial
Spark tutorialSpark tutorial
Spark tutorial
 
Dive into spark2
Dive into spark2Dive into spark2
Dive into spark2
 
OCF.tw's talk about "Introduction to spark"
OCF.tw's talk about "Introduction to spark"OCF.tw's talk about "Introduction to spark"
OCF.tw's talk about "Introduction to spark"
 
Shark
SharkShark
Shark
 
Bigdata processing with Spark - part II
Bigdata processing with Spark - part IIBigdata processing with Spark - part II
Bigdata processing with Spark - part II
 
Scala Meetup Hamburg - Spark
Scala Meetup Hamburg - SparkScala Meetup Hamburg - Spark
Scala Meetup Hamburg - Spark
 
Osd ctw spark
Osd ctw sparkOsd ctw spark
Osd ctw spark
 
Apache Spark: What? Why? When?
Apache Spark: What? Why? When?Apache Spark: What? Why? When?
Apache Spark: What? Why? When?
 
Leveraging Hadoop in your PostgreSQL Environment
Leveraging Hadoop in your PostgreSQL EnvironmentLeveraging Hadoop in your PostgreSQL Environment
Leveraging Hadoop in your PostgreSQL Environment
 
Apache Spark - San Diego Big Data Meetup Jan 14th 2015
Apache Spark - San Diego Big Data Meetup Jan 14th 2015Apache Spark - San Diego Big Data Meetup Jan 14th 2015
Apache Spark - San Diego Big Data Meetup Jan 14th 2015
 
Spark 计算模型
Spark 计算模型Spark 计算模型
Spark 计算模型
 
Why Functional Programming Is Important in Big Data Era
Why Functional Programming Is Important in Big Data EraWhy Functional Programming Is Important in Big Data Era
Why Functional Programming Is Important in Big Data Era
 
TriHUG talk on Spark and Shark
TriHUG talk on Spark and SharkTriHUG talk on Spark and Shark
TriHUG talk on Spark and Shark
 
Introduction to Apache Spark
Introduction to Apache SparkIntroduction to Apache Spark
Introduction to Apache Spark
 

Dernier

Scanning the Internet for External Cloud Exposures via SSL Certs
Scanning the Internet for External Cloud Exposures via SSL CertsScanning the Internet for External Cloud Exposures via SSL Certs
Scanning the Internet for External Cloud Exposures via SSL CertsRizwan Syed
 
Human Factors of XR: Using Human Factors to Design XR Systems
Human Factors of XR: Using Human Factors to Design XR SystemsHuman Factors of XR: Using Human Factors to Design XR Systems
Human Factors of XR: Using Human Factors to Design XR SystemsMark Billinghurst
 
The Future of Software Development - Devin AI Innovative Approach.pdf
The Future of Software Development - Devin AI Innovative Approach.pdfThe Future of Software Development - Devin AI Innovative Approach.pdf
The Future of Software Development - Devin AI Innovative Approach.pdfSeasiaInfotech2
 
Story boards and shot lists for my a level piece
Story boards and shot lists for my a level pieceStory boards and shot lists for my a level piece
Story boards and shot lists for my a level piececharlottematthew16
 
My INSURER PTE LTD - Insurtech Innovation Award 2024
My INSURER PTE LTD - Insurtech Innovation Award 2024My INSURER PTE LTD - Insurtech Innovation Award 2024
My INSURER PTE LTD - Insurtech Innovation Award 2024The Digital Insurer
 
Streamlining Python Development: A Guide to a Modern Project Setup
Streamlining Python Development: A Guide to a Modern Project SetupStreamlining Python Development: A Guide to a Modern Project Setup
Streamlining Python Development: A Guide to a Modern Project SetupFlorian Wilhelm
 
Vector Databases 101 - An introduction to the world of Vector Databases
Vector Databases 101 - An introduction to the world of Vector DatabasesVector Databases 101 - An introduction to the world of Vector Databases
Vector Databases 101 - An introduction to the world of Vector DatabasesZilliz
 
"Debugging python applications inside k8s environment", Andrii Soldatenko
"Debugging python applications inside k8s environment", Andrii Soldatenko"Debugging python applications inside k8s environment", Andrii Soldatenko
"Debugging python applications inside k8s environment", Andrii SoldatenkoFwdays
 
Advanced Test Driven-Development @ php[tek] 2024
Advanced Test Driven-Development @ php[tek] 2024Advanced Test Driven-Development @ php[tek] 2024
Advanced Test Driven-Development @ php[tek] 2024Scott Keck-Warren
 
Connect Wave/ connectwave Pitch Deck Presentation
Connect Wave/ connectwave Pitch Deck PresentationConnect Wave/ connectwave Pitch Deck Presentation
Connect Wave/ connectwave Pitch Deck PresentationSlibray Presentation
 
Gen AI in Business - Global Trends Report 2024.pdf
Gen AI in Business - Global Trends Report 2024.pdfGen AI in Business - Global Trends Report 2024.pdf
Gen AI in Business - Global Trends Report 2024.pdfAddepto
 
DevoxxFR 2024 Reproducible Builds with Apache Maven
DevoxxFR 2024 Reproducible Builds with Apache MavenDevoxxFR 2024 Reproducible Builds with Apache Maven
DevoxxFR 2024 Reproducible Builds with Apache MavenHervé Boutemy
 
Dev Dives: Streamline document processing with UiPath Studio Web
Dev Dives: Streamline document processing with UiPath Studio WebDev Dives: Streamline document processing with UiPath Studio Web
Dev Dives: Streamline document processing with UiPath Studio WebUiPathCommunity
 
My Hashitalk Indonesia April 2024 Presentation
My Hashitalk Indonesia April 2024 PresentationMy Hashitalk Indonesia April 2024 Presentation
My Hashitalk Indonesia April 2024 PresentationRidwan Fadjar
 
Kotlin Multiplatform & Compose Multiplatform - Starter kit for pragmatics
Kotlin Multiplatform & Compose Multiplatform - Starter kit for pragmaticsKotlin Multiplatform & Compose Multiplatform - Starter kit for pragmatics
Kotlin Multiplatform & Compose Multiplatform - Starter kit for pragmaticscarlostorres15106
 
Developer Data Modeling Mistakes: From Postgres to NoSQL
Developer Data Modeling Mistakes: From Postgres to NoSQLDeveloper Data Modeling Mistakes: From Postgres to NoSQL
Developer Data Modeling Mistakes: From Postgres to NoSQLScyllaDB
 
SAP Build Work Zone - Overview L2-L3.pptx
SAP Build Work Zone - Overview L2-L3.pptxSAP Build Work Zone - Overview L2-L3.pptx
SAP Build Work Zone - Overview L2-L3.pptxNavinnSomaal
 
Artificial intelligence in cctv survelliance.pptx
Artificial intelligence in cctv survelliance.pptxArtificial intelligence in cctv survelliance.pptx
Artificial intelligence in cctv survelliance.pptxhariprasad279825
 

Dernier (20)

Scanning the Internet for External Cloud Exposures via SSL Certs
Scanning the Internet for External Cloud Exposures via SSL CertsScanning the Internet for External Cloud Exposures via SSL Certs
Scanning the Internet for External Cloud Exposures via SSL Certs
 
E-Vehicle_Hacking_by_Parul Sharma_null_owasp.pptx
E-Vehicle_Hacking_by_Parul Sharma_null_owasp.pptxE-Vehicle_Hacking_by_Parul Sharma_null_owasp.pptx
E-Vehicle_Hacking_by_Parul Sharma_null_owasp.pptx
 
Human Factors of XR: Using Human Factors to Design XR Systems
Human Factors of XR: Using Human Factors to Design XR SystemsHuman Factors of XR: Using Human Factors to Design XR Systems
Human Factors of XR: Using Human Factors to Design XR Systems
 
The Future of Software Development - Devin AI Innovative Approach.pdf
The Future of Software Development - Devin AI Innovative Approach.pdfThe Future of Software Development - Devin AI Innovative Approach.pdf
The Future of Software Development - Devin AI Innovative Approach.pdf
 
DMCC Future of Trade Web3 - Special Edition
DMCC Future of Trade Web3 - Special EditionDMCC Future of Trade Web3 - Special Edition
DMCC Future of Trade Web3 - Special Edition
 
Story boards and shot lists for my a level piece
Story boards and shot lists for my a level pieceStory boards and shot lists for my a level piece
Story boards and shot lists for my a level piece
 
My INSURER PTE LTD - Insurtech Innovation Award 2024
My INSURER PTE LTD - Insurtech Innovation Award 2024My INSURER PTE LTD - Insurtech Innovation Award 2024
My INSURER PTE LTD - Insurtech Innovation Award 2024
 
Streamlining Python Development: A Guide to a Modern Project Setup
Streamlining Python Development: A Guide to a Modern Project SetupStreamlining Python Development: A Guide to a Modern Project Setup
Streamlining Python Development: A Guide to a Modern Project Setup
 
Vector Databases 101 - An introduction to the world of Vector Databases
Vector Databases 101 - An introduction to the world of Vector DatabasesVector Databases 101 - An introduction to the world of Vector Databases
Vector Databases 101 - An introduction to the world of Vector Databases
 
"Debugging python applications inside k8s environment", Andrii Soldatenko
"Debugging python applications inside k8s environment", Andrii Soldatenko"Debugging python applications inside k8s environment", Andrii Soldatenko
"Debugging python applications inside k8s environment", Andrii Soldatenko
 
Advanced Test Driven-Development @ php[tek] 2024
Advanced Test Driven-Development @ php[tek] 2024Advanced Test Driven-Development @ php[tek] 2024
Advanced Test Driven-Development @ php[tek] 2024
 
Connect Wave/ connectwave Pitch Deck Presentation
Connect Wave/ connectwave Pitch Deck PresentationConnect Wave/ connectwave Pitch Deck Presentation
Connect Wave/ connectwave Pitch Deck Presentation
 
Gen AI in Business - Global Trends Report 2024.pdf
Gen AI in Business - Global Trends Report 2024.pdfGen AI in Business - Global Trends Report 2024.pdf
Gen AI in Business - Global Trends Report 2024.pdf
 
DevoxxFR 2024 Reproducible Builds with Apache Maven
DevoxxFR 2024 Reproducible Builds with Apache MavenDevoxxFR 2024 Reproducible Builds with Apache Maven
DevoxxFR 2024 Reproducible Builds with Apache Maven
 
Dev Dives: Streamline document processing with UiPath Studio Web
Dev Dives: Streamline document processing with UiPath Studio WebDev Dives: Streamline document processing with UiPath Studio Web
Dev Dives: Streamline document processing with UiPath Studio Web
 
My Hashitalk Indonesia April 2024 Presentation
My Hashitalk Indonesia April 2024 PresentationMy Hashitalk Indonesia April 2024 Presentation
My Hashitalk Indonesia April 2024 Presentation
 
Kotlin Multiplatform & Compose Multiplatform - Starter kit for pragmatics
Kotlin Multiplatform & Compose Multiplatform - Starter kit for pragmaticsKotlin Multiplatform & Compose Multiplatform - Starter kit for pragmatics
Kotlin Multiplatform & Compose Multiplatform - Starter kit for pragmatics
 
Developer Data Modeling Mistakes: From Postgres to NoSQL
Developer Data Modeling Mistakes: From Postgres to NoSQLDeveloper Data Modeling Mistakes: From Postgres to NoSQL
Developer Data Modeling Mistakes: From Postgres to NoSQL
 
SAP Build Work Zone - Overview L2-L3.pptx
SAP Build Work Zone - Overview L2-L3.pptxSAP Build Work Zone - Overview L2-L3.pptx
SAP Build Work Zone - Overview L2-L3.pptx
 
Artificial intelligence in cctv survelliance.pptx
Artificial intelligence in cctv survelliance.pptxArtificial intelligence in cctv survelliance.pptx
Artificial intelligence in cctv survelliance.pptx
 

DTCC '14 Spark Runtime Internals

  • 2. What is • A fast and general engine for large-scale data processing • An open source implementation of Resilient Distributed Datasets (RDD) • Has an advanced DAG execution engine that supports cyclic data flow and in-memory computing
  • 3. Why • Fast – Run machine learning like iterative programs up to 100x faster than Hadoop MapReduce in memory, or 10x faster on disk – Run HiveQL compatible queries 100x faster than Hive (with Shark/Spark SQL)
  • 4. Why • Easy to use – Fluent Scala/Java/Python API – Interactive shell – 2-5x less code (than Hadoop MapReduce)
  • 5. Why • Easy to use – Fluent Scala/Java/Python API – Interactive shell – 2-5x less code (than Hadoop MapReduce) sc.textFile("hdfs://...") .flatMap(_.split(" ")) .map(_ -> 1) .reduceByKey(_ + _) .collectAsMap()
  • 6. Why • Easy to use – Fluent Scala/Java/Python API – Interactive shell – 2-5x less code (than Hadoop MapReduce) sc.textFile("hdfs://...") .flatMap(_.split(" ")) .map(_ -> 1) .reduceByKey(_ + _) .collectAsMap() Can you write down WordCount in 30 seconds with Hadoop MapReduce?
  • 7. Why • Unified big data pipeline for: – Batch/Interactive (Spark Core vs MR/Tez) – SQL (Shark/Spark SQL vs Hive) – Streaming (Spark Streaming vs Storm) – Machine learning (MLlib vs Mahout) – Graph (GraphX vs Giraph)
  • 9. An Even Bigger Picture... • Components of the Spark stack focus on big data analysis and are compatible with existing Hadoop storage systems • Users don’t need to suffer expensive ETL cost to use the Spark stack
  • 10. “One Stack To Rule Them All” • Well, mostly :-) • And don't forget Shark/Spark SQL vs Hive
  • 12. Resilient Distributed Datasets • Conceptually, RDDs can be roughly viewed as partitioned, locality aware distributed vectors • An RDD… – either points to a direct data source – or applies some transformation to its parent RDD(s) to generate new data elements – Computation can be represented by lazy evaluated lineage DAGs composed by connected RDDs
  • 13. Resilient Distributed Datasets • Frequently used RDDs can be materialized and cached in-memory to accelerate computation • Spark scheduler takes data locality into account
  • 14. The In-Memory Magic • “In fact, one study [1] analyzed the access patterns in the Hive warehouses at Facebook and discovered that for the vast majority (96%) of jobs, the entire inputs could fit into a fraction of the cluster’s total memory.” [1] G. Ananthanarayanan, A. Ghodsi, S. Shenker, and I. Stoica. Disk-locality in datacenter computing considered irrelevant. In HotOS ’11, 2011.
  • 15. The In-Memory Magic • Without cache – Elements are accessed in an iterator-based streaming style – One element a time, no bulk copy – Space complexity is almost O(1) when there’s only narrow dependencies
  • 16. The In-Memory Magic • With cache – One block per RDD partition – LRU cache eviction – Locality aware – Evicted blocks can be recomputed in parallel with the help of RDD lineage DAG
  • 17. Play with Error Logs sc.textFile("hdfs://<input>") .filter(_.startsWith("ERROR")) .map(_.split(" ")(1)) .saveAsTextFile("hdfs://<output>")
  • 18. Step by Step The RDD.saveAsTextFile() action triggers a job. Tasks are started on scheduled executors.
  • 19. The task pulls data lazily from the final RDD (the MappedRDD here). Step by Step
  • 20. A record (text line) is pulled out from HDFS... Step by Step
  • 21. ... into a HadoopRDD Step by Step
  • 22. Then filtered with the FilteredRDD Step by Step
  • 23. Oops, not a match Step by Step
  • 24. Another record is pulled out... Step by Step
  • 26. Passed to the MappedRDD for next transformation Step by Step
  • 27. Transformed by the MappedRDD (the error message is extracted) Step by Step
  • 28. Saves the message to HDFS Step by Step
  • 29. A Synthesized Iterator new Iterator[String] { private var head: String = _ private var headDefined: Boolean = false def hasNext: Boolean = headDefined || { do { try head = readOneLineFromHDFS(...) // (1) read from HDFS catch { case _: EOFException => return false } } while (!head.startsWith("ERROR")) // (2) filter closure true } def next: String = if (hasNext) { headDefined = false head.split(" ")(1) // (3) map closure } else { throw new NoSuchElementException("...") } } Constant Space Complexity!
  • 30. If Cached... val cached = sc .textFile("hdfs://<input>") .filter(_.startsWith("ERROR")) .cache() cached .map(_.split(" ")(1)) .saveAsTextFile("hdfs://<output>")
  • 31. If Cached... All filtered error messages are cached in memory before being passed to the next RDD. LRU cache eviction is applied when memory is insufficient
  • 32. But… I Don’t Have Enough Memory To Cache All Data
  • 33. Don’t Worry • In most cases, you only need to cache a fraction of hot data extracted/transformed from the whole input dataset • Spark performance downgrades linearly & gracefully when memory decreases
  • 34. How Does The Job Get Distributed?
  • 36. Driver, Cluster Manager, Worker, Executor, ... So... Where THE HELL Does My Code Run?
  • 38. Driver and SparkContext import org.apache.spark.SparkContext import org.apache.spark.SparkContext._ val sc = new SparkContext(...) • A SparkContext initializes the application driver, the latter then registers the application to the cluster manager, and gets a list of executors • Since then, the driver takes full responsibilities
  • 39. WordCount Revisited val lines = sc.textFile("input") val words = lines.flatMap(_.split(" ")) val ones = words.map(_ -> 1) val counts = ones.reduceByKey(_ + _) val result = counts.collectAsMap() • RDD lineage DAG is built on driver side with: – Data source RDD(s) – Transformation RDD(s), which are created by transformations
  • 40. WordCount Revisited val lines = sc.textFile("input") val words = lines.flatMap(_.split(" ")) val ones = words.map(_ -> 1) val counts = ones.reduceByKey(_ + _) val result = counts.collectAsMap() • Once an action is triggered on driver side, a job is submitted to the DAG scheduler of the driver
  • 41. WordCount Revisited • DAG scheduler cuts the DAG into stages and turns each partition of a stage into a single task. • DAG scheduler decides what to run
  • 42. WordCount Revisited • Tasks are then scheduled to executors by driver side task scheduler according to resource and locality constraints • Task scheduler decides where to run
  • 43. WordCount Revisited val lines = sc.textFile("input") val words = lines.flatMap(_.split(" ")) val ones = words.map(_ -> 1) val counts = ones.reduceByKey(_ + _) val result = counts.collectAsMap() • Within a task, the lineage DAG of corresponding stage is serialized together with closures of transformations, then sent to and executed on scheduled executors
  • 44. WordCount Revisited val lines = sc.textFile("input") val words = lines.flatMap(_.split(" ")) val ones = words.map(_ -> 1) val counts = ones.reduceByKey(_ + _) val result = counts.collectAsMap() • The reduceByKey transformation introduces in a shuffle • Shuffle outputs are written to local FS on the mapper side, then downloaded by reducers
  • 45. WordCount Revisited val lines = sc.textFile("input") val words = lines.flatMap(_.split(" ")) val ones = words.map(_ -> 1) val counts = ones.reduceByKey(_ + _) val result = counts.collectAsMap() • ReduceByKey automatically combines values within a single partition locally on the mapper side and then reduce them globally on the reducer side.
  • 46. WordCount Revisited val lines = sc.textFile("input") val words = lines.flatMap(_.split(" ")) val ones = words.map(_ -> 1) val counts = ones.reduceByKey(_ + _) val result = counts.collectAsMap() • At last, results of the action are sent back to the driver, then the job finishes.
  • 48. How To Control Parallelism? • Can be specified in a number of ways – RDD partition number • sc.textFile("input", minSplits = 10) • sc.parallelize(1 to 10000, numSlices = 10) – Mapper side parallelism • Usually inherited from parent RDD(s) – Reducer side parallelism • rdd.reduceByKey(_ + _, numPartitions = 10) • rdd.reduceByKey(partitioner = p, _ + _) • …
  • 49. How To Control Parallelism? • “Zoom in/out” – RDD.repartition(numPartitions: Int) – RDD.coalesce( numPartitions: Int, shuffle: Boolean)
  • 50. A Trap of Partitions sc.textFile("input", minSplits = 2) .map { line => val Array(key, value) = line.split(",") key.toInt -> value.toInt } .reduceByKey(_ + _) .saveAsText("output") • Run in local mode • 60+GB input file
  • 51. A Trap of Partitions sc.textFile("input", minSplits = 2) .map { line => val Array(key, value) = line.split(",") key.toInt -> value.toInt } .reduceByKey(_ + _) .saveAsText("output") Hmm... Pretty cute?
  • 52. A Trap of Partitions sc.textFile("input", minSplits = 2) .map { line => val Array(key, value) = line.split(",") key.toInt -> value.toInt } .reduceByKey(_ + _) .saveAsText("output") • Actually, this may generate 4,000,000 temporary files... • Why 4M? (Hint: 2K2) !!!
  • 53. A Trap of Partitions sc.textFile("input", minSplits = 2) .map { line => val Array(key, value) = line.split(",") key.toInt -> value.toInt } .reduceByKey(_ + _) .saveAsText("output") • 1 partition per HDFS split • Similar to mapper task number in Hadoop MapReduce, the minSplits parameter is only a hint for split number
  • 54. A Trap of Partitions sc.textFile("input", minSplits = 2) .map { line => val Array(key, value) = line.split(",") key.toInt -> value.toInt } .reduceByKey(_ + _) .saveAsText("output") • Actual split number is also controlled by some other properties like minSplitSize and default FS block size
  • 55. A Trap of Partitions sc.textFile("input", minSplits = 2) .map { line => val Array(key, value) = line.split(",") key.toInt -> value.toInt } .reduceByKey(_ + _) .saveAsText("output") • In this case, the final split size equals to local FS block size, which is 32MB by default, and 60GB / 32MB ≈ 2K • ReduceByKey generates 2K2 shuffle outputs
  • 56. A Trap of Partitions sc.textFile("input", minSplits = 2).coalesce(2) .map { line => val Array(key, value) = line.split(",") key.toInt -> value.toInt } .reduceByKey(_ + _) .saveAsText("output") • Use RDD.coalesce() to control partition number precisely.
  • 57. Summary • In-memory magic – Iterator-based streaming style data access – Usually you don’t need to cache all dataset – When memory is not enough, Spark performance downgrades linearly & gracefully
  • 58. Summary • Distributed job – Locality aware – Resource aware • Parallelism – Reasonable default behavior – Can be finely tuned
  • 59. References • Resilient distributed datasets: a fault-tolerant abstraction for in-memory cluster computing • An Architecture for Fast and General Data Processing on Large Clusters • Spark Internals (video, slides) • Shark: SQL and Rich Analytics at Scale • Spark Summit 2013
  • 60. Q&A