SlideShare a Scribd company logo
1 of 122
Download to read offline
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

What's hot

Hdfs ha using journal nodes
Hdfs ha using journal nodesHdfs ha using journal nodes
Hdfs ha using journal nodes
Evans Ye
 
Introduction to PostgreSQL
Introduction to PostgreSQLIntroduction to PostgreSQL
Introduction to PostgreSQL
Mark Wong
 
Overcoming Today's Data Challenges with MongoDB
Overcoming Today's Data Challenges with MongoDBOvercoming Today's Data Challenges with MongoDB
Overcoming Today's Data Challenges with MongoDB
MongoDB
 

What's hot (20)

Hdfs ha using journal nodes
Hdfs ha using journal nodesHdfs ha using journal nodes
Hdfs ha using journal nodes
 
Everyday I'm Shuffling - Tips for Writing Better Spark Programs, Strata San J...
Everyday I'm Shuffling - Tips for Writing Better Spark Programs, Strata San J...Everyday I'm Shuffling - Tips for Writing Better Spark Programs, Strata San J...
Everyday I'm Shuffling - Tips for Writing Better Spark Programs, Strata San J...
 
High Performance, High Reliability Data Loading on ClickHouse
High Performance, High Reliability Data Loading on ClickHouseHigh Performance, High Reliability Data Loading on ClickHouse
High Performance, High Reliability Data Loading on ClickHouse
 
Introduction to Pig
Introduction to PigIntroduction to Pig
Introduction to Pig
 
MongoDB at Scale
MongoDB at ScaleMongoDB at Scale
MongoDB at Scale
 
The internals of gporca optimizer
The internals of gporca optimizerThe internals of gporca optimizer
The internals of gporca optimizer
 
Jvm tuning for low latency application & Cassandra
Jvm tuning for low latency application & CassandraJvm tuning for low latency application & Cassandra
Jvm tuning for low latency application & Cassandra
 
Introduction to PostgreSQL
Introduction to PostgreSQLIntroduction to PostgreSQL
Introduction to PostgreSQL
 
Oracle RAC, Oracle Data Guard, and Pluggable Databases: When MAA Meets Oracle...
Oracle RAC, Oracle Data Guard, and Pluggable Databases: When MAA Meets Oracle...Oracle RAC, Oracle Data Guard, and Pluggable Databases: When MAA Meets Oracle...
Oracle RAC, Oracle Data Guard, and Pluggable Databases: When MAA Meets Oracle...
 
Introducing DataFrames in Spark for Large Scale Data Science
Introducing DataFrames in Spark for Large Scale Data ScienceIntroducing DataFrames in Spark for Large Scale Data Science
Introducing DataFrames in Spark for Large Scale Data Science
 
Spark introduction and architecture
Spark introduction and architectureSpark introduction and architecture
Spark introduction and architecture
 
Overcoming Today's Data Challenges with MongoDB
Overcoming Today's Data Challenges with MongoDBOvercoming Today's Data Challenges with MongoDB
Overcoming Today's Data Challenges with MongoDB
 
Apache Spark Architecture
Apache Spark ArchitectureApache Spark Architecture
Apache Spark Architecture
 
A Day in the Life of a ClickHouse Query Webinar Slides
A Day in the Life of a ClickHouse Query Webinar Slides A Day in the Life of a ClickHouse Query Webinar Slides
A Day in the Life of a ClickHouse Query Webinar Slides
 
New lessons in connection management
New lessons in connection managementNew lessons in connection management
New lessons in connection management
 
Oracle Drivers configuration for High Availability
Oracle Drivers configuration for High AvailabilityOracle Drivers configuration for High Availability
Oracle Drivers configuration for High Availability
 
Mvcc in postgreSQL 권건우
Mvcc in postgreSQL 권건우Mvcc in postgreSQL 권건우
Mvcc in postgreSQL 권건우
 
Maximum Overdrive: Tuning the Spark Cassandra Connector (Russell Spitzer, Dat...
Maximum Overdrive: Tuning the Spark Cassandra Connector (Russell Spitzer, Dat...Maximum Overdrive: Tuning the Spark Cassandra Connector (Russell Spitzer, Dat...
Maximum Overdrive: Tuning the Spark Cassandra Connector (Russell Spitzer, Dat...
 
Лекция 5. MapReduce в Hadoop (алгоритмы)
Лекция 5. MapReduce в Hadoop (алгоритмы)Лекция 5. MapReduce в Hadoop (алгоритмы)
Лекция 5. MapReduce в Hadoop (алгоритмы)
 
Beyond SQL: Speeding up Spark with DataFrames
Beyond SQL: Speeding up Spark with DataFramesBeyond SQL: Speeding up Spark with DataFrames
Beyond SQL: Speeding up Spark with DataFrames
 

Similar to Visual Api Training

Latinoware
LatinowareLatinoware
Latinoware
kchodorow
 

Similar to Visual Api Training (20)

Transformations and actions a visual guide training
Transformations and actions a visual guide trainingTransformations and actions a visual guide training
Transformations and actions a visual guide training
 
Introduction to spark
Introduction to sparkIntroduction to spark
Introduction to spark
 
Introduction to HADOOP
Introduction to HADOOPIntroduction to HADOOP
Introduction to HADOOP
 
Scala meetup - Intro to spark
Scala meetup - Intro to sparkScala meetup - Intro to spark
Scala meetup - Intro to spark
 
Spark Streaming Tips for Devs and Ops
Spark Streaming Tips for Devs and OpsSpark Streaming Tips for Devs and Ops
Spark Streaming Tips for Devs and Ops
 
Spark Streaming Tips for Devs and Ops by Fran perez y federico fernĂĄndez
Spark Streaming Tips for Devs and Ops by Fran perez y federico fernĂĄndezSpark Streaming Tips for Devs and Ops by Fran perez y federico fernĂĄndez
Spark Streaming Tips for Devs and Ops by Fran perez y federico fernĂĄndez
 
User Defined Aggregation in Apache Spark: A Love Story
User Defined Aggregation in Apache Spark: A Love StoryUser Defined Aggregation in Apache Spark: A Love Story
User Defined Aggregation in Apache Spark: A Love Story
 
User Defined Aggregation in Apache Spark: A Love Story
User Defined Aggregation in Apache Spark: A Love StoryUser Defined Aggregation in Apache Spark: A Love Story
User Defined Aggregation in Apache Spark: A Love Story
 
ACADILD:: HADOOP LESSON
ACADILD:: HADOOP LESSON ACADILD:: HADOOP LESSON
ACADILD:: HADOOP LESSON
 
Grouping & Summarizing Data in R
Grouping & Summarizing Data in RGrouping & Summarizing Data in R
Grouping & Summarizing Data in R
 
MongoDB Aggregation Framework
MongoDB Aggregation FrameworkMongoDB Aggregation Framework
MongoDB Aggregation Framework
 
7 key recipes for data engineering
7 key recipes for data engineering7 key recipes for data engineering
7 key recipes for data engineering
 
Webinar: Exploring the Aggregation Framework
Webinar: Exploring the Aggregation FrameworkWebinar: Exploring the Aggregation Framework
Webinar: Exploring the Aggregation Framework
 
Intro to Map Reduce
Intro to Map ReduceIntro to Map Reduce
Intro to Map Reduce
 
MongoDB Aggregation Framework in action !
MongoDB Aggregation Framework in action !MongoDB Aggregation Framework in action !
MongoDB Aggregation Framework in action !
 
Crunching Data with Google BigQuery. JORDAN TIGANI at Big Data Spain 2012
Crunching Data with Google BigQuery. JORDAN TIGANI at Big Data Spain 2012Crunching Data with Google BigQuery. JORDAN TIGANI at Big Data Spain 2012
Crunching Data with Google BigQuery. JORDAN TIGANI at Big Data Spain 2012
 
NLP on a Billion Documents: Scalable Machine Learning with Apache Spark
NLP on a Billion Documents: Scalable Machine Learning with Apache SparkNLP on a Billion Documents: Scalable Machine Learning with Apache Spark
NLP on a Billion Documents: Scalable Machine Learning with Apache Spark
 
Scoobi - Scala for Startups
Scoobi - Scala for StartupsScoobi - Scala for Startups
Scoobi - Scala for Startups
 
Clojure for Data Science
Clojure for Data ScienceClojure for Data Science
Clojure for Data Science
 
Latinoware
LatinowareLatinoware
Latinoware
 

More from Spark Summit

Apache Spark Structured Streaming Helps Smart Manufacturing with Xiaochang Wu
Apache Spark Structured Streaming Helps Smart Manufacturing with  Xiaochang WuApache Spark Structured Streaming Helps Smart Manufacturing with  Xiaochang Wu
Apache Spark Structured Streaming Helps Smart Manufacturing with Xiaochang Wu
Spark Summit
 
Improving Traffic Prediction Using Weather Data with Ramya Raghavendra
Improving Traffic Prediction Using Weather Data  with Ramya RaghavendraImproving Traffic Prediction Using Weather Data  with Ramya Raghavendra
Improving Traffic Prediction Using Weather Data with Ramya Raghavendra
Spark Summit
 
MMLSpark: Lessons from Building a SparkML-Compatible Machine Learning Library...
MMLSpark: Lessons from Building a SparkML-Compatible Machine Learning Library...MMLSpark: Lessons from Building a SparkML-Compatible Machine Learning Library...
MMLSpark: Lessons from Building a SparkML-Compatible Machine Learning Library...
Spark Summit
 
Improving Traffic Prediction Using Weather Datawith Ramya Raghavendra
Improving Traffic Prediction Using Weather Datawith Ramya RaghavendraImproving Traffic Prediction Using Weather Datawith Ramya Raghavendra
Improving Traffic Prediction Using Weather Datawith Ramya Raghavendra
Spark Summit
 
Deduplication and Author-Disambiguation of Streaming Records via Supervised M...
Deduplication and Author-Disambiguation of Streaming Records via Supervised M...Deduplication and Author-Disambiguation of Streaming Records via Supervised M...
Deduplication and Author-Disambiguation of Streaming Records via Supervised M...
Spark Summit
 
MatFast: In-Memory Distributed Matrix Computation Processing and Optimization...
MatFast: In-Memory Distributed Matrix Computation Processing and Optimization...MatFast: In-Memory Distributed Matrix Computation Processing and Optimization...
MatFast: In-Memory Distributed Matrix Computation Processing and Optimization...
Spark Summit
 

More from Spark Summit (20)

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

Recently uploaded

CHEAP Call Girls in Rabindra Nagar (-DELHI )🔝 9953056974🔝(=)/CALL GIRLS SERVICE
CHEAP Call Girls in Rabindra Nagar  (-DELHI )🔝 9953056974🔝(=)/CALL GIRLS SERVICECHEAP Call Girls in Rabindra Nagar  (-DELHI )🔝 9953056974🔝(=)/CALL GIRLS SERVICE
CHEAP Call Girls in Rabindra Nagar (-DELHI )🔝 9953056974🔝(=)/CALL GIRLS SERVICE
9953056974 Low Rate Call Girls In Saket, Delhi NCR
 
CHEAP Call Girls in Saket (-DELHI )🔝 9953056974🔝(=)/CALL GIRLS SERVICE
CHEAP Call Girls in Saket (-DELHI )🔝 9953056974🔝(=)/CALL GIRLS SERVICECHEAP Call Girls in Saket (-DELHI )🔝 9953056974🔝(=)/CALL GIRLS SERVICE
CHEAP Call Girls in Saket (-DELHI )🔝 9953056974🔝(=)/CALL GIRLS SERVICE
9953056974 Low Rate Call Girls In Saket, Delhi NCR
 
➥🔝 7737669865 🔝▻ Bangalore Call-girls in Women Seeking Men 🔝Bangalore🔝 Esc...
➥🔝 7737669865 🔝▻ Bangalore Call-girls in Women Seeking Men  🔝Bangalore🔝   Esc...➥🔝 7737669865 🔝▻ Bangalore Call-girls in Women Seeking Men  🔝Bangalore🔝   Esc...
➥🔝 7737669865 🔝▻ Bangalore Call-girls in Women Seeking Men 🔝Bangalore🔝 Esc...
amitlee9823
 
hybrid Seed Production In Chilli & Capsicum.pptx
hybrid Seed Production In Chilli & Capsicum.pptxhybrid Seed Production In Chilli & Capsicum.pptx
hybrid Seed Production In Chilli & Capsicum.pptx
9to5mart
 
Abortion pills in Doha Qatar (+966572737505 ! Get Cytotec
Abortion pills in Doha Qatar (+966572737505 ! Get CytotecAbortion pills in Doha Qatar (+966572737505 ! Get Cytotec
Abortion pills in Doha Qatar (+966572737505 ! Get Cytotec
Abortion pills in Riyadh +966572737505 get cytotec
 
Call Girls Begur Just Call 👗 7737669865 👗 Top Class Call Girl Service Bangalore
Call Girls Begur Just Call 👗 7737669865 👗 Top Class Call Girl Service BangaloreCall Girls Begur Just Call 👗 7737669865 👗 Top Class Call Girl Service Bangalore
Call Girls Begur Just Call 👗 7737669865 👗 Top Class Call Girl Service Bangalore
amitlee9823
 
Junnasandra Call Girls: 🍓 7737669865 🍓 High Profile Model Escorts | Bangalore...
Junnasandra Call Girls: 🍓 7737669865 🍓 High Profile Model Escorts | Bangalore...Junnasandra Call Girls: 🍓 7737669865 🍓 High Profile Model Escorts | Bangalore...
Junnasandra Call Girls: 🍓 7737669865 🍓 High Profile Model Escorts | Bangalore...
amitlee9823
 
Call Girls Jalahalli Just Call 👗 7737669865 👗 Top Class Call Girl Service Ban...
Call Girls Jalahalli Just Call 👗 7737669865 👗 Top Class Call Girl Service Ban...Call Girls Jalahalli Just Call 👗 7737669865 👗 Top Class Call Girl Service Ban...
Call Girls Jalahalli Just Call 👗 7737669865 👗 Top Class Call Girl Service Ban...
amitlee9823
 
Call Girls Bommasandra Just Call 👗 7737669865 👗 Top Class Call Girl Service B...
Call Girls Bommasandra Just Call 👗 7737669865 👗 Top Class Call Girl Service B...Call Girls Bommasandra Just Call 👗 7737669865 👗 Top Class Call Girl Service B...
Call Girls Bommasandra Just Call 👗 7737669865 👗 Top Class Call Girl Service B...
amitlee9823
 
Call Girls Indiranagar Just Call 👗 9155563397 👗 Top Class Call Girl Service B...
Call Girls Indiranagar Just Call 👗 9155563397 👗 Top Class Call Girl Service B...Call Girls Indiranagar Just Call 👗 9155563397 👗 Top Class Call Girl Service B...
Call Girls Indiranagar Just Call 👗 9155563397 👗 Top Class Call Girl Service B...
only4webmaster01
 
Call Girls Hsr Layout Just Call 👗 7737669865 👗 Top Class Call Girl Service Ba...
Call Girls Hsr Layout Just Call 👗 7737669865 👗 Top Class Call Girl Service Ba...Call Girls Hsr Layout Just Call 👗 7737669865 👗 Top Class Call Girl Service Ba...
Call Girls Hsr Layout Just Call 👗 7737669865 👗 Top Class Call Girl Service Ba...
amitlee9823
 
➥🔝 7737669865 🔝▻ mahisagar Call-girls in Women Seeking Men 🔝mahisagar🔝 Esc...
➥🔝 7737669865 🔝▻ mahisagar Call-girls in Women Seeking Men  🔝mahisagar🔝   Esc...➥🔝 7737669865 🔝▻ mahisagar Call-girls in Women Seeking Men  🔝mahisagar🔝   Esc...
➥🔝 7737669865 🔝▻ mahisagar Call-girls in Women Seeking Men 🔝mahisagar🔝 Esc...
amitlee9823
 
Call Girls In Attibele ☎ 7737669865 🥵 Book Your One night Stand
Call Girls In Attibele ☎ 7737669865 🥵 Book Your One night StandCall Girls In Attibele ☎ 7737669865 🥵 Book Your One night Stand
Call Girls In Attibele ☎ 7737669865 🥵 Book Your One night Stand
amitlee9823
 
Chintamani Call Girls: 🍓 7737669865 🍓 High Profile Model Escorts | Bangalore ...
Chintamani Call Girls: 🍓 7737669865 🍓 High Profile Model Escorts | Bangalore ...Chintamani Call Girls: 🍓 7737669865 🍓 High Profile Model Escorts | Bangalore ...
Chintamani Call Girls: 🍓 7737669865 🍓 High Profile Model Escorts | Bangalore ...
amitlee9823
 

Recently uploaded (20)

CHEAP Call Girls in Rabindra Nagar (-DELHI )🔝 9953056974🔝(=)/CALL GIRLS SERVICE
CHEAP Call Girls in Rabindra Nagar  (-DELHI )🔝 9953056974🔝(=)/CALL GIRLS SERVICECHEAP Call Girls in Rabindra Nagar  (-DELHI )🔝 9953056974🔝(=)/CALL GIRLS SERVICE
CHEAP Call Girls in Rabindra Nagar (-DELHI )🔝 9953056974🔝(=)/CALL GIRLS SERVICE
 
CHEAP Call Girls in Saket (-DELHI )🔝 9953056974🔝(=)/CALL GIRLS SERVICE
CHEAP Call Girls in Saket (-DELHI )🔝 9953056974🔝(=)/CALL GIRLS SERVICECHEAP Call Girls in Saket (-DELHI )🔝 9953056974🔝(=)/CALL GIRLS SERVICE
CHEAP Call Girls in Saket (-DELHI )🔝 9953056974🔝(=)/CALL GIRLS SERVICE
 
➥🔝 7737669865 🔝▻ Bangalore Call-girls in Women Seeking Men 🔝Bangalore🔝 Esc...
➥🔝 7737669865 🔝▻ Bangalore Call-girls in Women Seeking Men  🔝Bangalore🔝   Esc...➥🔝 7737669865 🔝▻ Bangalore Call-girls in Women Seeking Men  🔝Bangalore🔝   Esc...
➥🔝 7737669865 🔝▻ Bangalore Call-girls in Women Seeking Men 🔝Bangalore🔝 Esc...
 
hybrid Seed Production In Chilli & Capsicum.pptx
hybrid Seed Production In Chilli & Capsicum.pptxhybrid Seed Production In Chilli & Capsicum.pptx
hybrid Seed Production In Chilli & Capsicum.pptx
 
Cheap Rate Call girls Sarita Vihar Delhi 9205541914 shot 1500 night
Cheap Rate Call girls Sarita Vihar Delhi 9205541914 shot 1500 nightCheap Rate Call girls Sarita Vihar Delhi 9205541914 shot 1500 night
Cheap Rate Call girls Sarita Vihar Delhi 9205541914 shot 1500 night
 
Abortion pills in Doha Qatar (+966572737505 ! Get Cytotec
Abortion pills in Doha Qatar (+966572737505 ! Get CytotecAbortion pills in Doha Qatar (+966572737505 ! Get Cytotec
Abortion pills in Doha Qatar (+966572737505 ! Get Cytotec
 
Call Girls Begur Just Call 👗 7737669865 👗 Top Class Call Girl Service Bangalore
Call Girls Begur Just Call 👗 7737669865 👗 Top Class Call Girl Service BangaloreCall Girls Begur Just Call 👗 7737669865 👗 Top Class Call Girl Service Bangalore
Call Girls Begur Just Call 👗 7737669865 👗 Top Class Call Girl Service Bangalore
 
Predicting Loan Approval: A Data Science Project
Predicting Loan Approval: A Data Science ProjectPredicting Loan Approval: A Data Science Project
Predicting Loan Approval: A Data Science Project
 
Junnasandra Call Girls: 🍓 7737669865 🍓 High Profile Model Escorts | Bangalore...
Junnasandra Call Girls: 🍓 7737669865 🍓 High Profile Model Escorts | Bangalore...Junnasandra Call Girls: 🍓 7737669865 🍓 High Profile Model Escorts | Bangalore...
Junnasandra Call Girls: 🍓 7737669865 🍓 High Profile Model Escorts | Bangalore...
 
Call Girls Jalahalli Just Call 👗 7737669865 👗 Top Class Call Girl Service Ban...
Call Girls Jalahalli Just Call 👗 7737669865 👗 Top Class Call Girl Service Ban...Call Girls Jalahalli Just Call 👗 7737669865 👗 Top Class Call Girl Service Ban...
Call Girls Jalahalli Just Call 👗 7737669865 👗 Top Class Call Girl Service Ban...
 
Anomaly detection and data imputation within time series
Anomaly detection and data imputation within time seriesAnomaly detection and data imputation within time series
Anomaly detection and data imputation within time series
 
Call Girls Bommasandra Just Call 👗 7737669865 👗 Top Class Call Girl Service B...
Call Girls Bommasandra Just Call 👗 7737669865 👗 Top Class Call Girl Service B...Call Girls Bommasandra Just Call 👗 7737669865 👗 Top Class Call Girl Service B...
Call Girls Bommasandra Just Call 👗 7737669865 👗 Top Class Call Girl Service B...
 
(NEHA) Call Girls Katra Call Now 8617697112 Katra Escorts 24x7
(NEHA) Call Girls Katra Call Now 8617697112 Katra Escorts 24x7(NEHA) Call Girls Katra Call Now 8617697112 Katra Escorts 24x7
(NEHA) Call Girls Katra Call Now 8617697112 Katra Escorts 24x7
 
Call Girls Indiranagar Just Call 👗 9155563397 👗 Top Class Call Girl Service B...
Call Girls Indiranagar Just Call 👗 9155563397 👗 Top Class Call Girl Service B...Call Girls Indiranagar Just Call 👗 9155563397 👗 Top Class Call Girl Service B...
Call Girls Indiranagar Just Call 👗 9155563397 👗 Top Class Call Girl Service B...
 
Call Girls Hsr Layout Just Call 👗 7737669865 👗 Top Class Call Girl Service Ba...
Call Girls Hsr Layout Just Call 👗 7737669865 👗 Top Class Call Girl Service Ba...Call Girls Hsr Layout Just Call 👗 7737669865 👗 Top Class Call Girl Service Ba...
Call Girls Hsr Layout Just Call 👗 7737669865 👗 Top Class Call Girl Service Ba...
 
➥🔝 7737669865 🔝▻ mahisagar Call-girls in Women Seeking Men 🔝mahisagar🔝 Esc...
➥🔝 7737669865 🔝▻ mahisagar Call-girls in Women Seeking Men  🔝mahisagar🔝   Esc...➥🔝 7737669865 🔝▻ mahisagar Call-girls in Women Seeking Men  🔝mahisagar🔝   Esc...
➥🔝 7737669865 🔝▻ mahisagar Call-girls in Women Seeking Men 🔝mahisagar🔝 Esc...
 
Call Girls In Attibele ☎ 7737669865 🥵 Book Your One night Stand
Call Girls In Attibele ☎ 7737669865 🥵 Book Your One night StandCall Girls In Attibele ☎ 7737669865 🥵 Book Your One night Stand
Call Girls In Attibele ☎ 7737669865 🥵 Book Your One night Stand
 
Digital Advertising Lecture for Advanced Digital & Social Media Strategy at U...
Digital Advertising Lecture for Advanced Digital & Social Media Strategy at U...Digital Advertising Lecture for Advanced Digital & Social Media Strategy at U...
Digital Advertising Lecture for Advanced Digital & Social Media Strategy at U...
 
Chintamani Call Girls: 🍓 7737669865 🍓 High Profile Model Escorts | Bangalore ...
Chintamani Call Girls: 🍓 7737669865 🍓 High Profile Model Escorts | Bangalore ...Chintamani Call Girls: 🍓 7737669865 🍓 High Profile Model Escorts | Bangalore ...
Chintamani Call Girls: 🍓 7737669865 🍓 High Profile Model Escorts | Bangalore ...
 
Call Girls in Sarai Kale Khan Delhi 💯 Call Us 🔝9205541914 🔝( Delhi) Escorts S...
Call Girls in Sarai Kale Khan Delhi 💯 Call Us 🔝9205541914 🔝( Delhi) Escorts S...Call Girls in Sarai Kale Khan Delhi 💯 Call Us 🔝9205541914 🔝( Delhi) Escorts S...
Call Girls in Sarai Kale Khan Delhi 💯 Call Us 🔝9205541914 🔝( Delhi) Escorts S...
 

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
  • 69. KEYBY RDD: x James Anna Fred John RDD: y J “John” emits‘J’
  • 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