SlideShare a Scribd company logo
TRANSFORMATIONS AND ACTIONS
A Visual Guide of the APIhttp://training.databricks.com/visualapi.pdf
Databricks would like to give a special thanks to Jeff Thomspon for contributing 67
visual diagrams depicting the Spark API under the MIT license to the Spark
community.
Jeff’s original, creative work can be found here and you can read more about
Jeff’s project in his blog post.
After talking to Jeff, Databricks commissioned Adam Breindel to further evolve
Jeff’s work into the diagrams you see in this deck.
LinkedIn
Blog: data-frack
making big data simple
Databricks Cloud:
“A unified platform for building Big Data pipelines – from ETL to
Exploration and Dashboards, to Advanced Analytics and Data
Products.”
• Founded in late 2013
• by the creators of Apache Spark
• Original team from UC Berkeley AMPLab
• Raised $47 Million in 2 rounds
• ~55 employees
• We’re hiring!
• Level 2/3 support partnerships with
• Hortonworks
• MapR
• DataStax
(http://databricks.workable.com)
key
RDD Elements
original item
transformed
type
object on driver
RDD
partition(s) A
B
user functions
user input
input
emitted value
Legend
Randomized operation
Legend
Set Theory / Relational operation
Numeric calculation
Operations =
TRANSFORMATIONS
ACTIONS
+
• map
• filter
• flatMap
• mapPartitions
• mapPartitionsWithIndex
• groupBy
• sortBy
= medium
Essential Core & Intermediate Spark OperationsTRANSFORMATIONSACTIONS
General
• sample
• randomSplit
Math / Statistical
= easy
Set Theory / Relational
• union
• intersection
• subtract
• distinct
• cartesian
• zip
• takeOrdered
Data Structure / I/O
• saveAsTextFile
• saveAsSequenceFile
• saveAsObjectFile
• saveAsHadoopDataset
• saveAsHadoopFile
• saveAsNewAPIHadoopDataset
• saveAsNewAPIHadoopFile
• keyBy
• zipWithIndex
• zipWithUniqueID
• zipPartitions
• coalesce
• repartition
• repartitionAndSortWithinPartitions
• pipe
• count
• takeSample
• max
• min
• sum
• histogram
• mean
• variance
• stdev
• sampleVariance
• countApprox
• countApproxDistinct
• reduce
• collect
• aggregate
• fold
• first
• take
• forEach
• top
• treeAggregate
• treeReduce
• forEachPartition
• collectAsMap
= medium
Essential Core & Intermediate PairRDD OperationsTRANSFORMATIONSACTIONS
General
• sampleByKey
Math / Statistical
= easy
Set Theory / Relational Data Structure
• keys
• values
• partitionBy
• countByKey
• countByValue
• countByValueApprox
• countApproxDistinctByKey
• countApproxDistinctByKey
• countByKeyApprox
• sampleByKeyExact
• cogroup (=groupWith)
• join
• subtractByKey
• fullOuterJoin
• leftOuterJoin
• rightOuterJoin
• flatMapValues
• groupByKey
• reduceByKey
• reduceByKeyLocally
• foldByKey
• aggregateByKey
• sortByKey
• combineByKey
vs
narrow wide
each partition of the parent RDD is used by
at most one partition of the child RDD
multiple child RDD partitions may depend
on a single parent RDD partition
“One of the challenges in providing RDDs as an abstraction is choosing a
representation for them that can track lineage across a wide range of
transformations.”
“The most interesting question in designing this interface is how to represent
dependencies between RDDs.”
“We found it both sufficient and useful to classify dependencies into two types:
• narrow dependencies, where each partition of the parent RDD is used by at
most one partition of the child RDD
• wide dependencies, where multiple child partitions may depend on it.”
narrow wide
each partition of the parent RDD is used by
at most one partition of the child RDD
multiple child RDD partitions may depend
on a single parent RDD partition
map, filter union
join w/ inputs
co-partitioned
groupByKey
join w/ inputs not
co-partitioned
TRANSFORMATIONS Core Operations
MAP
3 items in RDD
RDD: x
MAP
User function
applied item by item
RDD: x RDD: y
MAP
RDD: x RDD: y
MAP
RDD: x RDD: y
MAP
RDD: x RDD: y
After map() has been applied…
before after
MAP
RDD: x RDD: y
Return a new RDD by applying a function to each element of this RDD.
MAP
x = sc.parallelize(["b", "a", "c"])
y = x.map(lambda z: (z, 1))
print(x.collect())
print(y.collect())
['b', 'a', 'c']
[('b', 1), ('a', 1), ('c', 1)]
RDD: x RDD: y
x:
y:
map(f, preservesPartitioning=False)
Return a new RDD by applying a function to each element of this RDD
val x = sc.parallelize(Array("b", "a", "c"))
val y = x.map(z => (z,1))
println(x.collect().mkString(", "))
println(y.collect().mkString(", "))
FILTER
3 items in RDD
RDD: x
FILTER
Apply user function:
keep item if function
returns true
RDD: x RDD: y
emits
True
FILTER
RDD: x RDD: y
emits
False
FILTER
RDD: x RDD: y
emits
True
FILTER
RDD: x RDD: y
After filter() has been applied…
before after
FILTER
x = sc.parallelize([1,2,3])
y = x.filter(lambda x: x%2 == 1) #keep odd values
print(x.collect())
print(y.collect())
[1, 2, 3]
[1, 3]
RDD: x RDD: y
x:
y:
filter(f)
Return a new RDD containing only the elements that satisfy a predicate
val x = sc.parallelize(Array(1,2,3))
val y = x.filter(n => n%2 == 1)
println(x.collect().mkString(", "))
println(y.collect().mkString(", "))
FLATMAP
3 items in RDD
RDD: x
FLATMAP
RDD: x RDD: y
FLATMAP
RDD: x RDD: y
FLATMAP
RDD: x RDD: y
FLATMAP
RDD: x RDD: y
After flatmap() has been applied…
before after
FLATMAP
RDD: x RDD: y
Return a new RDD by first applying a function to all elements of this RDD, and then flattening the results
FLATMAP
x = sc.parallelize([1,2,3])
y = x.flatMap(lambda x: (x, x*100, 42))
print(x.collect())
print(y.collect())
[1, 2, 3]
[1, 100, 42, 2, 200, 42, 3, 300, 42]
x:
y:
RDD: x RDD: y
flatMap(f, preservesPartitioning=False)
Return a new RDD by first applying a function to all elements of this RDD, and then flattening the results
val x = sc.parallelize(Array(1,2,3))
val y = x.flatMap(n => Array(n, n*100, 42))
println(x.collect().mkString(", "))
println(y.collect().mkString(", "))
GROUPBY
4 items in RDD
RDD: x
James
Anna
Fred
John
GROUPBY
RDD: x
James
Anna
Fred
John
emits‘J’
J [ “John” ]
RDD: y
F [ “Fred” ]
GROUPBY
RDD: x
James
Anna
Fred
emits‘F’
J [ “John” ]John
RDD: y
[ “Fred” ]
GROUPBY
RDD: x
James
Anna
emits‘A’
J [ “John” ]
A [ “Anna” ]
Fred
John
F
RDD: y
[ “Fred” ]
GROUPBY
RDD: x
James
Anna
emits‘J’
J [ “John”, “James” ]
[ “Anna” ]
Fred
John
F
A
RDD: y
GROUPBY
x = sc.parallelize(['John', 'Fred', 'Anna', 'James'])
y = x.groupBy(lambda w: w[0])
print [(k, list(v)) for (k, v) in y.collect()]
['John', 'Fred', 'Anna', 'James']
[('A',['Anna']),('J',['John','James']),('F',['Fred'])]
RDD: x RDD: y
x:
y:
groupBy(f, numPartitions=None)
Group the data in the original RDD. Create pairs where the key is the output of
a user function, and the value is all items for which the function yields this key.
val x = sc.parallelize(
Array("John", "Fred", "Anna", "James"))
val y = x.groupBy(w => w.charAt(0))
println(y.collect().mkString(", "))
GROUPBYKEY
5 items in RDD
Pair RDD: x
B
B
A
A
A
5
4
3
2
1
GROUPBYKEY
Pair RDD: x
5
4
3
2
1
RDD: y
A [ 2 , 3 , 1 ]
B
B
A
A
A
GROUPBYKEY
Pair RDD: x RDD: y
B [ 5 , 4 ]
A [ 2 , 3 , 1 ]
5
4
3
2
1
B
B
A
A
A
GROUPBYKEY
x = sc.parallelize([('B',5),('B',4),('A',3),('A',2),('A',1)])
y = x.groupByKey()
print(x.collect())
print(list((j[0], list(j[1])) for j in y.collect()))
[('B', 5),('B', 4),('A', 3),('A', 2),('A', 1)]
[('A', [2, 3, 1]),('B',[5, 4])]
RDD: x RDD: y
x:
y:
groupByKey(numPartitions=None)
Group the values for each key in the original RDD. Create a new pair where the
original key corresponds to this collected group of values.
val x = sc.parallelize(
Array(('B',5),('B',4),('A',3),('A',2),('A',1)))
val y = x.groupByKey()
println(x.collect().mkString(", "))
println(y.collect().mkString(", "))
MAPPARTITIONS
RDD: x RDD: y
partitions
A
B
A
B
REDUCEBYKEY VS GROUPBYKEY
val words = Array("one", "two", "two", "three", "three", "three")
val wordPairsRDD = sc.parallelize(words).map(word => (word, 1))
val wordCountsWithReduce = wordPairsRDD
.reduceByKey(_ + _)
.collect()
val wordCountsWithGroup = wordPairsRDD
.groupByKey()
.map(t => (t._1, t._2.sum))
.collect()
REDUCEBYKEY
(a, 1)
(b, 1)
(a, 1)
(b, 1)
(a, 1)
(a, 1) (a, 2)
(b, 2)(b, 1)
(b, 1)
(a, 1)
(a, 1)
(a, 3)
(b, 2)
(a, 1)
(b, 1)
(b, 1)
(a, 1)
(a, 2) (a, 6)
(a, 3)
(b, 1)
(b, 2) (b, 5)
(b, 2)
a b
GROUPBYKEY
(a, 1)
(b, 1)
(a, 1)
(a, 1)
(b, 1)
(b, 1)
(a, 1)
(a, 1)
(a, 1)
(b, 1)
(b, 1)
(a, 1)
(a, 1)
(a, 6)
(a, 1)
(b, 5)
a b
(a, 1)
(a, 1)
(a, 1)
(b, 1)
(b, 1)
(b, 1)
(b, 1)
(b, 1)
MAPPARTITIONS
x:
y:
mapPartitions(f, preservesPartitioning=False)
Return a new RDD by applying a function to each partition of this RDD
A
B
A
B
x = sc.parallelize([1,2,3], 2)
def f(iterator): yield sum(iterator); yield 42
y = x.mapPartitions(f)
# glom() flattens elements on the same partition
print(x.glom().collect())
print(y.glom().collect())
[[1], [2, 3]]
[[1, 42], [5, 42]]
MAPPARTITIONS
x:
y:
mapPartitions(f, preservesPartitioning=False)
Return a new RDD by applying a function to each partition of this RDD
A
B
A
B
Array(Array(1), Array(2, 3))
Array(Array(1, 42), Array(5, 42))
val x = sc.parallelize(Array(1,2,3), 2)
def f(i:Iterator[Int])={ (i.sum,42).productIterator }
val y = x.mapPartitions(f)
// glom() flattens elements on the same partition
val xOut = x.glom().collect()
val yOut = y.glom().collect()
MAPPARTITIONSWITHINDEX
RDD: x RDD: y
partitions
A
B
A
B
input
partition index
x:
y:
mapPartitionsWithIndex(f, preservesPartitioning=False)
Return a new RDD by applying a function to each partition of this RDD,
while tracking the index of the original partition
A
B
A
B
x = sc.parallelize([1,2,3], 2)
def f(partitionIndex, iterator): yield (partitionIndex, sum(iterator))
y = x.mapPartitionsWithIndex(f)
# glom() flattens elements on the same partition
print(x.glom().collect())
print(y.glom().collect())
[[1], [2, 3]]
[[0, 1], [1, 5]]
MAPPARTITIONSWITHINDEX
partition index
B A
x:
y:
mapPartitionsWithIndex(f, preservesPartitioning=False)
Return a new RDD by applying a function to each partition of this RDD,
while tracking the index of the original partition.
A
B
A
B
Array(Array(1), Array(2, 3))
Array(Array(0, 1), Array(1, 5))
MAPPARTITIONSWITHINDEX
partition index
B A
val x = sc.parallelize(Array(1,2,3), 2)
def f(partitionIndex:Int, i:Iterator[Int]) = {
(partitionIndex, i.sum).productIterator
}
val y = x.mapPartitionsWithIndex(f)
// glom() flattens elements on the same partition
val xOut = x.glom().collect()
val yOut = y.glom().collect()
SAMPLE
RDD: x RDD: y
1
3
5
4
3
2
1
SAMPLE
x = sc.parallelize([1, 2, 3, 4, 5])
y = x.sample(False, 0.4, 42)
print(x.collect())
print(y.collect())
[1, 2, 3, 4, 5]
[1, 3]
RDD: x RDD: y
x:
y:
sample(withReplacement, fraction, seed=None)
Return a new RDD containing a statistical sample of the original RDD
val x = sc.parallelize(Array(1, 2, 3, 4, 5))
val y = x.sample(false, 0.4)
// omitting seed will yield different output
println(y.collect().mkString(", "))
UNION RDD: x RDD: y
4
3
3
2
1
A
B
C
4
3
3
2
1
A
B
C
RDD: z
UNION
x = sc.parallelize([1,2,3], 2)
y = sc.parallelize([3,4], 1)
z = x.union(y)
print(z.glom().collect())
[1, 2, 3]
[3, 4]
[[1], [2, 3], [3, 4]]
x:
y:
union(otherRDD)
Return a new RDD containing all items from two original RDDs. Duplicates are not culled.
val x = sc.parallelize(Array(1,2,3), 2)
val y = sc.parallelize(Array(3,4), 1)
val z = x.union(y)
val zOut = z.glom().collect()
z:
5B
JOIN RDD: x RDD: y
4
2B
A
1A
3A
JOIN RDD: x RDD: y
RDD: z
(1, 3)A
5B
4
2B
A
1A
3A
JOIN RDD: x RDD: y
RDD: z
(1, 4)A
(1, 3)A
5B
4
2B
A
1A
3A
JOIN RDD: x RDD: y
(2, 5) RDD: zB
(1, 4)A
(1, 3)A
5B
4
2B
A
1A
3A
JOIN
x = sc.parallelize([("a", 1), ("b", 2)])
y = sc.parallelize([("a", 3), ("a", 4), ("b", 5)])
z = x.join(y)
print(z.collect()) [("a", 1), ("b", 2)]
[("a", 3), ("a", 4), ("b", 5)]
[('a', (1, 3)), ('a', (1, 4)), ('b', (2, 5))]
x:
y:
union(otherRDD, numPartitions=None)
Return a new RDD containing all pairs of elements having the same key in the original RDDs
val x = sc.parallelize(Array(("a", 1), ("b", 2)))
val y = sc.parallelize(Array(("a", 3), ("a", 4), ("b", 5)))
val z = x.join(y)
println(z.collect().mkString(", "))
z:
DISTINCT
4
3
3
2
1
RDD: x
DISTINCT
RDD: x
4
3
3
2
1
4
3
3
2
1
RDD: y
DISTINCT
RDD: x
4
3
3
2
1
4
3
2
1
RDD: y
DISTINCT
x = sc.parallelize([1,2,3,3,4])
y = x.distinct()
print(y.collect())
[1, 2, 3, 3, 4]
[1, 2, 3, 4]
x:
y:
distinct(numPartitions=None)
Return a new RDD containing distinct items from the original RDD (omitting all duplicates)
val x = sc.parallelize(Array(1,2,3,3,4))
val y = x.distinct()
println(y.collect().mkString(", "))
*
*
*¤
¤
COALESCE
RDD: x
A
B
C
COALESCE
RDD: x
B
C
RDD: y
A
AB
C
COALESCE
RDD: x
B
C
RDD: y
A
AB
COALESCE
x = sc.parallelize([1, 2, 3, 4, 5], 3)
y = x.coalesce(2)
print(x.glom().collect())
print(y.glom().collect())
[[1], [2, 3], [4, 5]]
[[1], [2, 3, 4, 5]]
x:
y:
coalesce(numPartitions, shuffle=False)
Return a new RDD which is reduced to a smaller number of partitions
val x = sc.parallelize(Array(1, 2, 3, 4, 5), 3)
val y = x.coalesce(2)
val xOut = x.glom().collect()
val yOut = y.glom().collect()
C
B
C
A
AB
KEYBY
RDD: x
James
Anna
Fred
John
RDD: y
J “John”
emits‘J’
KEYBY
RDD: x
James
Anna
‘F’
Fred “Fred”F
RDD: y
J “John”John
James
“Anna”A
KEYBY
RDD: x
Anna
‘A’
Fred
John
“Fred”F
RDD: y
J “John”
J “James”
“Anna”A
KEYBY
RDD: x
James
Anna
emits‘J’
Fred
John
“Fred”F
RDD: y
J “John”
KEYBY
x = sc.parallelize(['John', 'Fred', 'Anna', 'James'])
y = x.keyBy(lambda w: w[0])
print y.collect()
['John', 'Fred', 'Anna', 'James']
[('J','John'),('F','Fred'),('A','Anna'),('J','James')]
RDD: x RDD: y
x:
y:
keyBy(f)
Create a Pair RDD, forming one pair for each item in the original RDD. The
pair’s key is calculated from the value via a user-supplied function.
val x = sc.parallelize(
Array("John", "Fred", "Anna", "James"))
val y = x.keyBy(w => w.charAt(0))
println(y.collect().mkString(", "))
PARTITIONBY
RDD: x
J “John”
“Anna”A
“Fred”F
J “James”
PARTITIONBY
RDD: x
J “John”
“Anna”A
“Fred”F
J “James”
RDD: yRDD: y
J “James”
1
PARTITIONBY
RDD: x
J “John”
“Anna”A
“Fred”F
RDD: yRDD: y
J “James”
“Fred”F
0
J “James”
PARTITIONBY
RDD: x
J “John”
“Anna”A
RDD: yRDD: y
J “James”
“Anna”A
“Fred”F
0
“Fred”F
J “James”
PARTITIONBY
RDD: x
J “John”
RDD: yRDD: y
J “John”
J “James”
“Anna”A
“Fred”F1
“Anna”A
“Fred”F
J “James”
PARTITIONBY
x = sc.parallelize([('J','James'),('F','Fred'),
('A','Anna'),('J','John')], 3)
y = x.partitionBy(2, lambda w: 0 if w[0] < 'H' else 1)
print x.glom().collect()
print y.glom().collect()
[[('J', 'James')], [('F', 'Fred')],
[('A', 'Anna'), ('J', 'John')]]
[[('A', 'Anna'), ('F', 'Fred')],
[('J', 'James'), ('J', 'John')]]
x:
y:
partitionBy(numPartitions, partitioner=portable_hash)
Return a new RDD with the specified number of partitions, placing original
items into the partition returned by a user supplied function
PARTITIONBY
Array(Array((A,Anna), (F,Fred)),
Array((J,John), (J,James)))
Array(Array((F,Fred), (A,Anna)),
Array((J,John), (J,James)))
x:
y:
partitionBy(numPartitions, partitioner=portable_hash)
Return a new RDD with the specified number of partitions, placing original
items into the partition returned by a user supplied function.
import org.apache.spark.Partitioner
val x = sc.parallelize(Array(('J',"James"),('F',"Fred"),
('A',"Anna"),('J',"John")), 3)
val y = x.partitionBy(new Partitioner() {
val numPartitions = 2
def getPartition(k:Any) = {
if (k.asInstanceOf[Char] < 'H') 0 else 1
}
})
val yOut = y.glom().collect()
ZIP RDD: x RDD: y
3
2
1
A
B
9
4
1
A
B
ZIP RDD: x RDD: y
3
2
1
A
B
4
A
RDD: z
9
4
1
A
B
1 1
ZIP RDD: x RDD: y
3
2
1
A
B
4
A
RDD: z
9
4
1
A
B
2
1
4
1
ZIP RDD: x RDD: y
3
2
1
A
B
4
3
A
B
RDD: z
9
4
1
A
B
2
1
9
4
1
ZIP
x = sc.parallelize([1, 2, 3])
y = x.map(lambda n:n*n)
z = x.zip(y)
print(z.collect()) [1, 2, 3]
[1, 4, 9]
[(1, 1), (2, 4), (3, 9)]
x:
y:
zip(otherRDD)
Return a new RDD containing pairs whose key is the item in the original RDD, and whose
value is that item’s corresponding element (same partition, same index) in a second RDD
val x = sc.parallelize(Array(1,2,3))
val y = x.map(n=>n*n)
val z = x.zip(y)
println(z.collect().mkString(", "))
z:
ACTIONS Core Operations
vs
distributed driver
occurs across the cluster result must fit in driver JVM
GETNUMPARTITIONS
partition(s) A
B
2
x:
y:
getNumPartitions()
Return the number of partitions in RDD
[[1], [2, 3]]
2
GETNUMPARTITIONS
A
B
2
x = sc.parallelize([1,2,3], 2)
y = x.getNumPartitions()
print(x.glom().collect())
print(y)
val x = sc.parallelize(Array(1,2,3), 2)
val y = x.partitions.size
val xOut = x.glom().collect()
println(y)
COLLECT
partition(s) A
B
[ ]
x:
y:
collect()
Return all items in the RDD to the driver in a single list
[[1], [2, 3]]
[1, 2, 3]
A
B
x = sc.parallelize([1,2,3], 2)
y = x.collect()
print(x.glom().collect())
print(y)
val x = sc.parallelize(Array(1,2,3), 2)
val y = x.collect()
val xOut = x.glom().collect()
println(y)
COLLECT [ ]
REDUCE
3
2
1
emits
4
3
4
REDUCE
3
2
1
emits
3
input
6
4
REDUCE
3
2
1
10
input
6
10
x:
y:
reduce(f)
Aggregate all the elements of the RDD by applying a user function
pairwise to elements and partial results, and returns a result to the driver
[1, 2, 3, 4]
10
x = sc.parallelize([1,2,3,4])
y = x.reduce(lambda a,b: a+b)
print(x.collect())
print(y)
val x = sc.parallelize(Array(1,2,3,4))
val y = x.reduce((a,b) => a+b)
println(x.collect.mkString(", "))
println(y)
REDUCE
******
*
**
***
AGGREGATE
3
2
1
4
A
B
AGGREGATE
3
2
1
4
AGGREGATE
3
2
1
4
emits
([], 0)
([1], 1)
AGGREGATE
3
2
1
4
([], 0)
([2], 2)
([1], 1)
AGGREGATE
3
2
1
4
([2], 2)
([1], 1)([1,2], 3)
AGGREGATE
3
2
1
4
([2], 2)
([1], 1)([1,2], 3)
([3], 3)
([], 0)
AGGREGATE
3
2
1
4
([2], 2)
([1], 1)([1,2], 3)
([3], 3)
([], 0)
([4], 4)
AGGREGATE
3
2
1
4
([2], 2)
([1], 1)([1,2], 3)
([4], 4)
([3], 3)([3,4], 7)
AGGREGATE
3
2
1
4
([2], 2)
([1], 1)([1,2], 3)
([4], 4)
([3], 3)([3,4], 7)
AGGREGATE
3
2
1
4
([1,2], 3)
([3,4], 7)
AGGREGATE
3
2
1
4
([1,2], 3)
([3,4], 7)
([1,2,3,4], 10)
([1,2,3,4], 10)
aggregate(identity, seqOp, combOp)
Aggregate all the elements of the RDD by:
- applying a user function to combine elements with user-supplied objects,
- then combining those user-defined results via a second user function,
- and finally returning a result to the driver.
seqOp = lambda data, item: (data[0] + [item], data[1] + item)
combOp = lambda d1, d2: (d1[0] + d2[0], d1[1] + d2[1])
x = sc.parallelize([1,2,3,4])
y = x.aggregate(([], 0), seqOp, combOp)
print(y)
AGGREGATE
***
****
**
***
[( ),#]
x:
y:
[1, 2, 3, 4]
([1, 2, 3, 4], 10)
def seqOp = (data:(Array[Int], Int), item:Int) =>
(data._1 :+ item, data._2 + item)
def combOp = (d1:(Array[Int], Int), d2:(Array[Int], Int)) =>
(d1._1.union(d2._1), d1._2 + d2._2)
val x = sc.parallelize(Array(1,2,3,4))
val y = x.aggregate((Array[Int](), 0))(seqOp, combOp)
println(y)
x:
y:
aggregate(identity, seqOp, combOp)
Aggregate all the elements of the RDD by:
- applying a user function to combine elements with user-supplied objects,
- then combining those user-defined results via a second user function,
- and finally returning a result to the driver.
[1, 2, 3, 4]
(Array(3, 1, 2, 4),10)
AGGREGATE
***
****
**
***
[( ),#]
MAX
4
2
1
4
x:
y:
max()
Return the maximum item in the RDD
[2, 4, 1]
4
MAX
4
x = sc.parallelize([2,4,1])
y = x.max()
print(x.collect())
print(y)
val x = sc.parallelize(Array(2,4,1))
val y = x.max
println(x.collect().mkString(", "))
println(y)
2
1
4
max
SUM
7
2
1
4
x:
y:
sum()
Return the sum of the items in the RDD
[2, 4, 1]
7
SUM
7
x = sc.parallelize([2,4,1])
y = x.sum()
print(x.collect())
print(y)
val x = sc.parallelize(Array(2,4,1))
val y = x.sum
println(x.collect().mkString(", "))
println(y)
2
1
4
Σ
MEAN
2.33333333
2
1
4
x:
y:
mean()
Return the mean of the items in the RDD
[2, 4, 1]
2.3333333
MEAN
2.3333333
x = sc.parallelize([2,4,1])
y = x.mean()
print(x.collect())
print(y)
val x = sc.parallelize(Array(2,4,1))
val y = x.mean
println(x.collect().mkString(", "))
println(y)
2
1
4
x
STDEV
1.2472191
2
1
4
x:
y:
stdev()
Return the standard deviation of the items in the RDD
[2, 4, 1]
1.2472191
STDEV
1.2472191
x = sc.parallelize([2,4,1])
y = x.stdev()
print(x.collect())
print(y)
val x = sc.parallelize(Array(2,4,1))
val y = x.stdev
println(x.collect().mkString(", "))
println(y)
2
1
4
σ
COUNTBYKEY
{'A': 1, 'J': 2, 'F': 1}
J “John”
“Anna”A
“Fred”F
J “James”
x:
y:
countByKey()
Return a map of keys and counts of their occurrences in the RDD
[('J', 'James'), ('F','Fred'),
('A','Anna'), ('J','John')]
{'A': 1, 'J': 2, 'F': 1}
COUNTBYKEY
{A: 1, 'J': 2, 'F': 1}
x = sc.parallelize([('J', 'James'), ('F','Fred'),
('A','Anna'), ('J','John')])
y = x.countByKey()
print(y)
val x = sc.parallelize(Array(('J',"James"),('F',"Fred"),
('A',"Anna"),('J',"John")))
val y = x.countByKey()
println(y)
SAVEASTEXTFILE
x:
y:
saveAsTextFile(path, compressionCodecClass=None)
Save the RDD to the filesystem indicated in the path
[2, 4, 1]
[u'2', u'4', u'1']
SAVEASTEXTFILE
dbutils.fs.rm("/temp/demo", True)
x = sc.parallelize([2,4,1])
x.saveAsTextFile("/temp/demo")
y = sc.textFile("/temp/demo")
print(y.collect())
dbutils.fs.rm("/temp/demo", true)
val x = sc.parallelize(Array(2,4,1))
x.saveAsTextFile("/temp/demo")
val y = sc.textFile("/temp/demo")
println(y.collect().mkString(", "))
LAB
Q&A

More Related Content

PDF
Introduction to Spark Internals
KEY
Graphdatabases
PDF
Mapreduce by examples
PPTX
Transformations and actions a visual guide training
PPTX
Using Wildcards with rsyslog's File Monitor imfile
PPTX
Spark architecture
PPTX
Intro to Apache Spark
PPT
Using AWR for IO Subsystem Analysis
Introduction to Spark Internals
Graphdatabases
Mapreduce by examples
Transformations and actions a visual guide training
Using Wildcards with rsyslog's File Monitor imfile
Spark architecture
Intro to Apache Spark
Using AWR for IO Subsystem Analysis

What's hot (20)

PDF
Espresso: LinkedIn's Distributed Data Serving Platform (Talk)
PPTX
Hive Tutorial | Hive Architecture | Hive Tutorial For Beginners | Hive In Had...
PPTX
Introduction to Apache Spark
PDF
Amazon RDS Proxy 집중 탐구 - 윤석찬 :: AWS Unboxing 온라인 세미나
PDF
Understanding Oracle RAC 11g Release 2 Internals
PDF
Graph Database and Amazon Neptune
PDF
5 Steps to PostgreSQL Performance
PPTX
What Is Apache Spark? | Introduction To Apache Spark | Apache Spark Tutorial ...
PPTX
Airflow를 이용한 데이터 Workflow 관리
PDF
Introduction to Apache Spark
PDF
How does PostgreSQL work with disks: a DBA's checklist in detail. PGConf.US 2015
PDF
Introduction to Apache Calcite
PDF
PostgreSQL and RAM usage
PDF
HTTP Analytics for 6M requests per second using ClickHouse, by Alexander Boc...
PDF
Let's talk about Garbage Collection
PPT
UKOUG, Oracle Transaction Locks
PDF
Introduction to DataFusion An Embeddable Query Engine Written in Rust
PDF
SQL on everything, in memory
PDF
Amazon DocumentDB vs MongoDB 의 내부 아키텍쳐 와 장단점 비교
PPT
NOSQL Database: Apache Cassandra
Espresso: LinkedIn's Distributed Data Serving Platform (Talk)
Hive Tutorial | Hive Architecture | Hive Tutorial For Beginners | Hive In Had...
Introduction to Apache Spark
Amazon RDS Proxy 집중 탐구 - 윤석찬 :: AWS Unboxing 온라인 세미나
Understanding Oracle RAC 11g Release 2 Internals
Graph Database and Amazon Neptune
5 Steps to PostgreSQL Performance
What Is Apache Spark? | Introduction To Apache Spark | Apache Spark Tutorial ...
Airflow를 이용한 데이터 Workflow 관리
Introduction to Apache Spark
How does PostgreSQL work with disks: a DBA's checklist in detail. PGConf.US 2015
Introduction to Apache Calcite
PostgreSQL and RAM usage
HTTP Analytics for 6M requests per second using ClickHouse, by Alexander Boc...
Let's talk about Garbage Collection
UKOUG, Oracle Transaction Locks
Introduction to DataFusion An Embeddable Query Engine Written in Rust
SQL on everything, in memory
Amazon DocumentDB vs MongoDB 의 내부 아키텍쳐 와 장단점 비교
NOSQL Database: Apache Cassandra
Ad

Similar to Visual Api Training (20)

PPTX
Scala meetup - Intro to spark
PDF
Apache Spark - Key Value RDD - Transformations | Big Data Hadoop Spark Tutori...
PDF
Apache Spark - Basics of RDD & RDD Operations | Big Data Hadoop Spark Tutoria...
PDF
Operations on rdd
PDF
Distributed computing with spark
PDF
Apache Spark - Basics of RDD | Big Data Hadoop Spark Tutorial | CloudxLab
PDF
PPTX
04 spark-pair rdd-rdd-persistence
PDF
Introduction to spark
PDF
Apache Spark and DataStax Enablement
PPTX
Spark real world use cases and optimizations
PDF
Meetup ml spark_ppt
PDF
Big Data processing with Apache Spark
DOCX
Spark_Documentation_Template1
PPTX
Introduction to Apache Spark
PPT
11. From Hadoop to Spark 2/2
PPTX
Ten tools for ten big data areas 03_Apache Spark
PDF
Apache Spark: What? Why? When?
ODT
Spark rdd part 2
PDF
Apache Spark & Streaming
Scala meetup - Intro to spark
Apache Spark - Key Value RDD - Transformations | Big Data Hadoop Spark Tutori...
Apache Spark - Basics of RDD & RDD Operations | Big Data Hadoop Spark Tutoria...
Operations on rdd
Distributed computing with spark
Apache Spark - Basics of RDD | Big Data Hadoop Spark Tutorial | CloudxLab
04 spark-pair rdd-rdd-persistence
Introduction to spark
Apache Spark and DataStax Enablement
Spark real world use cases and optimizations
Meetup ml spark_ppt
Big Data processing with Apache Spark
Spark_Documentation_Template1
Introduction to Apache Spark
11. From Hadoop to Spark 2/2
Ten tools for ten big data areas 03_Apache Spark
Apache Spark: What? Why? When?
Spark rdd part 2
Apache Spark & Streaming
Ad

More from Spark Summit (20)

PDF
FPGA-Based Acceleration Architecture for Spark SQL Qi Xie and Quanfu Wang
PDF
VEGAS: The Missing Matplotlib for Scala/Apache Spark with DB Tsai and Roger M...
PDF
Apache Spark Structured Streaming Helps Smart Manufacturing with Xiaochang Wu
PDF
Improving Traffic Prediction Using Weather Data with Ramya Raghavendra
PDF
A Tale of Two Graph Frameworks on Spark: GraphFrames and Tinkerpop OLAP Artem...
PDF
No More Cumbersomeness: Automatic Predictive Modeling on Apache Spark Marcin ...
PDF
Apache Spark and Tensorflow as a Service with Jim Dowling
PDF
Apache Spark and Tensorflow as a Service with Jim Dowling
PDF
MMLSpark: Lessons from Building a SparkML-Compatible Machine Learning Library...
PDF
Next CERN Accelerator Logging Service with Jakub Wozniak
PDF
Powering a Startup with Apache Spark with Kevin Kim
PDF
Improving Traffic Prediction Using Weather Datawith Ramya Raghavendra
PDF
Hiding Apache Spark Complexity for Fast Prototyping of Big Data Applications—...
PDF
How Nielsen Utilized Databricks for Large-Scale Research and Development with...
PDF
Spline: Apache Spark Lineage not Only for the Banking Industry with Marek Nov...
PDF
Goal Based Data Production with Sim Simeonov
PDF
Preventing Revenue Leakage and Monitoring Distributed Systems with Machine Le...
PDF
Getting Ready to Use Redis with Apache Spark with Dvir Volk
PDF
Deduplication and Author-Disambiguation of Streaming Records via Supervised M...
PDF
MatFast: In-Memory Distributed Matrix Computation Processing and Optimization...
FPGA-Based Acceleration Architecture for Spark SQL Qi Xie and Quanfu Wang
VEGAS: The Missing Matplotlib for Scala/Apache Spark with DB Tsai and Roger M...
Apache Spark Structured Streaming Helps Smart Manufacturing with Xiaochang Wu
Improving Traffic Prediction Using Weather Data with Ramya Raghavendra
A Tale of Two Graph Frameworks on Spark: GraphFrames and Tinkerpop OLAP Artem...
No More Cumbersomeness: Automatic Predictive Modeling on Apache Spark Marcin ...
Apache Spark and Tensorflow as a Service with Jim Dowling
Apache Spark and Tensorflow as a Service with Jim Dowling
MMLSpark: Lessons from Building a SparkML-Compatible Machine Learning Library...
Next CERN Accelerator Logging Service with Jakub Wozniak
Powering a Startup with Apache Spark with Kevin Kim
Improving Traffic Prediction Using Weather Datawith Ramya Raghavendra
Hiding Apache Spark Complexity for Fast Prototyping of Big Data Applications—...
How Nielsen Utilized Databricks for Large-Scale Research and Development with...
Spline: Apache Spark Lineage not Only for the Banking Industry with Marek Nov...
Goal Based Data Production with Sim Simeonov
Preventing Revenue Leakage and Monitoring Distributed Systems with Machine Le...
Getting Ready to Use Redis with Apache Spark with Dvir Volk
Deduplication and Author-Disambiguation of Streaming Records via Supervised M...
MatFast: In-Memory Distributed Matrix Computation Processing and Optimization...

Recently uploaded (20)

PPTX
The THESIS FINAL-DEFENSE-PRESENTATION.pptx
PPTX
Introduction to machine learning and Linear Models
PDF
.pdf is not working space design for the following data for the following dat...
PPTX
Acceptance and paychological effects of mandatory extra coach I classes.pptx
PPT
Reliability_Chapter_ presentation 1221.5784
PPT
Quality review (1)_presentation of this 21
PPTX
climate analysis of Dhaka ,Banglades.pptx
PPTX
Data_Analytics_and_PowerBI_Presentation.pptx
PPTX
Business Acumen Training GuidePresentation.pptx
PPTX
1_Introduction to advance data techniques.pptx
PPTX
ALIMENTARY AND BILIARY CONDITIONS 3-1.pptx
PPTX
oil_refinery_comprehensive_20250804084928 (1).pptx
PDF
Business Analytics and business intelligence.pdf
PPTX
IB Computer Science - Internal Assessment.pptx
PPTX
MODULE 8 - DISASTER risk PREPAREDNESS.pptx
PDF
168300704-gasification-ppt.pdfhghhhsjsjhsuxush
PPTX
Introduction to Basics of Ethical Hacking and Penetration Testing -Unit No. 1...
PDF
TRAFFIC-MANAGEMENT-AND-ACCIDENT-INVESTIGATION-WITH-DRIVING-PDF-FILE.pdf
PDF
BF and FI - Blockchain, fintech and Financial Innovation Lesson 2.pdf
The THESIS FINAL-DEFENSE-PRESENTATION.pptx
Introduction to machine learning and Linear Models
.pdf is not working space design for the following data for the following dat...
Acceptance and paychological effects of mandatory extra coach I classes.pptx
Reliability_Chapter_ presentation 1221.5784
Quality review (1)_presentation of this 21
climate analysis of Dhaka ,Banglades.pptx
Data_Analytics_and_PowerBI_Presentation.pptx
Business Acumen Training GuidePresentation.pptx
1_Introduction to advance data techniques.pptx
ALIMENTARY AND BILIARY CONDITIONS 3-1.pptx
oil_refinery_comprehensive_20250804084928 (1).pptx
Business Analytics and business intelligence.pdf
IB Computer Science - Internal Assessment.pptx
MODULE 8 - DISASTER risk PREPAREDNESS.pptx
168300704-gasification-ppt.pdfhghhhsjsjhsuxush
Introduction to Basics of Ethical Hacking and Penetration Testing -Unit No. 1...
TRAFFIC-MANAGEMENT-AND-ACCIDENT-INVESTIGATION-WITH-DRIVING-PDF-FILE.pdf
BF and FI - Blockchain, fintech and Financial Innovation Lesson 2.pdf

Visual Api Training

  • 1. TRANSFORMATIONS AND ACTIONS A Visual Guide of the APIhttp://training.databricks.com/visualapi.pdf
  • 2. Databricks would like to give a special thanks to Jeff Thomspon for contributing 67 visual diagrams depicting the Spark API under the MIT license to the Spark community. Jeff’s original, creative work can be found here and you can read more about Jeff’s project in his blog post. After talking to Jeff, Databricks commissioned Adam Breindel to further evolve Jeff’s work into the diagrams you see in this deck. LinkedIn Blog: data-frack
  • 3. making big data simple Databricks Cloud: “A unified platform for building Big Data pipelines – from ETL to Exploration and Dashboards, to Advanced Analytics and Data Products.” • Founded in late 2013 • by the creators of Apache Spark • Original team from UC Berkeley AMPLab • Raised $47 Million in 2 rounds • ~55 employees • We’re hiring! • Level 2/3 support partnerships with • Hortonworks • MapR • DataStax (http://databricks.workable.com)
  • 4. key RDD Elements original item transformed type object on driver RDD partition(s) A B user functions user input input emitted value Legend
  • 5. Randomized operation Legend Set Theory / Relational operation Numeric calculation
  • 7. • map • filter • flatMap • mapPartitions • mapPartitionsWithIndex • groupBy • sortBy = medium Essential Core & Intermediate Spark OperationsTRANSFORMATIONSACTIONS General • sample • randomSplit Math / Statistical = easy Set Theory / Relational • union • intersection • subtract • distinct • cartesian • zip • takeOrdered Data Structure / I/O • saveAsTextFile • saveAsSequenceFile • saveAsObjectFile • saveAsHadoopDataset • saveAsHadoopFile • saveAsNewAPIHadoopDataset • saveAsNewAPIHadoopFile • keyBy • zipWithIndex • zipWithUniqueID • zipPartitions • coalesce • repartition • repartitionAndSortWithinPartitions • pipe • count • takeSample • max • min • sum • histogram • mean • variance • stdev • sampleVariance • countApprox • countApproxDistinct • reduce • collect • aggregate • fold • first • take • forEach • top • treeAggregate • treeReduce • forEachPartition • collectAsMap
  • 8. = medium Essential Core & Intermediate PairRDD OperationsTRANSFORMATIONSACTIONS General • sampleByKey Math / Statistical = easy Set Theory / Relational Data Structure • keys • values • partitionBy • countByKey • countByValue • countByValueApprox • countApproxDistinctByKey • countApproxDistinctByKey • countByKeyApprox • sampleByKeyExact • cogroup (=groupWith) • join • subtractByKey • fullOuterJoin • leftOuterJoin • rightOuterJoin • flatMapValues • groupByKey • reduceByKey • reduceByKeyLocally • foldByKey • aggregateByKey • sortByKey • combineByKey
  • 9. vs narrow wide each partition of the parent RDD is used by at most one partition of the child RDD multiple child RDD partitions may depend on a single parent RDD partition
  • 10. “One of the challenges in providing RDDs as an abstraction is choosing a representation for them that can track lineage across a wide range of transformations.” “The most interesting question in designing this interface is how to represent dependencies between RDDs.” “We found it both sufficient and useful to classify dependencies into two types: • narrow dependencies, where each partition of the parent RDD is used by at most one partition of the child RDD • wide dependencies, where multiple child partitions may depend on it.”
  • 11. narrow wide each partition of the parent RDD is used by at most one partition of the child RDD multiple child RDD partitions may depend on a single parent RDD partition map, filter union join w/ inputs co-partitioned groupByKey join w/ inputs not co-partitioned
  • 13. MAP 3 items in RDD RDD: x
  • 14. MAP User function applied item by item RDD: x RDD: y
  • 17. MAP RDD: x RDD: y After map() has been applied… before after
  • 18. MAP RDD: x RDD: y Return a new RDD by applying a function to each element of this RDD.
  • 19. MAP x = sc.parallelize(["b", "a", "c"]) y = x.map(lambda z: (z, 1)) print(x.collect()) print(y.collect()) ['b', 'a', 'c'] [('b', 1), ('a', 1), ('c', 1)] RDD: x RDD: y x: y: map(f, preservesPartitioning=False) Return a new RDD by applying a function to each element of this RDD val x = sc.parallelize(Array("b", "a", "c")) val y = x.map(z => (z,1)) println(x.collect().mkString(", ")) println(y.collect().mkString(", "))
  • 20. FILTER 3 items in RDD RDD: x
  • 21. FILTER Apply user function: keep item if function returns true RDD: x RDD: y emits True
  • 22. FILTER RDD: x RDD: y emits False
  • 23. FILTER RDD: x RDD: y emits True
  • 24. FILTER RDD: x RDD: y After filter() has been applied… before after
  • 25. FILTER x = sc.parallelize([1,2,3]) y = x.filter(lambda x: x%2 == 1) #keep odd values print(x.collect()) print(y.collect()) [1, 2, 3] [1, 3] RDD: x RDD: y x: y: filter(f) Return a new RDD containing only the elements that satisfy a predicate val x = sc.parallelize(Array(1,2,3)) val y = x.filter(n => n%2 == 1) println(x.collect().mkString(", ")) println(y.collect().mkString(", "))
  • 26. FLATMAP 3 items in RDD RDD: x
  • 30. FLATMAP RDD: x RDD: y After flatmap() has been applied… before after
  • 31. FLATMAP RDD: x RDD: y Return a new RDD by first applying a function to all elements of this RDD, and then flattening the results
  • 32. FLATMAP x = sc.parallelize([1,2,3]) y = x.flatMap(lambda x: (x, x*100, 42)) print(x.collect()) print(y.collect()) [1, 2, 3] [1, 100, 42, 2, 200, 42, 3, 300, 42] x: y: RDD: x RDD: y flatMap(f, preservesPartitioning=False) Return a new RDD by first applying a function to all elements of this RDD, and then flattening the results val x = sc.parallelize(Array(1,2,3)) val y = x.flatMap(n => Array(n, n*100, 42)) println(x.collect().mkString(", ")) println(y.collect().mkString(", "))
  • 33. GROUPBY 4 items in RDD RDD: x James Anna Fred John
  • 35. F [ “Fred” ] GROUPBY RDD: x James Anna Fred emits‘F’ J [ “John” ]John RDD: y
  • 36. [ “Fred” ] GROUPBY RDD: x James Anna emits‘A’ J [ “John” ] A [ “Anna” ] Fred John F RDD: y
  • 37. [ “Fred” ] GROUPBY RDD: x James Anna emits‘J’ J [ “John”, “James” ] [ “Anna” ] Fred John F A RDD: y
  • 38. GROUPBY x = sc.parallelize(['John', 'Fred', 'Anna', 'James']) y = x.groupBy(lambda w: w[0]) print [(k, list(v)) for (k, v) in y.collect()] ['John', 'Fred', 'Anna', 'James'] [('A',['Anna']),('J',['John','James']),('F',['Fred'])] RDD: x RDD: y x: y: groupBy(f, numPartitions=None) Group the data in the original RDD. Create pairs where the key is the output of a user function, and the value is all items for which the function yields this key. val x = sc.parallelize( Array("John", "Fred", "Anna", "James")) val y = x.groupBy(w => w.charAt(0)) println(y.collect().mkString(", "))
  • 39. GROUPBYKEY 5 items in RDD Pair RDD: x B B A A A 5 4 3 2 1
  • 40. GROUPBYKEY Pair RDD: x 5 4 3 2 1 RDD: y A [ 2 , 3 , 1 ] B B A A A
  • 41. GROUPBYKEY Pair RDD: x RDD: y B [ 5 , 4 ] A [ 2 , 3 , 1 ] 5 4 3 2 1 B B A A A
  • 42. GROUPBYKEY x = sc.parallelize([('B',5),('B',4),('A',3),('A',2),('A',1)]) y = x.groupByKey() print(x.collect()) print(list((j[0], list(j[1])) for j in y.collect())) [('B', 5),('B', 4),('A', 3),('A', 2),('A', 1)] [('A', [2, 3, 1]),('B',[5, 4])] RDD: x RDD: y x: y: groupByKey(numPartitions=None) Group the values for each key in the original RDD. Create a new pair where the original key corresponds to this collected group of values. val x = sc.parallelize( Array(('B',5),('B',4),('A',3),('A',2),('A',1))) val y = x.groupByKey() println(x.collect().mkString(", ")) println(y.collect().mkString(", "))
  • 43. MAPPARTITIONS RDD: x RDD: y partitions A B A B
  • 44. REDUCEBYKEY VS GROUPBYKEY val words = Array("one", "two", "two", "three", "three", "three") val wordPairsRDD = sc.parallelize(words).map(word => (word, 1)) val wordCountsWithReduce = wordPairsRDD .reduceByKey(_ + _) .collect() val wordCountsWithGroup = wordPairsRDD .groupByKey() .map(t => (t._1, t._2.sum)) .collect()
  • 45. REDUCEBYKEY (a, 1) (b, 1) (a, 1) (b, 1) (a, 1) (a, 1) (a, 2) (b, 2)(b, 1) (b, 1) (a, 1) (a, 1) (a, 3) (b, 2) (a, 1) (b, 1) (b, 1) (a, 1) (a, 2) (a, 6) (a, 3) (b, 1) (b, 2) (b, 5) (b, 2) a b
  • 46. GROUPBYKEY (a, 1) (b, 1) (a, 1) (a, 1) (b, 1) (b, 1) (a, 1) (a, 1) (a, 1) (b, 1) (b, 1) (a, 1) (a, 1) (a, 6) (a, 1) (b, 5) a b (a, 1) (a, 1) (a, 1) (b, 1) (b, 1) (b, 1) (b, 1) (b, 1)
  • 47. MAPPARTITIONS x: y: mapPartitions(f, preservesPartitioning=False) Return a new RDD by applying a function to each partition of this RDD A B A B x = sc.parallelize([1,2,3], 2) def f(iterator): yield sum(iterator); yield 42 y = x.mapPartitions(f) # glom() flattens elements on the same partition print(x.glom().collect()) print(y.glom().collect()) [[1], [2, 3]] [[1, 42], [5, 42]]
  • 48. MAPPARTITIONS x: y: mapPartitions(f, preservesPartitioning=False) Return a new RDD by applying a function to each partition of this RDD A B A B Array(Array(1), Array(2, 3)) Array(Array(1, 42), Array(5, 42)) val x = sc.parallelize(Array(1,2,3), 2) def f(i:Iterator[Int])={ (i.sum,42).productIterator } val y = x.mapPartitions(f) // glom() flattens elements on the same partition val xOut = x.glom().collect() val yOut = y.glom().collect()
  • 49. MAPPARTITIONSWITHINDEX RDD: x RDD: y partitions A B A B input partition index
  • 50. x: y: mapPartitionsWithIndex(f, preservesPartitioning=False) Return a new RDD by applying a function to each partition of this RDD, while tracking the index of the original partition A B A B x = sc.parallelize([1,2,3], 2) def f(partitionIndex, iterator): yield (partitionIndex, sum(iterator)) y = x.mapPartitionsWithIndex(f) # glom() flattens elements on the same partition print(x.glom().collect()) print(y.glom().collect()) [[1], [2, 3]] [[0, 1], [1, 5]] MAPPARTITIONSWITHINDEX partition index B A
  • 51. x: y: mapPartitionsWithIndex(f, preservesPartitioning=False) Return a new RDD by applying a function to each partition of this RDD, while tracking the index of the original partition. A B A B Array(Array(1), Array(2, 3)) Array(Array(0, 1), Array(1, 5)) MAPPARTITIONSWITHINDEX partition index B A val x = sc.parallelize(Array(1,2,3), 2) def f(partitionIndex:Int, i:Iterator[Int]) = { (partitionIndex, i.sum).productIterator } val y = x.mapPartitionsWithIndex(f) // glom() flattens elements on the same partition val xOut = x.glom().collect() val yOut = y.glom().collect()
  • 52. SAMPLE RDD: x RDD: y 1 3 5 4 3 2 1
  • 53. SAMPLE x = sc.parallelize([1, 2, 3, 4, 5]) y = x.sample(False, 0.4, 42) print(x.collect()) print(y.collect()) [1, 2, 3, 4, 5] [1, 3] RDD: x RDD: y x: y: sample(withReplacement, fraction, seed=None) Return a new RDD containing a statistical sample of the original RDD val x = sc.parallelize(Array(1, 2, 3, 4, 5)) val y = x.sample(false, 0.4) // omitting seed will yield different output println(y.collect().mkString(", "))
  • 54. UNION RDD: x RDD: y 4 3 3 2 1 A B C 4 3 3 2 1 A B C RDD: z
  • 55. UNION x = sc.parallelize([1,2,3], 2) y = sc.parallelize([3,4], 1) z = x.union(y) print(z.glom().collect()) [1, 2, 3] [3, 4] [[1], [2, 3], [3, 4]] x: y: union(otherRDD) Return a new RDD containing all items from two original RDDs. Duplicates are not culled. val x = sc.parallelize(Array(1,2,3), 2) val y = sc.parallelize(Array(3,4), 1) val z = x.union(y) val zOut = z.glom().collect() z:
  • 56. 5B JOIN RDD: x RDD: y 4 2B A 1A 3A
  • 57. JOIN RDD: x RDD: y RDD: z (1, 3)A 5B 4 2B A 1A 3A
  • 58. JOIN RDD: x RDD: y RDD: z (1, 4)A (1, 3)A 5B 4 2B A 1A 3A
  • 59. JOIN RDD: x RDD: y (2, 5) RDD: zB (1, 4)A (1, 3)A 5B 4 2B A 1A 3A
  • 60. JOIN x = sc.parallelize([("a", 1), ("b", 2)]) y = sc.parallelize([("a", 3), ("a", 4), ("b", 5)]) z = x.join(y) print(z.collect()) [("a", 1), ("b", 2)] [("a", 3), ("a", 4), ("b", 5)] [('a', (1, 3)), ('a', (1, 4)), ('b', (2, 5))] x: y: union(otherRDD, numPartitions=None) Return a new RDD containing all pairs of elements having the same key in the original RDDs val x = sc.parallelize(Array(("a", 1), ("b", 2))) val y = sc.parallelize(Array(("a", 3), ("a", 4), ("b", 5))) val z = x.join(y) println(z.collect().mkString(", ")) z:
  • 64. DISTINCT x = sc.parallelize([1,2,3,3,4]) y = x.distinct() print(y.collect()) [1, 2, 3, 3, 4] [1, 2, 3, 4] x: y: distinct(numPartitions=None) Return a new RDD containing distinct items from the original RDD (omitting all duplicates) val x = sc.parallelize(Array(1,2,3,3,4)) val y = x.distinct() println(y.collect().mkString(", ")) * * *¤ ¤
  • 68. COALESCE x = sc.parallelize([1, 2, 3, 4, 5], 3) y = x.coalesce(2) print(x.glom().collect()) print(y.glom().collect()) [[1], [2, 3], [4, 5]] [[1], [2, 3, 4, 5]] x: y: coalesce(numPartitions, shuffle=False) Return a new RDD which is reduced to a smaller number of partitions val x = sc.parallelize(Array(1, 2, 3, 4, 5), 3) val y = x.coalesce(2) val xOut = x.glom().collect() val yOut = y.glom().collect() C B C A AB
  • 73. KEYBY x = sc.parallelize(['John', 'Fred', 'Anna', 'James']) y = x.keyBy(lambda w: w[0]) print y.collect() ['John', 'Fred', 'Anna', 'James'] [('J','John'),('F','Fred'),('A','Anna'),('J','James')] RDD: x RDD: y x: y: keyBy(f) Create a Pair RDD, forming one pair for each item in the original RDD. The pair’s key is calculated from the value via a user-supplied function. val x = sc.parallelize( Array("John", "Fred", "Anna", "James")) val y = x.keyBy(w => w.charAt(0)) println(y.collect().mkString(", "))
  • 75. PARTITIONBY RDD: x J “John” “Anna”A “Fred”F J “James” RDD: yRDD: y J “James” 1
  • 76. PARTITIONBY RDD: x J “John” “Anna”A “Fred”F RDD: yRDD: y J “James” “Fred”F 0 J “James”
  • 77. PARTITIONBY RDD: x J “John” “Anna”A RDD: yRDD: y J “James” “Anna”A “Fred”F 0 “Fred”F J “James”
  • 78. PARTITIONBY RDD: x J “John” RDD: yRDD: y J “John” J “James” “Anna”A “Fred”F1 “Anna”A “Fred”F J “James”
  • 79. PARTITIONBY x = sc.parallelize([('J','James'),('F','Fred'), ('A','Anna'),('J','John')], 3) y = x.partitionBy(2, lambda w: 0 if w[0] < 'H' else 1) print x.glom().collect() print y.glom().collect() [[('J', 'James')], [('F', 'Fred')], [('A', 'Anna'), ('J', 'John')]] [[('A', 'Anna'), ('F', 'Fred')], [('J', 'James'), ('J', 'John')]] x: y: partitionBy(numPartitions, partitioner=portable_hash) Return a new RDD with the specified number of partitions, placing original items into the partition returned by a user supplied function
  • 80. PARTITIONBY Array(Array((A,Anna), (F,Fred)), Array((J,John), (J,James))) Array(Array((F,Fred), (A,Anna)), Array((J,John), (J,James))) x: y: partitionBy(numPartitions, partitioner=portable_hash) Return a new RDD with the specified number of partitions, placing original items into the partition returned by a user supplied function. import org.apache.spark.Partitioner val x = sc.parallelize(Array(('J',"James"),('F',"Fred"), ('A',"Anna"),('J',"John")), 3) val y = x.partitionBy(new Partitioner() { val numPartitions = 2 def getPartition(k:Any) = { if (k.asInstanceOf[Char] < 'H') 0 else 1 } }) val yOut = y.glom().collect()
  • 81. ZIP RDD: x RDD: y 3 2 1 A B 9 4 1 A B
  • 82. ZIP RDD: x RDD: y 3 2 1 A B 4 A RDD: z 9 4 1 A B 1 1
  • 83. ZIP RDD: x RDD: y 3 2 1 A B 4 A RDD: z 9 4 1 A B 2 1 4 1
  • 84. ZIP RDD: x RDD: y 3 2 1 A B 4 3 A B RDD: z 9 4 1 A B 2 1 9 4 1
  • 85. ZIP x = sc.parallelize([1, 2, 3]) y = x.map(lambda n:n*n) z = x.zip(y) print(z.collect()) [1, 2, 3] [1, 4, 9] [(1, 1), (2, 4), (3, 9)] x: y: zip(otherRDD) Return a new RDD containing pairs whose key is the item in the original RDD, and whose value is that item’s corresponding element (same partition, same index) in a second RDD val x = sc.parallelize(Array(1,2,3)) val y = x.map(n=>n*n) val z = x.zip(y) println(z.collect().mkString(", ")) z:
  • 87. vs distributed driver occurs across the cluster result must fit in driver JVM
  • 89. x: y: getNumPartitions() Return the number of partitions in RDD [[1], [2, 3]] 2 GETNUMPARTITIONS A B 2 x = sc.parallelize([1,2,3], 2) y = x.getNumPartitions() print(x.glom().collect()) print(y) val x = sc.parallelize(Array(1,2,3), 2) val y = x.partitions.size val xOut = x.glom().collect() println(y)
  • 91. x: y: collect() Return all items in the RDD to the driver in a single list [[1], [2, 3]] [1, 2, 3] A B x = sc.parallelize([1,2,3], 2) y = x.collect() print(x.glom().collect()) print(y) val x = sc.parallelize(Array(1,2,3), 2) val y = x.collect() val xOut = x.glom().collect() println(y) COLLECT [ ]
  • 95. x: y: reduce(f) Aggregate all the elements of the RDD by applying a user function pairwise to elements and partial results, and returns a result to the driver [1, 2, 3, 4] 10 x = sc.parallelize([1,2,3,4]) y = x.reduce(lambda a,b: a+b) print(x.collect()) print(y) val x = sc.parallelize(Array(1,2,3,4)) val y = x.reduce((a,b) => a+b) println(x.collect.mkString(", ")) println(y) REDUCE ****** * ** ***
  • 102. AGGREGATE 3 2 1 4 ([2], 2) ([1], 1)([1,2], 3) ([3], 3) ([], 0) ([4], 4)
  • 103. AGGREGATE 3 2 1 4 ([2], 2) ([1], 1)([1,2], 3) ([4], 4) ([3], 3)([3,4], 7)
  • 104. AGGREGATE 3 2 1 4 ([2], 2) ([1], 1)([1,2], 3) ([4], 4) ([3], 3)([3,4], 7)
  • 107. aggregate(identity, seqOp, combOp) Aggregate all the elements of the RDD by: - applying a user function to combine elements with user-supplied objects, - then combining those user-defined results via a second user function, - and finally returning a result to the driver. seqOp = lambda data, item: (data[0] + [item], data[1] + item) combOp = lambda d1, d2: (d1[0] + d2[0], d1[1] + d2[1]) x = sc.parallelize([1,2,3,4]) y = x.aggregate(([], 0), seqOp, combOp) print(y) AGGREGATE *** **** ** *** [( ),#] x: y: [1, 2, 3, 4] ([1, 2, 3, 4], 10)
  • 108. def seqOp = (data:(Array[Int], Int), item:Int) => (data._1 :+ item, data._2 + item) def combOp = (d1:(Array[Int], Int), d2:(Array[Int], Int)) => (d1._1.union(d2._1), d1._2 + d2._2) val x = sc.parallelize(Array(1,2,3,4)) val y = x.aggregate((Array[Int](), 0))(seqOp, combOp) println(y) x: y: aggregate(identity, seqOp, combOp) Aggregate all the elements of the RDD by: - applying a user function to combine elements with user-supplied objects, - then combining those user-defined results via a second user function, - and finally returning a result to the driver. [1, 2, 3, 4] (Array(3, 1, 2, 4),10) AGGREGATE *** **** ** *** [( ),#]
  • 110. x: y: max() Return the maximum item in the RDD [2, 4, 1] 4 MAX 4 x = sc.parallelize([2,4,1]) y = x.max() print(x.collect()) print(y) val x = sc.parallelize(Array(2,4,1)) val y = x.max println(x.collect().mkString(", ")) println(y) 2 1 4 max
  • 112. x: y: sum() Return the sum of the items in the RDD [2, 4, 1] 7 SUM 7 x = sc.parallelize([2,4,1]) y = x.sum() print(x.collect()) print(y) val x = sc.parallelize(Array(2,4,1)) val y = x.sum println(x.collect().mkString(", ")) println(y) 2 1 4 Σ
  • 114. x: y: mean() Return the mean of the items in the RDD [2, 4, 1] 2.3333333 MEAN 2.3333333 x = sc.parallelize([2,4,1]) y = x.mean() print(x.collect()) print(y) val x = sc.parallelize(Array(2,4,1)) val y = x.mean println(x.collect().mkString(", ")) println(y) 2 1 4 x
  • 116. x: y: stdev() Return the standard deviation of the items in the RDD [2, 4, 1] 1.2472191 STDEV 1.2472191 x = sc.parallelize([2,4,1]) y = x.stdev() print(x.collect()) print(y) val x = sc.parallelize(Array(2,4,1)) val y = x.stdev println(x.collect().mkString(", ")) println(y) 2 1 4 σ
  • 117. COUNTBYKEY {'A': 1, 'J': 2, 'F': 1} J “John” “Anna”A “Fred”F J “James”
  • 118. x: y: countByKey() Return a map of keys and counts of their occurrences in the RDD [('J', 'James'), ('F','Fred'), ('A','Anna'), ('J','John')] {'A': 1, 'J': 2, 'F': 1} COUNTBYKEY {A: 1, 'J': 2, 'F': 1} x = sc.parallelize([('J', 'James'), ('F','Fred'), ('A','Anna'), ('J','John')]) y = x.countByKey() print(y) val x = sc.parallelize(Array(('J',"James"),('F',"Fred"), ('A',"Anna"),('J',"John"))) val y = x.countByKey() println(y)
  • 120. x: y: saveAsTextFile(path, compressionCodecClass=None) Save the RDD to the filesystem indicated in the path [2, 4, 1] [u'2', u'4', u'1'] SAVEASTEXTFILE dbutils.fs.rm("/temp/demo", True) x = sc.parallelize([2,4,1]) x.saveAsTextFile("/temp/demo") y = sc.textFile("/temp/demo") print(y.collect()) dbutils.fs.rm("/temp/demo", true) val x = sc.parallelize(Array(2,4,1)) x.saveAsTextFile("/temp/demo") val y = sc.textFile("/temp/demo") println(y.collect().mkString(", "))
  • 121. LAB
  • 122. Q&A