SlideShare una empresa de Scribd logo
1 de 46
Think Like Spark
Some Spark Concepts & 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
All instructions come from driver (arrows
show instructions not transfer of records)
Cluster manager helps
coordinate actual transfer of
records between nodes
One node may
have several
executors, but
each executor
must fit on one
node
One Spark Executor
• One JVM for in memory
computations / storage
• Partitions care computed on
executors
• Tasks correspond to partitions
• dynamically allocated slots for
running tasks
(max concurrent 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
Moving data between nodes required reading and writing
shuffle files
2. Out of memory errors on executors and driver
The driver and each executor have a static amount of
memory*
*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 computed
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 (depends on value of the key)
• Shuffle files can prevent re-computation
*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
.. …
GroupByKey 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 groupByKey phase
has finished
partitionAndSortWithinPartitions
• Takes a custom partitioner, partitions data according to
that partitioner and then on each partition sorts data by
the implicit ordering of the keys
• Pushes some of the sorting work for each partition into
the shuffle stage
• Partitioning can be different from ordering (e.g. partition
on part of a key)
• SortByKey uses this function with a range partitioner
V2 : Secondary Sort Style
1. Define a custom partitioner which partitions on column
index
2. Map to pairs to ((columnIndex, cellValue), 1) so that the
key is the column index AND cellvalue.
3. Use ‘partitionAndSortWithinPartitions’: with the
custom partitioner to sort all records on each partition
by column index and then value
4. Use mapPartitions to filter for the correct rank statistics
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
• class ColumnIndexPartition(override val numPartitions: Int)
extends Partitioner {
require(numPartitions >= 0, s"Number of partitions " +
s"($numPartitions) cannot be negative.")
override def getPartition(key: Any): Int = {
val k = key.asInstanceOf[(Int, Double)]
Math.abs(k._1) % numPartitions //hashcode of column index
}
}
Define a custom partition which partitions according to
Hash Value of the column index (first half of key)
def findRankStatistics(pairRDD: RDD[(Int, Double)],
targetRanks: List[Long], partitions: Int) = {
val partitioner = new ColumnIndexPartition(partitions)
val sorted = pairRDD.map((_1))
.repartitionAndSortWithinPartitions(
partitioner)
V2: Secondary Sort
Repartition + sort using
Hash Partitioner
val filterForTargetIndex = sorted.mapPartitions(iter => {
var currentColumnIndex = -1
var runningTotal = 0
iter.flatMap({
case (((colIndex, value), _)) =>
if (colIndex != currentColumnIndex) { //new column
//reset to the new column index
currentColumnIndex = colIndex runningTotal = 1
} else {
runningTotal += 1
}
if (targetRanks.contains(runningTotal)) {
Iterator((colIndex, value))
} else {
Iterator.empty
}
})
}, preservesPartitioning = true)
groupSorted(filterForTargetIndex.collect())
}
V2: Secondary Sort
Iterator-to-iterator
transformation
flatMap can be like both
map and filter
V2: Still Fails
We don’t have put each column into memory on on
executor,
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 or fails 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 of records not unique records.
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 before shuffling
• Shuffle across keys that are well distributed
• Best if records associated with one key fit in memory
• Know you data
Before We Part …
• Alpine Data is hiring!
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

Más contenido relacionado

La actualidad más candente

Apache Spark overview
Apache Spark overviewApache Spark overview
Apache Spark overviewDataArt
 
Processing Large Data with Apache Spark -- HasGeek
Processing Large Data with Apache Spark -- HasGeekProcessing Large Data with Apache Spark -- HasGeek
Processing Large Data with Apache Spark -- HasGeekVenkata Naga Ravi
 
BDM25 - Spark runtime internal
BDM25 - Spark runtime internalBDM25 - Spark runtime internal
BDM25 - Spark runtime internalDavid Lauzon
 
Frustration-Reduced PySpark: Data engineering with DataFrames
Frustration-Reduced PySpark: Data engineering with DataFramesFrustration-Reduced PySpark: Data engineering with DataFrames
Frustration-Reduced PySpark: Data engineering with DataFramesIlya Ganelin
 
Spark performance tuning - Maksud Ibrahimov
Spark performance tuning - Maksud IbrahimovSpark performance tuning - Maksud Ibrahimov
Spark performance tuning - Maksud IbrahimovMaksud Ibrahimov
 
Making Sense of Spark Performance-(Kay Ousterhout, UC Berkeley)
Making Sense of Spark Performance-(Kay Ousterhout, UC Berkeley)Making Sense of Spark Performance-(Kay Ousterhout, UC Berkeley)
Making Sense of Spark Performance-(Kay Ousterhout, UC Berkeley)Spark Summit
 
Rapid Prototyping in PySpark Streaming: The Thermodynamics of Docker Containe...
Rapid Prototyping in PySpark Streaming: The Thermodynamics of Docker Containe...Rapid Prototyping in PySpark Streaming: The Thermodynamics of Docker Containe...
Rapid Prototyping in PySpark Streaming: The Thermodynamics of Docker Containe...Richard Seymour
 
Apache Spark Tutorial
Apache Spark TutorialApache Spark Tutorial
Apache Spark TutorialAhmet Bulut
 
Introduction to Apache Spark
Introduction to Apache SparkIntroduction to Apache Spark
Introduction to Apache SparkDatio Big Data
 
Apache Spark: What's under the hood
Apache Spark: What's under the hoodApache Spark: What's under the hood
Apache Spark: What's under the hoodAdarsh Pannu
 
Introduction to Apache Spark
Introduction to Apache SparkIntroduction to Apache Spark
Introduction to Apache SparkSamy Dindane
 
DTCC '14 Spark Runtime Internals
DTCC '14 Spark Runtime InternalsDTCC '14 Spark Runtime Internals
DTCC '14 Spark Runtime InternalsCheng Lian
 
What is Distributed Computing, Why we use Apache Spark
What is Distributed Computing, Why we use Apache SparkWhat is Distributed Computing, Why we use Apache Spark
What is Distributed Computing, Why we use Apache SparkAndy Petrella
 
Apache Spark in Depth: Core Concepts, Architecture & Internals
Apache Spark in Depth: Core Concepts, Architecture & InternalsApache Spark in Depth: Core Concepts, Architecture & Internals
Apache Spark in Depth: Core Concepts, Architecture & InternalsAnton Kirillov
 
Hadoop Spark Introduction-20150130
Hadoop Spark Introduction-20150130Hadoop Spark Introduction-20150130
Hadoop Spark Introduction-20150130Xuan-Chao Huang
 
Introduction to Apache Spark Ecosystem
Introduction to Apache Spark EcosystemIntroduction to Apache Spark Ecosystem
Introduction to Apache Spark EcosystemBojan Babic
 
Spark introduction and architecture
Spark introduction and architectureSpark introduction and architecture
Spark introduction and architectureSohil Jain
 
SORT & JOIN IN SPARK 2.0
SORT & JOIN IN SPARK 2.0SORT & JOIN IN SPARK 2.0
SORT & JOIN IN SPARK 2.0Sigmoid
 
Spark-on-YARN: Empower Spark Applications on Hadoop Cluster
Spark-on-YARN: Empower Spark Applications on Hadoop ClusterSpark-on-YARN: Empower Spark Applications on Hadoop Cluster
Spark-on-YARN: Empower Spark Applications on Hadoop ClusterDataWorks Summit
 

La actualidad más candente (20)

Apache Spark overview
Apache Spark overviewApache Spark overview
Apache Spark overview
 
Processing Large Data with Apache Spark -- HasGeek
Processing Large Data with Apache Spark -- HasGeekProcessing Large Data with Apache Spark -- HasGeek
Processing Large Data with Apache Spark -- HasGeek
 
BDM25 - Spark runtime internal
BDM25 - Spark runtime internalBDM25 - Spark runtime internal
BDM25 - Spark runtime internal
 
Frustration-Reduced PySpark: Data engineering with DataFrames
Frustration-Reduced PySpark: Data engineering with DataFramesFrustration-Reduced PySpark: Data engineering with DataFrames
Frustration-Reduced PySpark: Data engineering with DataFrames
 
Spark performance tuning - Maksud Ibrahimov
Spark performance tuning - Maksud IbrahimovSpark performance tuning - Maksud Ibrahimov
Spark performance tuning - Maksud Ibrahimov
 
Making Sense of Spark Performance-(Kay Ousterhout, UC Berkeley)
Making Sense of Spark Performance-(Kay Ousterhout, UC Berkeley)Making Sense of Spark Performance-(Kay Ousterhout, UC Berkeley)
Making Sense of Spark Performance-(Kay Ousterhout, UC Berkeley)
 
Rapid Prototyping in PySpark Streaming: The Thermodynamics of Docker Containe...
Rapid Prototyping in PySpark Streaming: The Thermodynamics of Docker Containe...Rapid Prototyping in PySpark Streaming: The Thermodynamics of Docker Containe...
Rapid Prototyping in PySpark Streaming: The Thermodynamics of Docker Containe...
 
Apache Spark Tutorial
Apache Spark TutorialApache Spark Tutorial
Apache Spark Tutorial
 
Introduction to Apache Spark
Introduction to Apache SparkIntroduction to Apache Spark
Introduction to Apache Spark
 
Apache Spark: What's under the hood
Apache Spark: What's under the hoodApache Spark: What's under the hood
Apache Spark: What's under the hood
 
Introduction to Apache Spark
Introduction to Apache SparkIntroduction to Apache Spark
Introduction to Apache Spark
 
Apache Spark Architecture
Apache Spark ArchitectureApache Spark Architecture
Apache Spark Architecture
 
DTCC '14 Spark Runtime Internals
DTCC '14 Spark Runtime InternalsDTCC '14 Spark Runtime Internals
DTCC '14 Spark Runtime Internals
 
What is Distributed Computing, Why we use Apache Spark
What is Distributed Computing, Why we use Apache SparkWhat is Distributed Computing, Why we use Apache Spark
What is Distributed Computing, Why we use Apache Spark
 
Apache Spark in Depth: Core Concepts, Architecture & Internals
Apache Spark in Depth: Core Concepts, Architecture & InternalsApache Spark in Depth: Core Concepts, Architecture & Internals
Apache Spark in Depth: Core Concepts, Architecture & Internals
 
Hadoop Spark Introduction-20150130
Hadoop Spark Introduction-20150130Hadoop Spark Introduction-20150130
Hadoop Spark Introduction-20150130
 
Introduction to Apache Spark Ecosystem
Introduction to Apache Spark EcosystemIntroduction to Apache Spark Ecosystem
Introduction to Apache Spark Ecosystem
 
Spark introduction and architecture
Spark introduction and architectureSpark introduction and architecture
Spark introduction and architecture
 
SORT & JOIN IN SPARK 2.0
SORT & JOIN IN SPARK 2.0SORT & JOIN IN SPARK 2.0
SORT & JOIN IN SPARK 2.0
 
Spark-on-YARN: Empower Spark Applications on Hadoop Cluster
Spark-on-YARN: Empower Spark Applications on Hadoop ClusterSpark-on-YARN: Empower Spark Applications on Hadoop Cluster
Spark-on-YARN: Empower Spark Applications on Hadoop Cluster
 

Destacado

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
 
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
 
臺灣高中數學講義 - 第一冊 - 數與式
臺灣高中數學講義 - 第一冊 - 數與式臺灣高中數學講義 - 第一冊 - 數與式
臺灣高中數學講義 - 第一冊 - 數與式Xuan-Chao Huang
 
New Analytics Toolbox DevNexus 2015
New Analytics Toolbox DevNexus 2015New Analytics Toolbox DevNexus 2015
New Analytics Toolbox DevNexus 2015Robbie Strickland
 
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
 
Apache Spark Introduction @ University College London
Apache Spark Introduction @ University College LondonApache Spark Introduction @ University College London
Apache Spark Introduction @ University College LondonVitthal Gogate
 
Think Like Spark
Think Like SparkThink Like Spark
Think Like SparkAlpine Data
 
Escape from Hadoop: Ultra Fast Data Analysis with Spark & Cassandra
Escape from Hadoop: Ultra Fast Data Analysis with Spark & CassandraEscape from Hadoop: Ultra Fast Data Analysis with Spark & Cassandra
Escape from Hadoop: Ultra Fast Data Analysis with Spark & CassandraPiotr Kolaczkowski
 
What’s New in Spark 2.0: Structured Streaming and Datasets - StampedeCon 2016
What’s New in Spark 2.0: Structured Streaming and Datasets - StampedeCon 2016What’s New in Spark 2.0: Structured Streaming and Datasets - StampedeCon 2016
What’s New in Spark 2.0: Structured Streaming and Datasets - StampedeCon 2016StampedeCon
 
Intro to Spark development
 Intro to Spark development  Intro to Spark development
Intro to Spark development Spark Summit
 
Beneath RDD in Apache Spark by Jacek Laskowski
Beneath RDD in Apache Spark by Jacek LaskowskiBeneath RDD in Apache Spark by Jacek Laskowski
Beneath RDD in Apache Spark by Jacek LaskowskiSpark Summit
 
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
 
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
 

Destacado (20)

Resilient Distributed Dataset - Analisis paper
Resilient  Distributed Dataset - Analisis paper Resilient  Distributed Dataset - Analisis paper
Resilient Distributed Dataset - Analisis paper
 
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
 
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
 
臺灣高中數學講義 - 第一冊 - 數與式
臺灣高中數學講義 - 第一冊 - 數與式臺灣高中數學講義 - 第一冊 - 數與式
臺灣高中數學講義 - 第一冊 - 數與式
 
New Analytics Toolbox DevNexus 2015
New Analytics Toolbox DevNexus 2015New Analytics Toolbox DevNexus 2015
New Analytics Toolbox DevNexus 2015
 
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
 
Apache Spark Introduction @ University College London
Apache Spark Introduction @ University College LondonApache Spark Introduction @ University College London
Apache Spark Introduction @ University College London
 
Think Like Spark
Think Like SparkThink Like Spark
Think Like Spark
 
Hadoop to spark_v2
Hadoop to spark_v2Hadoop to spark_v2
Hadoop to spark_v2
 
Escape from Hadoop: Ultra Fast Data Analysis with Spark & Cassandra
Escape from Hadoop: Ultra Fast Data Analysis with Spark & CassandraEscape from Hadoop: Ultra Fast Data Analysis with Spark & Cassandra
Escape from Hadoop: Ultra Fast Data Analysis with Spark & Cassandra
 
Spark in 15 min
Spark in 15 minSpark in 15 min
Spark in 15 min
 
What’s New in Spark 2.0: Structured Streaming and Datasets - StampedeCon 2016
What’s New in Spark 2.0: Structured Streaming and Datasets - StampedeCon 2016What’s New in Spark 2.0: Structured Streaming and Datasets - StampedeCon 2016
What’s New in Spark 2.0: Structured Streaming and Datasets - StampedeCon 2016
 
Intro to Spark development
 Intro to Spark development  Intro to Spark development
Intro to Spark development
 
Beneath RDD in Apache Spark by Jacek Laskowski
Beneath RDD in Apache Spark by Jacek LaskowskiBeneath RDD in Apache Spark by Jacek Laskowski
Beneath RDD in Apache Spark by Jacek Laskowski
 
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
 
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
 

Similar a Think Like Spark: Some Spark Concepts and a Use Case

Dive into spark2
Dive into spark2Dive into spark2
Dive into spark2Gal Marder
 
UiPath Studio Web workshop series - Day 4
UiPath Studio Web workshop series - Day 4UiPath Studio Web workshop series - Day 4
UiPath Studio Web workshop series - Day 4DianaGray10
 
AI與大數據數據處理 Spark實戰(20171216)
AI與大數據數據處理 Spark實戰(20171216)AI與大數據數據處理 Spark實戰(20171216)
AI與大數據數據處理 Spark實戰(20171216)Paul Chao
 
OVERVIEW ON SPARK.pptx
OVERVIEW ON SPARK.pptxOVERVIEW ON SPARK.pptx
OVERVIEW ON SPARK.pptxAishg4
 
SparkSQL: A Compiler from Queries to RDDs
SparkSQL: A Compiler from Queries to RDDsSparkSQL: A Compiler from Queries to RDDs
SparkSQL: A Compiler from Queries to RDDsDatabricks
 
Learnings Using Spark Streaming and DataFrames for Walmart Search: Spark Summ...
Learnings Using Spark Streaming and DataFrames for Walmart Search: Spark Summ...Learnings Using Spark Streaming and DataFrames for Walmart Search: Spark Summ...
Learnings Using Spark Streaming and DataFrames for Walmart Search: Spark Summ...Spark Summit
 
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
 
Data structure and algorithm.
Data structure and algorithm. Data structure and algorithm.
Data structure and algorithm. Abdul salam
 
A full Machine learning pipeline in Scikit-learn vs in scala-Spark: pros and ...
A full Machine learning pipeline in Scikit-learn vs in scala-Spark: pros and ...A full Machine learning pipeline in Scikit-learn vs in scala-Spark: pros and ...
A full Machine learning pipeline in Scikit-learn vs in scala-Spark: pros and ...Jose Quesada (hiring)
 
Implementing IDR in __alloc_fd()
Implementing IDR in __alloc_fd()Implementing IDR in __alloc_fd()
Implementing IDR in __alloc_fd()Sandhya Bankar
 
Be A Hero: Transforming GoPro Analytics Data Pipeline
Be A Hero: Transforming GoPro Analytics Data PipelineBe A Hero: Transforming GoPro Analytics Data Pipeline
Be A Hero: Transforming GoPro Analytics Data PipelineChester Chen
 
AWS (Amazon Redshift) presentation
AWS (Amazon Redshift) presentationAWS (Amazon Redshift) presentation
AWS (Amazon Redshift) presentationVolodymyr Rovetskiy
 
1. Data structures introduction
1. Data structures introduction1. Data structures introduction
1. Data structures introductionMandeep Singh
 
Using Apache Spark as ETL engine. Pros and Cons
Using Apache Spark as ETL engine. Pros and Cons          Using Apache Spark as ETL engine. Pros and Cons
Using Apache Spark as ETL engine. Pros and Cons Provectus
 
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
 
BDAS Shark study report 03 v1.1
BDAS Shark study report  03 v1.1BDAS Shark study report  03 v1.1
BDAS Shark study report 03 v1.1Stefanie Zhao
 

Similar a Think Like Spark: Some Spark Concepts and a Use Case (20)

stack.pptx
stack.pptxstack.pptx
stack.pptx
 
Dive into spark2
Dive into spark2Dive into spark2
Dive into spark2
 
UiPath Studio Web workshop series - Day 4
UiPath Studio Web workshop series - Day 4UiPath Studio Web workshop series - Day 4
UiPath Studio Web workshop series - Day 4
 
AI與大數據數據處理 Spark實戰(20171216)
AI與大數據數據處理 Spark實戰(20171216)AI與大數據數據處理 Spark實戰(20171216)
AI與大數據數據處理 Spark實戰(20171216)
 
OVERVIEW ON SPARK.pptx
OVERVIEW ON SPARK.pptxOVERVIEW ON SPARK.pptx
OVERVIEW ON SPARK.pptx
 
SparkSQL: A Compiler from Queries to RDDs
SparkSQL: A Compiler from Queries to RDDsSparkSQL: A Compiler from Queries to RDDs
SparkSQL: A Compiler from Queries to RDDs
 
Learnings Using Spark Streaming and DataFrames for Walmart Search: Spark Summ...
Learnings Using Spark Streaming and DataFrames for Walmart Search: Spark Summ...Learnings Using Spark Streaming and DataFrames for Walmart Search: Spark Summ...
Learnings Using Spark Streaming and DataFrames for Walmart Search: Spark Summ...
 
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
 
Meetup talk
Meetup talkMeetup talk
Meetup talk
 
Data structure and algorithm.
Data structure and algorithm. Data structure and algorithm.
Data structure and algorithm.
 
A full Machine learning pipeline in Scikit-learn vs in scala-Spark: pros and ...
A full Machine learning pipeline in Scikit-learn vs in scala-Spark: pros and ...A full Machine learning pipeline in Scikit-learn vs in scala-Spark: pros and ...
A full Machine learning pipeline in Scikit-learn vs in scala-Spark: pros and ...
 
DynamodbDB Deep Dive
DynamodbDB Deep DiveDynamodbDB Deep Dive
DynamodbDB Deep Dive
 
Implementing IDR in __alloc_fd()
Implementing IDR in __alloc_fd()Implementing IDR in __alloc_fd()
Implementing IDR in __alloc_fd()
 
Be A Hero: Transforming GoPro Analytics Data Pipeline
Be A Hero: Transforming GoPro Analytics Data PipelineBe A Hero: Transforming GoPro Analytics Data Pipeline
Be A Hero: Transforming GoPro Analytics Data Pipeline
 
AWS (Amazon Redshift) presentation
AWS (Amazon Redshift) presentationAWS (Amazon Redshift) presentation
AWS (Amazon Redshift) presentation
 
1. Data structures introduction
1. Data structures introduction1. Data structures introduction
1. Data structures introduction
 
Using Apache Spark as ETL engine. Pros and Cons
Using Apache Spark as ETL engine. Pros and Cons          Using Apache Spark as ETL engine. Pros and Cons
Using Apache Spark as ETL engine. Pros and Cons
 
Hadoop map reduce concepts
Hadoop map reduce conceptsHadoop map reduce concepts
Hadoop map reduce concepts
 
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
 
BDAS Shark study report 03 v1.1
BDAS Shark study report  03 v1.1BDAS Shark study report  03 v1.1
BDAS Shark study report 03 v1.1
 

Último

%in Midrand+277-882-255-28 abortion pills for sale in midrand
%in Midrand+277-882-255-28 abortion pills for sale in midrand%in Midrand+277-882-255-28 abortion pills for sale in midrand
%in Midrand+277-882-255-28 abortion pills for sale in midrandmasabamasaba
 
8257 interfacing 2 in microprocessor for btech students
8257 interfacing 2 in microprocessor for btech students8257 interfacing 2 in microprocessor for btech students
8257 interfacing 2 in microprocessor for btech studentsHimanshiGarg82
 
Payment Gateway Testing Simplified_ A Step-by-Step Guide for Beginners.pdf
Payment Gateway Testing Simplified_ A Step-by-Step Guide for Beginners.pdfPayment Gateway Testing Simplified_ A Step-by-Step Guide for Beginners.pdf
Payment Gateway Testing Simplified_ A Step-by-Step Guide for Beginners.pdfkalichargn70th171
 
call girls in Vaishali (Ghaziabad) 🔝 >༒8448380779 🔝 genuine Escort Service 🔝✔️✔️
call girls in Vaishali (Ghaziabad) 🔝 >༒8448380779 🔝 genuine Escort Service 🔝✔️✔️call girls in Vaishali (Ghaziabad) 🔝 >༒8448380779 🔝 genuine Escort Service 🔝✔️✔️
call girls in Vaishali (Ghaziabad) 🔝 >༒8448380779 🔝 genuine Escort Service 🔝✔️✔️Delhi Call girls
 
%in kempton park+277-882-255-28 abortion pills for sale in kempton park
%in kempton park+277-882-255-28 abortion pills for sale in kempton park %in kempton park+277-882-255-28 abortion pills for sale in kempton park
%in kempton park+277-882-255-28 abortion pills for sale in kempton park masabamasaba
 
tonesoftg
tonesoftgtonesoftg
tonesoftglanshi9
 
WSO2Con2024 - Enabling Transactional System's Exponential Growth With Simplicity
WSO2Con2024 - Enabling Transactional System's Exponential Growth With SimplicityWSO2Con2024 - Enabling Transactional System's Exponential Growth With Simplicity
WSO2Con2024 - Enabling Transactional System's Exponential Growth With SimplicityWSO2
 
%in Soweto+277-882-255-28 abortion pills for sale in soweto
%in Soweto+277-882-255-28 abortion pills for sale in soweto%in Soweto+277-882-255-28 abortion pills for sale in soweto
%in Soweto+277-882-255-28 abortion pills for sale in sowetomasabamasaba
 
OpenChain - The Ramifications of ISO/IEC 5230 and ISO/IEC 18974 for Legal Pro...
OpenChain - The Ramifications of ISO/IEC 5230 and ISO/IEC 18974 for Legal Pro...OpenChain - The Ramifications of ISO/IEC 5230 and ISO/IEC 18974 for Legal Pro...
OpenChain - The Ramifications of ISO/IEC 5230 and ISO/IEC 18974 for Legal Pro...Shane Coughlan
 
%+27788225528 love spells in Boston Psychic Readings, Attraction spells,Bring...
%+27788225528 love spells in Boston Psychic Readings, Attraction spells,Bring...%+27788225528 love spells in Boston Psychic Readings, Attraction spells,Bring...
%+27788225528 love spells in Boston Psychic Readings, Attraction spells,Bring...masabamasaba
 
AI Mastery 201: Elevating Your Workflow with Advanced LLM Techniques
AI Mastery 201: Elevating Your Workflow with Advanced LLM TechniquesAI Mastery 201: Elevating Your Workflow with Advanced LLM Techniques
AI Mastery 201: Elevating Your Workflow with Advanced LLM TechniquesVictorSzoltysek
 
%+27788225528 love spells in new york Psychic Readings, Attraction spells,Bri...
%+27788225528 love spells in new york Psychic Readings, Attraction spells,Bri...%+27788225528 love spells in new york Psychic Readings, Attraction spells,Bri...
%+27788225528 love spells in new york Psychic Readings, Attraction spells,Bri...masabamasaba
 
%in Hazyview+277-882-255-28 abortion pills for sale in Hazyview
%in Hazyview+277-882-255-28 abortion pills for sale in Hazyview%in Hazyview+277-882-255-28 abortion pills for sale in Hazyview
%in Hazyview+277-882-255-28 abortion pills for sale in Hazyviewmasabamasaba
 
WSO2CON2024 - It's time to go Platformless
WSO2CON2024 - It's time to go PlatformlessWSO2CON2024 - It's time to go Platformless
WSO2CON2024 - It's time to go PlatformlessWSO2
 
call girls in Vaishali (Ghaziabad) 🔝 >༒8448380779 🔝 genuine Escort Service 🔝✔️✔️
call girls in Vaishali (Ghaziabad) 🔝 >༒8448380779 🔝 genuine Escort Service 🔝✔️✔️call girls in Vaishali (Ghaziabad) 🔝 >༒8448380779 🔝 genuine Escort Service 🔝✔️✔️
call girls in Vaishali (Ghaziabad) 🔝 >༒8448380779 🔝 genuine Escort Service 🔝✔️✔️Delhi Call girls
 
AI & Machine Learning Presentation Template
AI & Machine Learning Presentation TemplateAI & Machine Learning Presentation Template
AI & Machine Learning Presentation TemplatePresentation.STUDIO
 
%in Harare+277-882-255-28 abortion pills for sale in Harare
%in Harare+277-882-255-28 abortion pills for sale in Harare%in Harare+277-882-255-28 abortion pills for sale in Harare
%in Harare+277-882-255-28 abortion pills for sale in Hararemasabamasaba
 
Crypto Cloud Review - How To Earn Up To $500 Per DAY Of Bitcoin 100% On AutoP...
Crypto Cloud Review - How To Earn Up To $500 Per DAY Of Bitcoin 100% On AutoP...Crypto Cloud Review - How To Earn Up To $500 Per DAY Of Bitcoin 100% On AutoP...
Crypto Cloud Review - How To Earn Up To $500 Per DAY Of Bitcoin 100% On AutoP...SelfMade bd
 
%in kaalfontein+277-882-255-28 abortion pills for sale in kaalfontein
%in kaalfontein+277-882-255-28 abortion pills for sale in kaalfontein%in kaalfontein+277-882-255-28 abortion pills for sale in kaalfontein
%in kaalfontein+277-882-255-28 abortion pills for sale in kaalfonteinmasabamasaba
 
Define the academic and professional writing..pdf
Define the academic and professional writing..pdfDefine the academic and professional writing..pdf
Define the academic and professional writing..pdfPearlKirahMaeRagusta1
 

Último (20)

%in Midrand+277-882-255-28 abortion pills for sale in midrand
%in Midrand+277-882-255-28 abortion pills for sale in midrand%in Midrand+277-882-255-28 abortion pills for sale in midrand
%in Midrand+277-882-255-28 abortion pills for sale in midrand
 
8257 interfacing 2 in microprocessor for btech students
8257 interfacing 2 in microprocessor for btech students8257 interfacing 2 in microprocessor for btech students
8257 interfacing 2 in microprocessor for btech students
 
Payment Gateway Testing Simplified_ A Step-by-Step Guide for Beginners.pdf
Payment Gateway Testing Simplified_ A Step-by-Step Guide for Beginners.pdfPayment Gateway Testing Simplified_ A Step-by-Step Guide for Beginners.pdf
Payment Gateway Testing Simplified_ A Step-by-Step Guide for Beginners.pdf
 
call girls in Vaishali (Ghaziabad) 🔝 >༒8448380779 🔝 genuine Escort Service 🔝✔️✔️
call girls in Vaishali (Ghaziabad) 🔝 >༒8448380779 🔝 genuine Escort Service 🔝✔️✔️call girls in Vaishali (Ghaziabad) 🔝 >༒8448380779 🔝 genuine Escort Service 🔝✔️✔️
call girls in Vaishali (Ghaziabad) 🔝 >༒8448380779 🔝 genuine Escort Service 🔝✔️✔️
 
%in kempton park+277-882-255-28 abortion pills for sale in kempton park
%in kempton park+277-882-255-28 abortion pills for sale in kempton park %in kempton park+277-882-255-28 abortion pills for sale in kempton park
%in kempton park+277-882-255-28 abortion pills for sale in kempton park
 
tonesoftg
tonesoftgtonesoftg
tonesoftg
 
WSO2Con2024 - Enabling Transactional System's Exponential Growth With Simplicity
WSO2Con2024 - Enabling Transactional System's Exponential Growth With SimplicityWSO2Con2024 - Enabling Transactional System's Exponential Growth With Simplicity
WSO2Con2024 - Enabling Transactional System's Exponential Growth With Simplicity
 
%in Soweto+277-882-255-28 abortion pills for sale in soweto
%in Soweto+277-882-255-28 abortion pills for sale in soweto%in Soweto+277-882-255-28 abortion pills for sale in soweto
%in Soweto+277-882-255-28 abortion pills for sale in soweto
 
OpenChain - The Ramifications of ISO/IEC 5230 and ISO/IEC 18974 for Legal Pro...
OpenChain - The Ramifications of ISO/IEC 5230 and ISO/IEC 18974 for Legal Pro...OpenChain - The Ramifications of ISO/IEC 5230 and ISO/IEC 18974 for Legal Pro...
OpenChain - The Ramifications of ISO/IEC 5230 and ISO/IEC 18974 for Legal Pro...
 
%+27788225528 love spells in Boston Psychic Readings, Attraction spells,Bring...
%+27788225528 love spells in Boston Psychic Readings, Attraction spells,Bring...%+27788225528 love spells in Boston Psychic Readings, Attraction spells,Bring...
%+27788225528 love spells in Boston Psychic Readings, Attraction spells,Bring...
 
AI Mastery 201: Elevating Your Workflow with Advanced LLM Techniques
AI Mastery 201: Elevating Your Workflow with Advanced LLM TechniquesAI Mastery 201: Elevating Your Workflow with Advanced LLM Techniques
AI Mastery 201: Elevating Your Workflow with Advanced LLM Techniques
 
%+27788225528 love spells in new york Psychic Readings, Attraction spells,Bri...
%+27788225528 love spells in new york Psychic Readings, Attraction spells,Bri...%+27788225528 love spells in new york Psychic Readings, Attraction spells,Bri...
%+27788225528 love spells in new york Psychic Readings, Attraction spells,Bri...
 
%in Hazyview+277-882-255-28 abortion pills for sale in Hazyview
%in Hazyview+277-882-255-28 abortion pills for sale in Hazyview%in Hazyview+277-882-255-28 abortion pills for sale in Hazyview
%in Hazyview+277-882-255-28 abortion pills for sale in Hazyview
 
WSO2CON2024 - It's time to go Platformless
WSO2CON2024 - It's time to go PlatformlessWSO2CON2024 - It's time to go Platformless
WSO2CON2024 - It's time to go Platformless
 
call girls in Vaishali (Ghaziabad) 🔝 >༒8448380779 🔝 genuine Escort Service 🔝✔️✔️
call girls in Vaishali (Ghaziabad) 🔝 >༒8448380779 🔝 genuine Escort Service 🔝✔️✔️call girls in Vaishali (Ghaziabad) 🔝 >༒8448380779 🔝 genuine Escort Service 🔝✔️✔️
call girls in Vaishali (Ghaziabad) 🔝 >༒8448380779 🔝 genuine Escort Service 🔝✔️✔️
 
AI & Machine Learning Presentation Template
AI & Machine Learning Presentation TemplateAI & Machine Learning Presentation Template
AI & Machine Learning Presentation Template
 
%in Harare+277-882-255-28 abortion pills for sale in Harare
%in Harare+277-882-255-28 abortion pills for sale in Harare%in Harare+277-882-255-28 abortion pills for sale in Harare
%in Harare+277-882-255-28 abortion pills for sale in Harare
 
Crypto Cloud Review - How To Earn Up To $500 Per DAY Of Bitcoin 100% On AutoP...
Crypto Cloud Review - How To Earn Up To $500 Per DAY Of Bitcoin 100% On AutoP...Crypto Cloud Review - How To Earn Up To $500 Per DAY Of Bitcoin 100% On AutoP...
Crypto Cloud Review - How To Earn Up To $500 Per DAY Of Bitcoin 100% On AutoP...
 
%in kaalfontein+277-882-255-28 abortion pills for sale in kaalfontein
%in kaalfontein+277-882-255-28 abortion pills for sale in kaalfontein%in kaalfontein+277-882-255-28 abortion pills for sale in kaalfontein
%in kaalfontein+277-882-255-28 abortion pills for sale in kaalfontein
 
Define the academic and professional writing..pdf
Define the academic and professional writing..pdfDefine the academic and professional writing..pdf
Define the academic and professional writing..pdf
 

Think Like Spark: Some Spark Concepts and a Use Case

  • 1. Think Like Spark Some Spark Concepts & 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?
  • 6. Spark Driver ExecutorExecutor Executor Executor Stable storage e.g. HDFS All instructions come from driver (arrows show instructions not transfer of records) Cluster manager helps coordinate actual transfer of records between nodes One node may have several executors, but each executor must fit on one node
  • 7. One Spark Executor • One JVM for in memory computations / storage • Partitions care computed on executors • Tasks correspond to partitions • dynamically allocated slots for running tasks (max concurrent 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 Moving data between nodes required reading and writing shuffle files 2. Out of memory errors on executors and driver The driver and each executor have a static amount of memory* *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 computed 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 (depends on value of the key) • Shuffle files can prevent re-computation *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. GroupByKey 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 groupByKey phase has finished
  • 26. partitionAndSortWithinPartitions • Takes a custom partitioner, partitions data according to that partitioner and then on each partition sorts data by the implicit ordering of the keys • Pushes some of the sorting work for each partition into the shuffle stage • Partitioning can be different from ordering (e.g. partition on part of a key) • SortByKey uses this function with a range partitioner
  • 27. V2 : Secondary Sort Style 1. Define a custom partitioner which partitions on column index 2. Map to pairs to ((columnIndex, cellValue), 1) so that the key is the column index AND cellvalue. 3. Use ‘partitionAndSortWithinPartitions’: with the custom partitioner to sort all records on each partition by column index and then value 4. Use mapPartitions to filter for the correct rank statistics
  • 28. 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
  • 29. • class ColumnIndexPartition(override val numPartitions: Int) extends Partitioner { require(numPartitions >= 0, s"Number of partitions " + s"($numPartitions) cannot be negative.") override def getPartition(key: Any): Int = { val k = key.asInstanceOf[(Int, Double)] Math.abs(k._1) % numPartitions //hashcode of column index } } Define a custom partition which partitions according to Hash Value of the column index (first half of key)
  • 30. def findRankStatistics(pairRDD: RDD[(Int, Double)], targetRanks: List[Long], partitions: Int) = { val partitioner = new ColumnIndexPartition(partitions) val sorted = pairRDD.map((_1)) .repartitionAndSortWithinPartitions( partitioner) V2: Secondary Sort Repartition + sort using Hash Partitioner
  • 31. val filterForTargetIndex = sorted.mapPartitions(iter => { var currentColumnIndex = -1 var runningTotal = 0 iter.flatMap({ case (((colIndex, value), _)) => if (colIndex != currentColumnIndex) { //new column //reset to the new column index currentColumnIndex = colIndex runningTotal = 1 } else { runningTotal += 1 } if (targetRanks.contains(runningTotal)) { Iterator((colIndex, value)) } else { Iterator.empty } }) }, preservesPartitioning = true) groupSorted(filterForTargetIndex.collect()) } V2: Secondary Sort Iterator-to-iterator transformation flatMap can be like both map and filter
  • 32. V2: Still Fails We don’t have put each column into memory on on executor, but columns with the same hash value still have to be able to fit on one partition
  • 33. 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
  • 34. 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.
  • 35. 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
  • 37. V3: Still Blows up! • First partitions show lots of failures and straggler tasks • Jobs lags or fails in the sort stage and fails in final mapPartitions stage More digging reveled data was not evenly distributed
  • 38. Data skew¼ of columns are zero
  • 39. 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 of records not unique records.
  • 40. 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) ….
  • 41. 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
  • 42. 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
  • 43. V4: Success! • 4 times faster than previous solution on small data • More robust, more parallelizable! Scaling to billions of rows! Happy Goldilocks!
  • 44. 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
  • 45. 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 before shuffling • Shuffle across keys that are well distributed • Best if records associated with one key fit in memory • Know you data
  • 46. Before We Part … • Alpine Data is hiring! 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

Notas del editor

  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 Arrows are meant to show not Driver has to coordinate data movement although no data actually goes through the driver
  3. One spark executor is one JVM that evaluates RDDs. Recall that RDDs are distributed collection of partition objects. These partitions are evaluated on the executors. We say that the work to evaluate one partition (one part of an RDD) is one Task Each executor has a dynamically allocated number of slots for running tasks. Thus, the number of parallel computations that spark can do at one time is equal to executor cores * executors In addition to the compute layer, the executor has some space allocated for caching data (e..g storing partitions in memory). Note: Caching is not free!!! It takes up space on executors
  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
  10. On one executor we still need the resources to write and read a shuffle file for all the records associated with one key.
  11. 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 If a hundred percent of the data is distinct this solution is worse.