SlideShare ist ein Scribd-Unternehmen logo
1 von 44
Think Like Spark
Some Spark Concept & A Use Case
Who am I?
• Software engineer, data scientist, and Spark enthusiast at
Alpine Data (SF Based Analytics Company)
• Co – Author High Performance Spark
http://shop.oreilly.com/product/0636920046967.do
Linked in: https://www.linkedin.com/in/rachelbwarren
• Slide share: http://www.slideshare.net/RachelWarren4
• Github : rachelwarren. Code for this talk
https://github.com/high-performance-spark/high-performance-
spark-examples
• Twitter: @warre_n_peace
Overview
• A little Spark Architecture: How are Spark Jobs
Evaluated? Why does that matter for performance?
• Execution context: driver, executors, partitions, cores
• Spark Application hierarchy: jobs/stages/tasks
• Actions vs. Transformations (lazy evaluation)
• Wide vs. Narrow Transformations (shuffles & data locality)
• Apply what we have learned with four versions of the
same algorithm to find rank statistics
What is Spark?
Distributed computing framework. Must run in tandem with a
data storage system
- Standalone (For Local Testing)
- Cloud (S3, EC2)
- Distributed storage, with cluster manager,
- (Hadoop Yarn, Apache Messos)
Built around and abstraction called RDDs “Resilient,
Distributed, Datasets”
- Lazily evaluated, immutable, distributed collection of
partition objects
What happens when I launch
a Spark Application?
Spark
Driver
ExecutorExecutor Executor Executor
Stable storage e.g. HDFS
One Spark Executor
• One JVM for in memory
computations
• Partitions care computed on
executors
• Tasks correspond to partitions
• dynamically allocated slots for
running tasks
(executor cores x executors)
• Caching takes up space on
executors Partitions / Tasks
Implications
Two Most common cases of failures
1. Failure during shuffle stage
• Moving data between Partitions requires communication with
the driver
 Failures often occur in the shuffle stage
2. Out of memory errors on executors and driver
The driver and each executor have a static amount of memory*
 It is easy to run out of memory on the executors or on the driver
*dynamic allocation allows changing the number of executors
How are Jobs Evaluated?
API Call Execution Element
Computation to evaluation
one partition (combine
narrow transforms)
Wide transformations (sort,
groupByKey)
Actions (e.g. collect,
saveAsTextFile)
Spark Context Object Spark
Application
Job
Stage
Task Task
Stage
Executed in
Sequence
Executed in
Parallel
Types Of Spark Operations
Actions
• RDD  Not RDD
• Force execution: Each job
ends in exactly one action
• Three Kinds
• Move data to driver: collect,
take, count
• Move data to external system
Write / Save
• Force evaluation: foreach
Transformations
• RDD  RDD
• Lazily evaluated
• Can be combined and
executed in one pass of
the data
• Computed on Spark
executors
Implications of Lazy Evaluation
Frustrating:
• Debugging = 
• Lineage graph is built backwards from action to reading in
data or persist/ cache/ checkpoint  if you aren’t careful you
will repeat computations *
* some times we get help from shuffle files
Awesome:
• Spark can combine some types of transformations and execute
them in a single task
• We only compute partitions that we need
Types of Transformations
Narrow
• Never require a shuffle
• map, mapPartitions, filter
• coalesce*
• Input partitions >= output
partitions
• & output partitions known
at design time
• A sequence of narrow
transformations are
combined and executed in
one stage as several tasks
Wide
• May require a shuffle
• sort, groupByKey,
reduceByKey, join
• Requires data movement
• Partitioning depends on data
it self (not known at design
time)
• Cause stage boundary:
Stage 2 cannot be complete
until all the partitions in
Stage 1 are computed.
Partition Dependencies for
input and output partitions
Narrow Wide
Implications of Shuffles
• Narrow transformations are faster/ more parallelizable
• Narrow transformation must be written so that they can
be computed on any subset of records
• Narrow transformations can rely on some partitioning
information, (partition remains constant in each stage)*
• Wide transformations may distribute data unevenly across
machines (for example according to the hash value of the
key)
*we can loose partitioning information with map or
mapPartitions(preservesPartitioner = false)
The “Goldilocks Use Case”
Rank Statistics on Wide Data
Design an application that would takes an arbitrary list of
longs `n1`...`nk` and return the `nth` best element in each
column of a DataFrame of doubles.
For example, if the input list is ( 8, 1000, and 20 million),
our function would need to return the 8th, 1000th and 20
millionth largest element in each column.
Input Data
If we were looking for 2 and 4th elements, result would be:
V0: Iterative solution
Loop through each column:
• map to value in the one column
• Sort the column
• Zip with index and filter for the correct rank statistic (i.e.
nth element)
• Add the result for each column to a map
def findRankStatistics(
dataFrame: DataFrame, ranks: List[Long]): Map[Int, Iterable[Double]] = {
val numberOfColumns = dataFrame.schema.length
var i = 0
var result = Map[Int, Iterable[Double]]()
dataFrame.persist()
while(i < numberOfColumns){
val col = dataFrame.rdd.map(row => row.getDouble(i))
val sortedCol : RDD[(Double, Long)] =
col.sortBy(v => v).zipWithIndex()
val ranksOnly = sortedCol.filter{
//rank statistics are indexed from one
case (colValue, index) => ranks.contains(index + 1)
}.keys
val list = ranksOnly.collect()
result += (i -> list)
i+=1
}
result
}
Persist prevents
multiple data reads
SortBy is Spark’s sort
V0 = Too Many Sorts 
• Turtle Picture
• One distributed sort
per column
(800 cols = 800 sorts)
• Each of these sorts
is executed in
sequence
• Cannot save
partitioning data
between sorts
300 Million rows
takes days!
V1: Parallelize by Column
• The work to sort each column can be done without
information about the other columns
• Can map the data to (column index, value) pairs
• GroupByKey on column index
• Sort each group
• Filter for desired rank statistics
Get Col Index, Value Pairs
private def getValueColumnPairs(dataFrame : DataFrame): RDD[(Double,
Int)] = {
dataFrame.rdd.flatMap{
row: Row => row.toSeq.zipWithIndex.map{
case (v, index) =>
(v.toString.toDouble, index)}
}
}
Flatmap is a narrow transformation
Column Index Value
1 15.0
1 2.0
.. …
Group By Key Solution
• def findRankStatistics(
dataFrame: DataFrame ,
ranks: List[Long]): Map[Int, Iterable[Double]] = {
require(ranks.forall(_ > 0))
//Map to column index, value pairs
val pairRDD: RDD[(Int, Double)] = mapToKeyValuePairs(dataFrame)
val groupColumns: RDD[(Int, Iterable[Double])] =
pairRDD.groupByKey()
groupColumns.mapValues(
iter => {
//convert to an array and sort
val sortedIter = iter.toArray.sorted
sortedIter.toIterable.zipWithIndex.flatMap({
case (colValue, index) =>
if (ranks.contains(index + 1))
Iterator(colValue)
else
Iterator.empty
})
}).collectAsMap()
}
V1. Faster on Small Data fails on Big Data
300 K rows = quick
300 M rows = fails
Problems with V1
• GroupByKey puts records from all the columns with the
same hash value on the same partition THEN loads them
into memory
• All columns with the same hash value have to fit in
memory on each executor
• Can’t start the sorting until after the group by key phase
has finished
V2 : Secondary Sort Style
1. ‘partitionAndSortWithinPartitions’: use the same hash
partitioner as GroupByKey
- Partition by key and sort all records on each partition
- Pushes some of the sorting work done on each partition
into the shuffle stage
2. Use mapPartitions to filter for the correct rank statistics
- Doesn’t force each column to be stored as an in memory
data structure (each partition stays as one iterator)
Still fails on 300 M rows
Iterator-Iterator-Transformation
With Map Partitions
• Iterators are not collections. They are a routine for
accessing each element
• Allows Spark to selectively spill to disk
• Don’t need to put all elements into memory
In our case: Prevents loading each column into memory
after the sorting stage
def findRankStatisticsV2(pairRDD: RDD[(Int, Double)],
targetRanks: List[Long],
partitions : Int ) = {
val partitioner = new HashPartitioner(partitions)
val sorted =
pairRDD
.repartitionAndSortWithinPartitions(partitioner)
}
V2: Secondary Sort
Repartition + sort using
Hash Partitioner
def findRankStatisticsV2(pairRDD: RDD[(Int, Double)],
sorted.mapPartitions(iter => {
var currentIndex = -1
var elementsPerIndex = 0
val filtered = iter.filter {
case (colIndex, value) =>
if (colIndex != currentIndex) {
currentIndex = 1
elementsPerIndex = 1
} else {
elementsPerIndex += 1
}
targetRanks.contains(elementsPerIndex)
}
groupSorted(filtered) //groups together ranks in same column
}, preservesPartitioning = true)
filterForTargetIndex.collectAsMap()
}
V2: Secondary Sort
Iterator-to-iterator
transformation
V2: Still Fails
We don’t have put each column into memory
but columns with the same hash value still have to be able
to fit on one partition
Back to the drawing board
• Narrow transformations are quick and easy to parallelize
• Partition locality can be retained across narrow transformations
• Wide transformations are best with many unique keys.
• Using iterator-to-iterator transforms in mapPartitions prevents
whole partitions from being loaded into memory
• We can rely on shuffle files to prevent re-computation of a
wide transformations be several subsequent actions
We can solve the problem with one sortByKey and three map
partitions
V3: Mo Parallel, Mo Better
1. Map to (cell value, column index) pairs
2. Do one very large sortByKey
3. Use mapPartitions to count the values per column on each
partition
4. (Locally) using the results of 3 compute location of each rank
statistic on each partition
5. Revisit each partition and find the correct rank statistics
using the information from step 4.
e.g. If the first partition has 10 elements from one column .
The13th element will be the third element on the second partition
in that column.
def findRankStatistics(dataFrame: DataFrame, targetRanks: List[Long]):
Map[Int, Iterable[Double]] = {
val valueColumnPairs: RDD[(Double, Int)] = getValueColumnPairs(dataFrame)
val sortedValueColumnPairs = valueColumnPairs.sortByKey()
sortedValueColumnPairs.persist(StorageLevel.MEMORY_AND_DISK)
val numOfColumns = dataFrame.schema.length
val partitionColumnsFreq =
getColumnsFreqPerPartition(sortedValueColumnPairs, numOfColumns)
val ranksLocations =
getRanksLocationsWithinEachPart(targetRanks, partitionColumnsFreq, numOfColumns)
val targetRanksValues = findTargetRanksIteratively(sortedValueColumnPairs, ranksLocations)
targetRanksValues.groupByKey().collectAsMap()
}
Complete code here: https://github.com/high-performance-spark/high-performance-spark-
examples/blob/master/src/main/scala/com/high-performance-spark-examples/GoldiLocks/GoldiLocksFirstTry.scala
1. Map to (val, col) pairs
2. Sort
3. Count per partition
4.
5. Filter for element
computed in 4
Complete code here:
https://github.com/high-performance-spark/high-
performance-spark-
examples/blob/master/src/main/scala/com/high-
performance-spark-
examples/GoldiLocks/GoldiLocksFirstTry.scala
V3: Still Blows up!
• First partitions show lots of failures and straggler tasks
• Jobs lags in the sort stage and fails in final mapPartitions
stage
More digging reveled data was not evenly distributed
Data skew¼ of columns are zero
V4: Distinct values per Partition
• Instead of mapping from (value, column index pairs),
map to ((value, column index), count) pairs on each
partition
e. g. if on a given partition, there are ten rows with 0.0 in
the 2nd column, we could save just one tuple:
(0.0, 2), 10)
• Use same sort and mapPartitions routines, but adjusted
for counts.
Different Key
column0 column2
2.0 3.0
0.0 3.0
0.0 1.0
0.0 0.0
(value, column Index), count)
((2.0, 0), 1)
(2.0,0), 3)
(3.0, 1), 2) ….
V4: Get (value, o
• Code for V4
def getAggregatedValueColumnPairs(dataFrame : DataFrame) : RDD[((Double, Int),
Long)] = {
val aggregatedValueColumnRDD = dataFrame.rdd.mapPartitions(rows => {
val valueColumnMap = new mutable.HashMap[(Double, Int), Long]()
rows.foreach(row => {
row.toSeq.zipWithIndex.foreach{
case (value, columnIndex) =>
val key = (value.toString.toDouble, columnIndex)
val count = valueColumnMap.getOrElseUpdate(key, 0)
valueColumnMap.update(key, count + 1)
}
})
valueColumnMap.toIterator
})
aggregatedValueColumnRDD
}
Map to ((value, column Index) ,count)
Using a hashmap to keep track of uniques
Code for V4
• Lots more code to complete the whole algorithm
https://github.com/high-performance-spark/high-
performance-spark-
examples/blob/master/src/main/scala/com/high-
performance-spark-
examples/GoldiLocks/GoldiLocksWithHashMap.scala
V4: Success!
• 4 times faster than previous
solution on small data
• More robust, more
parallelizable! Scaling to
billions of rows!
Happy Goldilocks!
Why is V4: Better
Advantages
• Sorting 75% of original records
• Most keys are distinct
• No stragglers, easy to parallelize
• Can parallelize in many different ways
Lessons
• Sometimes performance looks ugly
• Best unit of parallelization?  Not always the most intuitive
• Shuffle Less
• Push work into narrow transformations
• leverage data locality to prevent shuffles
• Shuffle Better
• Shuffle fewer records
• Use narrow transformations to filter or reduce when possible
• Shuffle across keys that are well distributed
• Best if records associated with one key fit in memory
• Be aware of data skew, know you data
Before We Part …
• Alpine Data is hiring!
Data scientists, engineers (ruby, java, as well as Hadoop/ Scala) ,
support, technical sales
“I continue to be amazed, Alpine has the nicest people ever” –
Former alpine engineer
http://alpinedata.com/careers/
• Buy my book!
http://shop.oreilly.com/product/0636920046967.do
Also contact me if you are interested in being a reviewer

Weitere ähnliche Inhalte

Was ist angesagt?

R-programming-training-in-mumbai
R-programming-training-in-mumbaiR-programming-training-in-mumbai
R-programming-training-in-mumbaiUnmesh Baile
 
Introduction to data analysis using R
Introduction to data analysis using RIntroduction to data analysis using R
Introduction to data analysis using RVictoria López
 
Aaa ped-6-Data manipulation: Data Files, and Data Cleaning & Preparation
Aaa ped-6-Data manipulation:  Data Files, and Data Cleaning & PreparationAaa ped-6-Data manipulation:  Data Files, and Data Cleaning & Preparation
Aaa ped-6-Data manipulation: Data Files, and Data Cleaning & PreparationAminaRepo
 
R programming Fundamentals
R programming  FundamentalsR programming  Fundamentals
R programming FundamentalsRagia Ibrahim
 
Lecture 4 - Comm Lab: Web @ ITP
Lecture 4 - Comm Lab: Web @ ITPLecture 4 - Comm Lab: Web @ ITP
Lecture 4 - Comm Lab: Web @ ITPyucefmerhi
 
Data engineering and analytics using python
Data engineering and analytics using pythonData engineering and analytics using python
Data engineering and analytics using pythonPurna Chander
 
Data all over the place! How SQL and Apache Calcite bring sanity to streaming...
Data all over the place! How SQL and Apache Calcite bring sanity to streaming...Data all over the place! How SQL and Apache Calcite bring sanity to streaming...
Data all over the place! How SQL and Apache Calcite bring sanity to streaming...Julian Hyde
 
Python and CSV Connectivity
Python and CSV ConnectivityPython and CSV Connectivity
Python and CSV ConnectivityNeeru Mittal
 
Data Analysis with Python Pandas
Data Analysis with Python PandasData Analysis with Python Pandas
Data Analysis with Python PandasNeeru Mittal
 
pandas - Python Data Analysis
pandas - Python Data Analysispandas - Python Data Analysis
pandas - Python Data AnalysisAndrew Henshaw
 
Is there a perfect data-parallel programming language? (Experiments with More...
Is there a perfect data-parallel programming language? (Experiments with More...Is there a perfect data-parallel programming language? (Experiments with More...
Is there a perfect data-parallel programming language? (Experiments with More...Julian Hyde
 
Latex for beginners
Latex for beginnersLatex for beginners
Latex for beginnersKaushik Naik
 

Was ist angesagt? (20)

R-programming-training-in-mumbai
R-programming-training-in-mumbaiR-programming-training-in-mumbai
R-programming-training-in-mumbai
 
Introduction to data analysis using R
Introduction to data analysis using RIntroduction to data analysis using R
Introduction to data analysis using R
 
R Get Started I
R Get Started IR Get Started I
R Get Started I
 
Essentials of R
Essentials of REssentials of R
Essentials of R
 
Aaa ped-6-Data manipulation: Data Files, and Data Cleaning & Preparation
Aaa ped-6-Data manipulation:  Data Files, and Data Cleaning & PreparationAaa ped-6-Data manipulation:  Data Files, and Data Cleaning & Preparation
Aaa ped-6-Data manipulation: Data Files, and Data Cleaning & Preparation
 
محاضرة برنامج التحليل الكمي R program د.هديل القفيدي
محاضرة برنامج التحليل الكمي   R program د.هديل القفيديمحاضرة برنامج التحليل الكمي   R program د.هديل القفيدي
محاضرة برنامج التحليل الكمي R program د.هديل القفيدي
 
Map/Reduce intro
Map/Reduce introMap/Reduce intro
Map/Reduce intro
 
R Get Started II
R Get Started IIR Get Started II
R Get Started II
 
محاضرة برنامج التحليل الكمي R program د.هديل القفيدي
محاضرة برنامج التحليل الكمي   R program د.هديل القفيديمحاضرة برنامج التحليل الكمي   R program د.هديل القفيدي
محاضرة برنامج التحليل الكمي R program د.هديل القفيدي
 
R programming Fundamentals
R programming  FundamentalsR programming  Fundamentals
R programming Fundamentals
 
Lecture 4 - Comm Lab: Web @ ITP
Lecture 4 - Comm Lab: Web @ ITPLecture 4 - Comm Lab: Web @ ITP
Lecture 4 - Comm Lab: Web @ ITP
 
Data engineering and analytics using python
Data engineering and analytics using pythonData engineering and analytics using python
Data engineering and analytics using python
 
Data all over the place! How SQL and Apache Calcite bring sanity to streaming...
Data all over the place! How SQL and Apache Calcite bring sanity to streaming...Data all over the place! How SQL and Apache Calcite bring sanity to streaming...
Data all over the place! How SQL and Apache Calcite bring sanity to streaming...
 
Python and CSV Connectivity
Python and CSV ConnectivityPython and CSV Connectivity
Python and CSV Connectivity
 
R language
R languageR language
R language
 
Data Analysis with Python Pandas
Data Analysis with Python PandasData Analysis with Python Pandas
Data Analysis with Python Pandas
 
pandas - Python Data Analysis
pandas - Python Data Analysispandas - Python Data Analysis
pandas - Python Data Analysis
 
Is there a perfect data-parallel programming language? (Experiments with More...
Is there a perfect data-parallel programming language? (Experiments with More...Is there a perfect data-parallel programming language? (Experiments with More...
Is there a perfect data-parallel programming language? (Experiments with More...
 
Latex for beginners
Latex for beginnersLatex for beginners
Latex for beginners
 
Pandas
PandasPandas
Pandas
 

Andere mochten auch

Taking Spark Streaming to the Next Level with Datasets and DataFrames
Taking Spark Streaming to the Next Level with Datasets and DataFramesTaking Spark Streaming to the Next Level with Datasets and DataFrames
Taking Spark Streaming to the Next Level with Datasets and DataFramesDatabricks
 
How to deploy spark instance using ansible 2.0 in fiware lab v2
How to deploy spark instance using ansible 2.0 in fiware lab v2How to deploy spark instance using ansible 2.0 in fiware lab v2
How to deploy spark instance using ansible 2.0 in fiware lab v2Fernando Lopez Aguilar
 
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
 
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
 
Faster Batch Processing with Cloudera 5.7: Hive-on-Spark is ready for production
Faster Batch Processing with Cloudera 5.7: Hive-on-Spark is ready for productionFaster Batch Processing with Cloudera 5.7: Hive-on-Spark is ready for production
Faster Batch Processing with Cloudera 5.7: Hive-on-Spark is ready for productionCloudera, Inc.
 
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
 
Clickstream Analysis with Spark
Clickstream Analysis with Spark Clickstream Analysis with Spark
Clickstream Analysis with Spark Josef Adersberger
 
臺灣高中數學講義 - 第一冊 - 數與式
臺灣高中數學講義 - 第一冊 - 數與式臺灣高中數學講義 - 第一冊 - 數與式
臺灣高中數學講義 - 第一冊 - 數與式Xuan-Chao Huang
 
New Analytics Toolbox DevNexus 2015
New Analytics Toolbox DevNexus 2015New Analytics Toolbox DevNexus 2015
New Analytics Toolbox DevNexus 2015Robbie Strickland
 
Lessons Learned: Using Spark and Microservices
Lessons Learned: Using Spark and MicroservicesLessons Learned: Using Spark and Microservices
Lessons Learned: Using Spark and MicroservicesAlexis Seigneurin
 
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
 
Resilient Distributed Datasets
Resilient Distributed DatasetsResilient Distributed Datasets
Resilient Distributed DatasetsGabriele Modena
 
Apache Spark: killer or savior of Apache Hadoop?
Apache Spark: killer or savior of Apache Hadoop?Apache Spark: killer or savior of Apache Hadoop?
Apache Spark: killer or savior of Apache Hadoop?rhatr
 
IBM Spark Meetup - RDD & Spark Basics
IBM Spark Meetup - RDD & Spark BasicsIBM Spark Meetup - RDD & Spark Basics
IBM Spark Meetup - RDD & Spark BasicsSatya Narayan
 

Andere mochten auch (20)

Taking Spark Streaming to the Next Level with Datasets and DataFrames
Taking Spark Streaming to the Next Level with Datasets and DataFramesTaking Spark Streaming to the Next Level with Datasets and DataFrames
Taking Spark Streaming to the Next Level with Datasets and DataFrames
 
How to deploy spark instance using ansible 2.0 in fiware lab v2
How to deploy spark instance using ansible 2.0 in fiware lab v2How to deploy spark instance using ansible 2.0 in fiware lab v2
How to deploy spark instance using ansible 2.0 in fiware lab v2
 
MLeap: Release Spark ML Pipelines
MLeap: Release Spark ML PipelinesMLeap: Release Spark ML Pipelines
MLeap: Release Spark ML Pipelines
 
Spark Uber Development Kit
Spark Uber Development KitSpark Uber Development Kit
Spark Uber Development Kit
 
Resilient Distributed Dataset - Analisis paper
Resilient  Distributed Dataset - Analisis paper Resilient  Distributed Dataset - Analisis paper
Resilient Distributed Dataset - Analisis paper
 
Spark meets Smart Meters
Spark meets Smart MetersSpark meets Smart Meters
Spark meets Smart Meters
 
Advanced Visualization of Spark jobs
Advanced Visualization of Spark jobsAdvanced Visualization of Spark jobs
Advanced Visualization of Spark jobs
 
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
 
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
 
Faster Batch Processing with Cloudera 5.7: Hive-on-Spark is ready for production
Faster Batch Processing with Cloudera 5.7: Hive-on-Spark is ready for productionFaster Batch Processing with Cloudera 5.7: Hive-on-Spark is ready for production
Faster Batch Processing with Cloudera 5.7: Hive-on-Spark is ready for production
 
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
 
Clickstream Analysis with Spark
Clickstream Analysis with Spark Clickstream Analysis with Spark
Clickstream Analysis with Spark
 
臺灣高中數學講義 - 第一冊 - 數與式
臺灣高中數學講義 - 第一冊 - 數與式臺灣高中數學講義 - 第一冊 - 數與式
臺灣高中數學講義 - 第一冊 - 數與式
 
New Analytics Toolbox DevNexus 2015
New Analytics Toolbox DevNexus 2015New Analytics Toolbox DevNexus 2015
New Analytics Toolbox DevNexus 2015
 
Lessons Learned: Using Spark and Microservices
Lessons Learned: Using Spark and MicroservicesLessons Learned: Using Spark and Microservices
Lessons Learned: Using Spark and Microservices
 
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
 
Resilient Distributed Datasets
Resilient Distributed DatasetsResilient Distributed Datasets
Resilient Distributed Datasets
 
Apache Spark: killer or savior of Apache Hadoop?
Apache Spark: killer or savior of Apache Hadoop?Apache Spark: killer or savior of Apache Hadoop?
Apache Spark: killer or savior of Apache Hadoop?
 
IBM Spark Meetup - RDD & Spark Basics
IBM Spark Meetup - RDD & Spark BasicsIBM Spark Meetup - RDD & Spark Basics
IBM Spark Meetup - RDD & Spark Basics
 

Ähnlich wie Think Like Spark

AI與大數據數據處理 Spark實戰(20171216)
AI與大數據數據處理 Spark實戰(20171216)AI與大數據數據處理 Spark實戰(20171216)
AI與大數據數據處理 Spark實戰(20171216)Paul Chao
 
Dive into spark2
Dive into spark2Dive into spark2
Dive into spark2Gal Marder
 
Spark Sql and DataFrame
Spark Sql and DataFrameSpark Sql and DataFrame
Spark Sql and DataFramePrashant Gupta
 
Tulsa techfest Spark Core Aug 5th 2016
Tulsa techfest Spark Core Aug 5th 2016Tulsa techfest Spark Core Aug 5th 2016
Tulsa techfest Spark Core Aug 5th 2016Mark Smith
 
MAP REDUCE IN DATA SCIENCE.pptx
MAP REDUCE IN DATA SCIENCE.pptxMAP REDUCE IN DATA SCIENCE.pptx
MAP REDUCE IN DATA SCIENCE.pptxHARIKRISHNANU13
 
Spark as the Gateway Drug to Typed Functional Programming: Spark Summit East ...
Spark as the Gateway Drug to Typed Functional Programming: Spark Summit East ...Spark as the Gateway Drug to Typed Functional Programming: Spark Summit East ...
Spark as the Gateway Drug to Typed Functional Programming: Spark Summit East ...Spark Summit
 
No more struggles with Apache Spark workloads in production
No more struggles with Apache Spark workloads in productionNo more struggles with Apache Spark workloads in production
No more struggles with Apache Spark workloads in productionChetan Khatri
 
Mapreduce advanced
Mapreduce advancedMapreduce advanced
Mapreduce advancedChirag Ahuja
 
Apache Spark, the Next Generation Cluster Computing
Apache Spark, the Next Generation Cluster ComputingApache Spark, the Next Generation Cluster Computing
Apache Spark, the Next Generation Cluster ComputingGerger
 
Scala meetup - Intro to spark
Scala meetup - Intro to sparkScala meetup - Intro to spark
Scala meetup - Intro to sparkJavier Arrieta
 
Beyond SQL: Speeding up Spark with DataFrames
Beyond SQL: Speeding up Spark with DataFramesBeyond SQL: Speeding up Spark with DataFrames
Beyond SQL: Speeding up Spark with DataFramesDatabricks
 
Apache Spark: What? Why? When?
Apache Spark: What? Why? When?Apache Spark: What? Why? When?
Apache Spark: What? Why? When?Massimo Schenone
 
Spark SQL Deep Dive @ Melbourne Spark Meetup
Spark SQL Deep Dive @ Melbourne Spark MeetupSpark SQL Deep Dive @ Melbourne Spark Meetup
Spark SQL Deep Dive @ Melbourne Spark MeetupDatabricks
 
Spark real world use cases and optimizations
Spark real world use cases and optimizationsSpark real world use cases and optimizations
Spark real world use cases and optimizationsGal Marder
 
Performing Data Science with HBase
Performing Data Science with HBasePerforming Data Science with HBase
Performing Data Science with HBaseWibiData
 
Sumedh Wale's presentation
Sumedh Wale's presentationSumedh Wale's presentation
Sumedh Wale's presentationpunesparkmeetup
 

Ähnlich wie Think Like Spark (20)

AI與大數據數據處理 Spark實戰(20171216)
AI與大數據數據處理 Spark實戰(20171216)AI與大數據數據處理 Spark實戰(20171216)
AI與大數據數據處理 Spark實戰(20171216)
 
Dive into spark2
Dive into spark2Dive into spark2
Dive into spark2
 
Spark Sql and DataFrame
Spark Sql and DataFrameSpark Sql and DataFrame
Spark Sql and DataFrame
 
Tulsa techfest Spark Core Aug 5th 2016
Tulsa techfest Spark Core Aug 5th 2016Tulsa techfest Spark Core Aug 5th 2016
Tulsa techfest Spark Core Aug 5th 2016
 
MAP REDUCE IN DATA SCIENCE.pptx
MAP REDUCE IN DATA SCIENCE.pptxMAP REDUCE IN DATA SCIENCE.pptx
MAP REDUCE IN DATA SCIENCE.pptx
 
Spark as the Gateway Drug to Typed Functional Programming: Spark Summit East ...
Spark as the Gateway Drug to Typed Functional Programming: Spark Summit East ...Spark as the Gateway Drug to Typed Functional Programming: Spark Summit East ...
Spark as the Gateway Drug to Typed Functional Programming: Spark Summit East ...
 
No more struggles with Apache Spark workloads in production
No more struggles with Apache Spark workloads in productionNo more struggles with Apache Spark workloads in production
No more struggles with Apache Spark workloads in production
 
Map Reduce
Map ReduceMap Reduce
Map Reduce
 
Meetup ml spark_ppt
Meetup ml spark_pptMeetup ml spark_ppt
Meetup ml spark_ppt
 
Mapreduce advanced
Mapreduce advancedMapreduce advanced
Mapreduce advanced
 
Apache Spark, the Next Generation Cluster Computing
Apache Spark, the Next Generation Cluster ComputingApache Spark, the Next Generation Cluster Computing
Apache Spark, the Next Generation Cluster Computing
 
Scala meetup - Intro to spark
Scala meetup - Intro to sparkScala meetup - Intro to spark
Scala meetup - Intro to spark
 
Beyond SQL: Speeding up Spark with DataFrames
Beyond SQL: Speeding up Spark with DataFramesBeyond SQL: Speeding up Spark with DataFrames
Beyond SQL: Speeding up Spark with DataFrames
 
Apache Spark: What? Why? When?
Apache Spark: What? Why? When?Apache Spark: What? Why? When?
Apache Spark: What? Why? When?
 
Scala and spark
Scala and sparkScala and spark
Scala and spark
 
Spark SQL Deep Dive @ Melbourne Spark Meetup
Spark SQL Deep Dive @ Melbourne Spark MeetupSpark SQL Deep Dive @ Melbourne Spark Meetup
Spark SQL Deep Dive @ Melbourne Spark Meetup
 
Spark real world use cases and optimizations
Spark real world use cases and optimizationsSpark real world use cases and optimizations
Spark real world use cases and optimizations
 
Performing Data Science with HBase
Performing Data Science with HBasePerforming Data Science with HBase
Performing Data Science with HBase
 
Sumedh Wale's presentation
Sumedh Wale's presentationSumedh Wale's presentation
Sumedh Wale's presentation
 
Hadoop Architecture
Hadoop ArchitectureHadoop Architecture
Hadoop Architecture
 

Mehr von Alpine Data

Spark Autotuning - Spark Summit East 2017
Spark Autotuning - Spark Summit East 2017 Spark Autotuning - Spark Summit East 2017
Spark Autotuning - Spark Summit East 2017 Alpine Data
 
Big Data Day LA 2017
Big Data Day LA 2017Big Data Day LA 2017
Big Data Day LA 2017Alpine Data
 
Operationalizing Data Science using Cloud Foundry
Operationalizing Data Science using Cloud FoundryOperationalizing Data Science using Cloud Foundry
Operationalizing Data Science using Cloud FoundryAlpine Data
 
UC Berkeley Data Science Webinar
UC Berkeley Data Science WebinarUC Berkeley Data Science Webinar
UC Berkeley Data Science WebinarAlpine Data
 
Enterprise Scale Topological Data Analysis Using Spark
Enterprise Scale Topological Data Analysis Using SparkEnterprise Scale Topological Data Analysis Using Spark
Enterprise Scale Topological Data Analysis Using SparkAlpine Data
 
Spark Tuning for Enterprise System Administrators
Spark Tuning for Enterprise System AdministratorsSpark Tuning for Enterprise System Administrators
Spark Tuning for Enterprise System AdministratorsAlpine Data
 
Real Time Visualization with Spark
Real Time Visualization with SparkReal Time Visualization with Spark
Real Time Visualization with SparkAlpine Data
 
Harnessing Big Data with Spark
Harnessing Big Data with SparkHarnessing Big Data with Spark
Harnessing Big Data with SparkAlpine Data
 

Mehr von Alpine Data (8)

Spark Autotuning - Spark Summit East 2017
Spark Autotuning - Spark Summit East 2017 Spark Autotuning - Spark Summit East 2017
Spark Autotuning - Spark Summit East 2017
 
Big Data Day LA 2017
Big Data Day LA 2017Big Data Day LA 2017
Big Data Day LA 2017
 
Operationalizing Data Science using Cloud Foundry
Operationalizing Data Science using Cloud FoundryOperationalizing Data Science using Cloud Foundry
Operationalizing Data Science using Cloud Foundry
 
UC Berkeley Data Science Webinar
UC Berkeley Data Science WebinarUC Berkeley Data Science Webinar
UC Berkeley Data Science Webinar
 
Enterprise Scale Topological Data Analysis Using Spark
Enterprise Scale Topological Data Analysis Using SparkEnterprise Scale Topological Data Analysis Using Spark
Enterprise Scale Topological Data Analysis Using Spark
 
Spark Tuning for Enterprise System Administrators
Spark Tuning for Enterprise System AdministratorsSpark Tuning for Enterprise System Administrators
Spark Tuning for Enterprise System Administrators
 
Real Time Visualization with Spark
Real Time Visualization with SparkReal Time Visualization with Spark
Real Time Visualization with Spark
 
Harnessing Big Data with Spark
Harnessing Big Data with SparkHarnessing Big Data with Spark
Harnessing Big Data with Spark
 

Kürzlich hochgeladen

Week-01-2.ppt BBB human Computer interaction
Week-01-2.ppt BBB human Computer interactionWeek-01-2.ppt BBB human Computer interaction
Week-01-2.ppt BBB human Computer interactionfulawalesam
 
꧁❤ Greater Noida Call Girls Delhi ❤꧂ 9711199171 ☎️ Hard And Sexy Vip Call
꧁❤ Greater Noida Call Girls Delhi ❤꧂ 9711199171 ☎️ Hard And Sexy Vip Call꧁❤ Greater Noida Call Girls Delhi ❤꧂ 9711199171 ☎️ Hard And Sexy Vip Call
꧁❤ Greater Noida Call Girls Delhi ❤꧂ 9711199171 ☎️ Hard And Sexy Vip Callshivangimorya083
 
BigBuy dropshipping via API with DroFx.pptx
BigBuy dropshipping via API with DroFx.pptxBigBuy dropshipping via API with DroFx.pptx
BigBuy dropshipping via API with DroFx.pptxolyaivanovalion
 
04242024_CCC TUG_Joins and Relationships
04242024_CCC TUG_Joins and Relationships04242024_CCC TUG_Joins and Relationships
04242024_CCC TUG_Joins and Relationshipsccctableauusergroup
 
Invezz.com - Grow your wealth with trading signals
Invezz.com - Grow your wealth with trading signalsInvezz.com - Grow your wealth with trading signals
Invezz.com - Grow your wealth with trading signalsInvezz1
 
(PARI) Call Girls Wanowrie ( 7001035870 ) HI-Fi Pune Escorts Service
(PARI) Call Girls Wanowrie ( 7001035870 ) HI-Fi Pune Escorts Service(PARI) Call Girls Wanowrie ( 7001035870 ) HI-Fi Pune Escorts Service
(PARI) Call Girls Wanowrie ( 7001035870 ) HI-Fi Pune Escorts Serviceranjana rawat
 
Low Rate Call Girls Bhilai Anika 8250192130 Independent Escort Service Bhilai
Low Rate Call Girls Bhilai Anika 8250192130 Independent Escort Service BhilaiLow Rate Call Girls Bhilai Anika 8250192130 Independent Escort Service Bhilai
Low Rate Call Girls Bhilai Anika 8250192130 Independent Escort Service BhilaiSuhani Kapoor
 
Industrialised data - the key to AI success.pdf
Industrialised data - the key to AI success.pdfIndustrialised data - the key to AI success.pdf
Industrialised data - the key to AI success.pdfLars Albertsson
 
Call Girls in Sarai Kale Khan Delhi 💯 Call Us 🔝9205541914 🔝( Delhi) Escorts S...
Call Girls in Sarai Kale Khan Delhi 💯 Call Us 🔝9205541914 🔝( Delhi) Escorts S...Call Girls in Sarai Kale Khan Delhi 💯 Call Us 🔝9205541914 🔝( Delhi) Escorts S...
Call Girls in Sarai Kale Khan Delhi 💯 Call Us 🔝9205541914 🔝( Delhi) Escorts S...Delhi Call girls
 
BabyOno dropshipping via API with DroFx.pptx
BabyOno dropshipping via API with DroFx.pptxBabyOno dropshipping via API with DroFx.pptx
BabyOno dropshipping via API with DroFx.pptxolyaivanovalion
 
Carero dropshipping via API with DroFx.pptx
Carero dropshipping via API with DroFx.pptxCarero dropshipping via API with DroFx.pptx
Carero dropshipping via API with DroFx.pptxolyaivanovalion
 
VIP High Profile Call Girls Amravati Aarushi 8250192130 Independent Escort Se...
VIP High Profile Call Girls Amravati Aarushi 8250192130 Independent Escort Se...VIP High Profile Call Girls Amravati Aarushi 8250192130 Independent Escort Se...
VIP High Profile Call Girls Amravati Aarushi 8250192130 Independent Escort Se...Suhani Kapoor
 
代办国外大学文凭《原版美国UCLA文凭证书》加州大学洛杉矶分校毕业证制作成绩单修改
代办国外大学文凭《原版美国UCLA文凭证书》加州大学洛杉矶分校毕业证制作成绩单修改代办国外大学文凭《原版美国UCLA文凭证书》加州大学洛杉矶分校毕业证制作成绩单修改
代办国外大学文凭《原版美国UCLA文凭证书》加州大学洛杉矶分校毕业证制作成绩单修改atducpo
 
RA-11058_IRR-COMPRESS Do 198 series of 1998
RA-11058_IRR-COMPRESS Do 198 series of 1998RA-11058_IRR-COMPRESS Do 198 series of 1998
RA-11058_IRR-COMPRESS Do 198 series of 1998YohFuh
 
BPAC WITH UFSBI GENERAL PRESENTATION 18_05_2017-1.pptx
BPAC WITH UFSBI GENERAL PRESENTATION 18_05_2017-1.pptxBPAC WITH UFSBI GENERAL PRESENTATION 18_05_2017-1.pptx
BPAC WITH UFSBI GENERAL PRESENTATION 18_05_2017-1.pptxMohammedJunaid861692
 
Log Analysis using OSSEC sasoasasasas.pptx
Log Analysis using OSSEC sasoasasasas.pptxLog Analysis using OSSEC sasoasasasas.pptx
Log Analysis using OSSEC sasoasasasas.pptxJohnnyPlasten
 
Call me @ 9892124323 Cheap Rate Call Girls in Vashi with Real Photo 100% Secure
Call me @ 9892124323  Cheap Rate Call Girls in Vashi with Real Photo 100% SecureCall me @ 9892124323  Cheap Rate Call Girls in Vashi with Real Photo 100% Secure
Call me @ 9892124323 Cheap Rate Call Girls in Vashi with Real Photo 100% SecurePooja Nehwal
 
Apidays Singapore 2024 - Building Digital Trust in a Digital Economy by Veron...
Apidays Singapore 2024 - Building Digital Trust in a Digital Economy by Veron...Apidays Singapore 2024 - Building Digital Trust in a Digital Economy by Veron...
Apidays Singapore 2024 - Building Digital Trust in a Digital Economy by Veron...apidays
 
Smarteg dropshipping via API with DroFx.pptx
Smarteg dropshipping via API with DroFx.pptxSmarteg dropshipping via API with DroFx.pptx
Smarteg dropshipping via API with DroFx.pptxolyaivanovalion
 

Kürzlich hochgeladen (20)

Week-01-2.ppt BBB human Computer interaction
Week-01-2.ppt BBB human Computer interactionWeek-01-2.ppt BBB human Computer interaction
Week-01-2.ppt BBB human Computer interaction
 
꧁❤ Greater Noida Call Girls Delhi ❤꧂ 9711199171 ☎️ Hard And Sexy Vip Call
꧁❤ Greater Noida Call Girls Delhi ❤꧂ 9711199171 ☎️ Hard And Sexy Vip Call꧁❤ Greater Noida Call Girls Delhi ❤꧂ 9711199171 ☎️ Hard And Sexy Vip Call
꧁❤ Greater Noida Call Girls Delhi ❤꧂ 9711199171 ☎️ Hard And Sexy Vip Call
 
BigBuy dropshipping via API with DroFx.pptx
BigBuy dropshipping via API with DroFx.pptxBigBuy dropshipping via API with DroFx.pptx
BigBuy dropshipping via API with DroFx.pptx
 
04242024_CCC TUG_Joins and Relationships
04242024_CCC TUG_Joins and Relationships04242024_CCC TUG_Joins and Relationships
04242024_CCC TUG_Joins and Relationships
 
Invezz.com - Grow your wealth with trading signals
Invezz.com - Grow your wealth with trading signalsInvezz.com - Grow your wealth with trading signals
Invezz.com - Grow your wealth with trading signals
 
(PARI) Call Girls Wanowrie ( 7001035870 ) HI-Fi Pune Escorts Service
(PARI) Call Girls Wanowrie ( 7001035870 ) HI-Fi Pune Escorts Service(PARI) Call Girls Wanowrie ( 7001035870 ) HI-Fi Pune Escorts Service
(PARI) Call Girls Wanowrie ( 7001035870 ) HI-Fi Pune Escorts Service
 
Low Rate Call Girls Bhilai Anika 8250192130 Independent Escort Service Bhilai
Low Rate Call Girls Bhilai Anika 8250192130 Independent Escort Service BhilaiLow Rate Call Girls Bhilai Anika 8250192130 Independent Escort Service Bhilai
Low Rate Call Girls Bhilai Anika 8250192130 Independent Escort Service Bhilai
 
Industrialised data - the key to AI success.pdf
Industrialised data - the key to AI success.pdfIndustrialised data - the key to AI success.pdf
Industrialised data - the key to AI success.pdf
 
Call Girls in Sarai Kale Khan Delhi 💯 Call Us 🔝9205541914 🔝( Delhi) Escorts S...
Call Girls in Sarai Kale Khan Delhi 💯 Call Us 🔝9205541914 🔝( Delhi) Escorts S...Call Girls in Sarai Kale Khan Delhi 💯 Call Us 🔝9205541914 🔝( Delhi) Escorts S...
Call Girls in Sarai Kale Khan Delhi 💯 Call Us 🔝9205541914 🔝( Delhi) Escorts S...
 
BabyOno dropshipping via API with DroFx.pptx
BabyOno dropshipping via API with DroFx.pptxBabyOno dropshipping via API with DroFx.pptx
BabyOno dropshipping via API with DroFx.pptx
 
Carero dropshipping via API with DroFx.pptx
Carero dropshipping via API with DroFx.pptxCarero dropshipping via API with DroFx.pptx
Carero dropshipping via API with DroFx.pptx
 
VIP High Profile Call Girls Amravati Aarushi 8250192130 Independent Escort Se...
VIP High Profile Call Girls Amravati Aarushi 8250192130 Independent Escort Se...VIP High Profile Call Girls Amravati Aarushi 8250192130 Independent Escort Se...
VIP High Profile Call Girls Amravati Aarushi 8250192130 Independent Escort Se...
 
Sampling (random) method and Non random.ppt
Sampling (random) method and Non random.pptSampling (random) method and Non random.ppt
Sampling (random) method and Non random.ppt
 
代办国外大学文凭《原版美国UCLA文凭证书》加州大学洛杉矶分校毕业证制作成绩单修改
代办国外大学文凭《原版美国UCLA文凭证书》加州大学洛杉矶分校毕业证制作成绩单修改代办国外大学文凭《原版美国UCLA文凭证书》加州大学洛杉矶分校毕业证制作成绩单修改
代办国外大学文凭《原版美国UCLA文凭证书》加州大学洛杉矶分校毕业证制作成绩单修改
 
RA-11058_IRR-COMPRESS Do 198 series of 1998
RA-11058_IRR-COMPRESS Do 198 series of 1998RA-11058_IRR-COMPRESS Do 198 series of 1998
RA-11058_IRR-COMPRESS Do 198 series of 1998
 
BPAC WITH UFSBI GENERAL PRESENTATION 18_05_2017-1.pptx
BPAC WITH UFSBI GENERAL PRESENTATION 18_05_2017-1.pptxBPAC WITH UFSBI GENERAL PRESENTATION 18_05_2017-1.pptx
BPAC WITH UFSBI GENERAL PRESENTATION 18_05_2017-1.pptx
 
Log Analysis using OSSEC sasoasasasas.pptx
Log Analysis using OSSEC sasoasasasas.pptxLog Analysis using OSSEC sasoasasasas.pptx
Log Analysis using OSSEC sasoasasasas.pptx
 
Call me @ 9892124323 Cheap Rate Call Girls in Vashi with Real Photo 100% Secure
Call me @ 9892124323  Cheap Rate Call Girls in Vashi with Real Photo 100% SecureCall me @ 9892124323  Cheap Rate Call Girls in Vashi with Real Photo 100% Secure
Call me @ 9892124323 Cheap Rate Call Girls in Vashi with Real Photo 100% Secure
 
Apidays Singapore 2024 - Building Digital Trust in a Digital Economy by Veron...
Apidays Singapore 2024 - Building Digital Trust in a Digital Economy by Veron...Apidays Singapore 2024 - Building Digital Trust in a Digital Economy by Veron...
Apidays Singapore 2024 - Building Digital Trust in a Digital Economy by Veron...
 
Smarteg dropshipping via API with DroFx.pptx
Smarteg dropshipping via API with DroFx.pptxSmarteg dropshipping via API with DroFx.pptx
Smarteg dropshipping via API with DroFx.pptx
 

Think Like Spark

  • 1. Think Like Spark Some Spark Concept & A Use Case
  • 2. Who am I? • Software engineer, data scientist, and Spark enthusiast at Alpine Data (SF Based Analytics Company) • Co – Author High Performance Spark http://shop.oreilly.com/product/0636920046967.do Linked in: https://www.linkedin.com/in/rachelbwarren • Slide share: http://www.slideshare.net/RachelWarren4 • Github : rachelwarren. Code for this talk https://github.com/high-performance-spark/high-performance- spark-examples • Twitter: @warre_n_peace
  • 3. Overview • A little Spark Architecture: How are Spark Jobs Evaluated? Why does that matter for performance? • Execution context: driver, executors, partitions, cores • Spark Application hierarchy: jobs/stages/tasks • Actions vs. Transformations (lazy evaluation) • Wide vs. Narrow Transformations (shuffles & data locality) • Apply what we have learned with four versions of the same algorithm to find rank statistics
  • 4. What is Spark? Distributed computing framework. Must run in tandem with a data storage system - Standalone (For Local Testing) - Cloud (S3, EC2) - Distributed storage, with cluster manager, - (Hadoop Yarn, Apache Messos) Built around and abstraction called RDDs “Resilient, Distributed, Datasets” - Lazily evaluated, immutable, distributed collection of partition objects
  • 5. What happens when I launch a Spark Application?
  • 7. One Spark Executor • One JVM for in memory computations • Partitions care computed on executors • Tasks correspond to partitions • dynamically allocated slots for running tasks (executor cores x executors) • Caching takes up space on executors Partitions / Tasks
  • 8. Implications Two Most common cases of failures 1. Failure during shuffle stage • Moving data between Partitions requires communication with the driver  Failures often occur in the shuffle stage 2. Out of memory errors on executors and driver The driver and each executor have a static amount of memory*  It is easy to run out of memory on the executors or on the driver *dynamic allocation allows changing the number of executors
  • 9. How are Jobs Evaluated? API Call Execution Element Computation to evaluation one partition (combine narrow transforms) Wide transformations (sort, groupByKey) Actions (e.g. collect, saveAsTextFile) Spark Context Object Spark Application Job Stage Task Task Stage Executed in Sequence Executed in Parallel
  • 10. Types Of Spark Operations Actions • RDD  Not RDD • Force execution: Each job ends in exactly one action • Three Kinds • Move data to driver: collect, take, count • Move data to external system Write / Save • Force evaluation: foreach Transformations • RDD  RDD • Lazily evaluated • Can be combined and executed in one pass of the data • Computed on Spark executors
  • 11. Implications of Lazy Evaluation Frustrating: • Debugging =  • Lineage graph is built backwards from action to reading in data or persist/ cache/ checkpoint  if you aren’t careful you will repeat computations * * some times we get help from shuffle files Awesome: • Spark can combine some types of transformations and execute them in a single task • We only compute partitions that we need
  • 12. Types of Transformations Narrow • Never require a shuffle • map, mapPartitions, filter • coalesce* • Input partitions >= output partitions • & output partitions known at design time • A sequence of narrow transformations are combined and executed in one stage as several tasks Wide • May require a shuffle • sort, groupByKey, reduceByKey, join • Requires data movement • Partitioning depends on data it self (not known at design time) • Cause stage boundary: Stage 2 cannot be complete until all the partitions in Stage 1 are computed.
  • 13. Partition Dependencies for input and output partitions Narrow Wide
  • 14. Implications of Shuffles • Narrow transformations are faster/ more parallelizable • Narrow transformation must be written so that they can be computed on any subset of records • Narrow transformations can rely on some partitioning information, (partition remains constant in each stage)* • Wide transformations may distribute data unevenly across machines (for example according to the hash value of the key) *we can loose partitioning information with map or mapPartitions(preservesPartitioner = false)
  • 16. Rank Statistics on Wide Data Design an application that would takes an arbitrary list of longs `n1`...`nk` and return the `nth` best element in each column of a DataFrame of doubles. For example, if the input list is ( 8, 1000, and 20 million), our function would need to return the 8th, 1000th and 20 millionth largest element in each column.
  • 17. Input Data If we were looking for 2 and 4th elements, result would be:
  • 18. V0: Iterative solution Loop through each column: • map to value in the one column • Sort the column • Zip with index and filter for the correct rank statistic (i.e. nth element) • Add the result for each column to a map
  • 19. def findRankStatistics( dataFrame: DataFrame, ranks: List[Long]): Map[Int, Iterable[Double]] = { val numberOfColumns = dataFrame.schema.length var i = 0 var result = Map[Int, Iterable[Double]]() dataFrame.persist() while(i < numberOfColumns){ val col = dataFrame.rdd.map(row => row.getDouble(i)) val sortedCol : RDD[(Double, Long)] = col.sortBy(v => v).zipWithIndex() val ranksOnly = sortedCol.filter{ //rank statistics are indexed from one case (colValue, index) => ranks.contains(index + 1) }.keys val list = ranksOnly.collect() result += (i -> list) i+=1 } result } Persist prevents multiple data reads SortBy is Spark’s sort
  • 20. V0 = Too Many Sorts  • Turtle Picture • One distributed sort per column (800 cols = 800 sorts) • Each of these sorts is executed in sequence • Cannot save partitioning data between sorts 300 Million rows takes days!
  • 21. V1: Parallelize by Column • The work to sort each column can be done without information about the other columns • Can map the data to (column index, value) pairs • GroupByKey on column index • Sort each group • Filter for desired rank statistics
  • 22. Get Col Index, Value Pairs private def getValueColumnPairs(dataFrame : DataFrame): RDD[(Double, Int)] = { dataFrame.rdd.flatMap{ row: Row => row.toSeq.zipWithIndex.map{ case (v, index) => (v.toString.toDouble, index)} } } Flatmap is a narrow transformation Column Index Value 1 15.0 1 2.0 .. …
  • 23. Group By Key Solution • def findRankStatistics( dataFrame: DataFrame , ranks: List[Long]): Map[Int, Iterable[Double]] = { require(ranks.forall(_ > 0)) //Map to column index, value pairs val pairRDD: RDD[(Int, Double)] = mapToKeyValuePairs(dataFrame) val groupColumns: RDD[(Int, Iterable[Double])] = pairRDD.groupByKey() groupColumns.mapValues( iter => { //convert to an array and sort val sortedIter = iter.toArray.sorted sortedIter.toIterable.zipWithIndex.flatMap({ case (colValue, index) => if (ranks.contains(index + 1)) Iterator(colValue) else Iterator.empty }) }).collectAsMap() }
  • 24. V1. Faster on Small Data fails on Big Data 300 K rows = quick 300 M rows = fails
  • 25. Problems with V1 • GroupByKey puts records from all the columns with the same hash value on the same partition THEN loads them into memory • All columns with the same hash value have to fit in memory on each executor • Can’t start the sorting until after the group by key phase has finished
  • 26. V2 : Secondary Sort Style 1. ‘partitionAndSortWithinPartitions’: use the same hash partitioner as GroupByKey - Partition by key and sort all records on each partition - Pushes some of the sorting work done on each partition into the shuffle stage 2. Use mapPartitions to filter for the correct rank statistics - Doesn’t force each column to be stored as an in memory data structure (each partition stays as one iterator) Still fails on 300 M rows
  • 27. Iterator-Iterator-Transformation With Map Partitions • Iterators are not collections. They are a routine for accessing each element • Allows Spark to selectively spill to disk • Don’t need to put all elements into memory In our case: Prevents loading each column into memory after the sorting stage
  • 28. def findRankStatisticsV2(pairRDD: RDD[(Int, Double)], targetRanks: List[Long], partitions : Int ) = { val partitioner = new HashPartitioner(partitions) val sorted = pairRDD .repartitionAndSortWithinPartitions(partitioner) } V2: Secondary Sort Repartition + sort using Hash Partitioner
  • 29. def findRankStatisticsV2(pairRDD: RDD[(Int, Double)], sorted.mapPartitions(iter => { var currentIndex = -1 var elementsPerIndex = 0 val filtered = iter.filter { case (colIndex, value) => if (colIndex != currentIndex) { currentIndex = 1 elementsPerIndex = 1 } else { elementsPerIndex += 1 } targetRanks.contains(elementsPerIndex) } groupSorted(filtered) //groups together ranks in same column }, preservesPartitioning = true) filterForTargetIndex.collectAsMap() } V2: Secondary Sort Iterator-to-iterator transformation
  • 30. V2: Still Fails We don’t have put each column into memory but columns with the same hash value still have to be able to fit on one partition
  • 31. Back to the drawing board • Narrow transformations are quick and easy to parallelize • Partition locality can be retained across narrow transformations • Wide transformations are best with many unique keys. • Using iterator-to-iterator transforms in mapPartitions prevents whole partitions from being loaded into memory • We can rely on shuffle files to prevent re-computation of a wide transformations be several subsequent actions We can solve the problem with one sortByKey and three map partitions
  • 32. V3: Mo Parallel, Mo Better 1. Map to (cell value, column index) pairs 2. Do one very large sortByKey 3. Use mapPartitions to count the values per column on each partition 4. (Locally) using the results of 3 compute location of each rank statistic on each partition 5. Revisit each partition and find the correct rank statistics using the information from step 4. e.g. If the first partition has 10 elements from one column . The13th element will be the third element on the second partition in that column.
  • 33. def findRankStatistics(dataFrame: DataFrame, targetRanks: List[Long]): Map[Int, Iterable[Double]] = { val valueColumnPairs: RDD[(Double, Int)] = getValueColumnPairs(dataFrame) val sortedValueColumnPairs = valueColumnPairs.sortByKey() sortedValueColumnPairs.persist(StorageLevel.MEMORY_AND_DISK) val numOfColumns = dataFrame.schema.length val partitionColumnsFreq = getColumnsFreqPerPartition(sortedValueColumnPairs, numOfColumns) val ranksLocations = getRanksLocationsWithinEachPart(targetRanks, partitionColumnsFreq, numOfColumns) val targetRanksValues = findTargetRanksIteratively(sortedValueColumnPairs, ranksLocations) targetRanksValues.groupByKey().collectAsMap() } Complete code here: https://github.com/high-performance-spark/high-performance-spark- examples/blob/master/src/main/scala/com/high-performance-spark-examples/GoldiLocks/GoldiLocksFirstTry.scala 1. Map to (val, col) pairs 2. Sort 3. Count per partition 4. 5. Filter for element computed in 4
  • 35. V3: Still Blows up! • First partitions show lots of failures and straggler tasks • Jobs lags in the sort stage and fails in final mapPartitions stage More digging reveled data was not evenly distributed
  • 36. Data skew¼ of columns are zero
  • 37. V4: Distinct values per Partition • Instead of mapping from (value, column index pairs), map to ((value, column index), count) pairs on each partition e. g. if on a given partition, there are ten rows with 0.0 in the 2nd column, we could save just one tuple: (0.0, 2), 10) • Use same sort and mapPartitions routines, but adjusted for counts.
  • 38. Different Key column0 column2 2.0 3.0 0.0 3.0 0.0 1.0 0.0 0.0 (value, column Index), count) ((2.0, 0), 1) (2.0,0), 3) (3.0, 1), 2) ….
  • 39. V4: Get (value, o • Code for V4 def getAggregatedValueColumnPairs(dataFrame : DataFrame) : RDD[((Double, Int), Long)] = { val aggregatedValueColumnRDD = dataFrame.rdd.mapPartitions(rows => { val valueColumnMap = new mutable.HashMap[(Double, Int), Long]() rows.foreach(row => { row.toSeq.zipWithIndex.foreach{ case (value, columnIndex) => val key = (value.toString.toDouble, columnIndex) val count = valueColumnMap.getOrElseUpdate(key, 0) valueColumnMap.update(key, count + 1) } }) valueColumnMap.toIterator }) aggregatedValueColumnRDD } Map to ((value, column Index) ,count) Using a hashmap to keep track of uniques
  • 40. Code for V4 • Lots more code to complete the whole algorithm https://github.com/high-performance-spark/high- performance-spark- examples/blob/master/src/main/scala/com/high- performance-spark- examples/GoldiLocks/GoldiLocksWithHashMap.scala
  • 41. V4: Success! • 4 times faster than previous solution on small data • More robust, more parallelizable! Scaling to billions of rows! Happy Goldilocks!
  • 42. Why is V4: Better Advantages • Sorting 75% of original records • Most keys are distinct • No stragglers, easy to parallelize • Can parallelize in many different ways
  • 43. Lessons • Sometimes performance looks ugly • Best unit of parallelization?  Not always the most intuitive • Shuffle Less • Push work into narrow transformations • leverage data locality to prevent shuffles • Shuffle Better • Shuffle fewer records • Use narrow transformations to filter or reduce when possible • Shuffle across keys that are well distributed • Best if records associated with one key fit in memory • Be aware of data skew, know you data
  • 44. Before We Part … • Alpine Data is hiring! Data scientists, engineers (ruby, java, as well as Hadoop/ Scala) , support, technical sales “I continue to be amazed, Alpine has the nicest people ever” – Former alpine engineer http://alpinedata.com/careers/ • Buy my book! http://shop.oreilly.com/product/0636920046967.do Also contact me if you are interested in being a reviewer

Hinweis der Redaktion

  1. Uggly Fix What happens when launch a spark Job
  2. Launches a Spark driver on one node of the distributed system. Launches a number of Spark Executors Each node of a distributed system can contain several executors, but each executor must live on a single node Executors are JVMs that do the computational work to evaluate Spark queries Driver has to coordinate data movement although no data actually goes through the driver
  3. Why this matters: Spark doesn’t store data
  4. 2 Most common cases of fialure
  5. Starting the Spark Context sets up the Spark Execution environment Each Spark Application can contain multiple Jobs Each job is launched by an action in the driver program Each job consists of several stages which correspond to shuffles (wide transformations) Each stage contains tasks which are the computations needed to compute each partition of an RDD (computed on executors)
  6. From an evaluation standpoint, actions are moving data from the Spark executors to either the Spark driver, or evaluating the computation
  7. https://www.youtube.com/watch?v=qmtXcbU7OXA
  8. Point
  9. 1. One of the implications of wide transformations causing stages is that a narrow transformation after a wide transformation cannot start until the wide transformation finished. Some clever operations like tree reduce and aggregate get around this by creatively pushing the work map side 2. Partitions are not loaded into memory unless the computation requires it LOOKKING
  10. Notice this is not an iterator – iterator transform. Some times that is life. Also, we have know partitioning is good This is a REDUCTION. Hashmap is probably smaller the iterator