SlideShare ist ein Scribd-Unternehmen logo
1 von 37
Introduction to Apache Spark
Brendan Dillon
Javier Arrieta
Spark Core
Your Applications
The Stack
Spark SQL MLLib GraphX
Spark
Streaming
Mesos YARN Standalone
sqlContext.sql("SELECT name FROM people WHERE age >= 13 AND age <=
19").map(t => "Name: " + t(0)).collect().foreach(println)
Distributed Execution
Driver
Spark
Context
Worker Node
Executor
Task
Task
Worker Node
Executor
Task
Task
Resilient Distributed Datasets (RDD)
• Immutable: never modified – just transformed to new RDDs
• Distributed: split into multiple partitions and spread across
multiple servers in a cluster
• Resilient: can be re-computed if they get destroyed
• Created by:
– Loading external data
– Distributing a collection of objects in the driver program
RDD Implementation
• Array of partitions
• List of dependencies on parent RDDs
• Function to compute a partition given its parents
– Returns Iterator over a partition
• Preferred locations: list of strings for each partition (Nil by
default)
• Partitioner (None by default)
Persistence / Caching
• By default RDDs (and all of their dependencies) are
recomputed every time an action is called on them!
• Need to explicitly tell Spark when to persist
• Options:
– Default: stored in heap as unserialized objects (pickled objects for
Python)
– Memory only: serialized or not
– Memory and disk: spills to disk, option to serialize in memory
– Disk only
• Tachyon: off-heap distributed caching
– Aims to make Spark more resilient
– Avoid GC overheads
Dependency Types: Narrow
E.g. map, filter
E.g. union
E.g. join with
co-partitioned input
Each partition of parent is used by at most
one partition of the child
Dependency Types: Wide
E.g. groupByKey
E.g. join with inputs
non co-partitioned
Each partition of the parent is used by more than
one partition of the child
Transformations
• Return a new RDD
• Lazy evaluation
• Single RDD transformations: map, flatMap, filter, distinct
• Pair RDDs: keyBy, reduceByKey, groupByKey, combineByKey,
mapValues, flatMapValues, sortByKey
• Two RDD transformations: union, intersection, subtract,
cartesian
• Two pair RDDs: join, rightOuterJoin, leftOuterJoin, cogroup
Actions
• Force evaluation of the transformations and return a value to
the driver program or write to external storage
• Actions on RDDs:
– reduce, fold, aggregate
– foreach(func), collect
– count, countByValue
– top(num)
– take(num), takeOrdered(num)(ordering)
• Actions on pair RDDs:
– countByKey
– collectAsMap
– lookup(key)
Single RDD Transformations
map and flatMap
• map takes a function that transforms each element of a
collection: map(f: T => U)
• RDD[T] => RDD[U]
• flatMap takes a function that transforms a single element of a
collection into a sequence of elements: flatMap(f: T => Seq[U])
• Flattens out the output into a single sequence
• RDD[T] => RDD[U]
filter, distinct
• filter takes a (predicate) function that returns true if an
element should be in the output collection: map(f: T => Bool)
• distinct removes duplicates from the RDD
• Both filter and distinct transform from RDD[T] => RDD[T]
Actions
reduce, fold & aggregate
• reduce takes a function that combines pairwise element of a
collection: reduce(f: (T, T) => T)
• fold is like reduce except it takes a zero value i.e. fold(zero: T)
(f: (T, T) => T)
• reduce and fold: RDD[T] => T
• aggregate is the most general form
• aggregate(zero: U)(seqOp: (U, T) => U, combOp: (U, U) => U)
• aggregate: RDD[T] => U
Pair RDD Transformations
keyBy, reduceByKey
• keyBy creates tuples of the elements in an RDD by applying a
function: keyBy(f: T => K)
• RDD[ T ] => RDD[ (K, T) ]
• reduceByKey takes a function that takes a two values and
returns a single value: reduceByKey(f: (V,V) => V)
• RDD[ (K, V) ] => RDD[ (K, V) ]
groupByKey
• Takes a collection of key-value pairs and no parameters
• Returns a sequence of values associated with each key
• RDD[ ( K, V ) ] => RDD[ ( K, Iterable[V] ) ]
• Results must fit in memory
• Can be slow – use aggregateByKey or reduceByKey where
possible
• Ordering of values not guaranteed and can vary on every
evaluation
combineByKey
• def combineByKey[C](createCombiner: V => C,
mergeValue: (C, V) => C,
mergeCombiners: (C, C) => C,
partitioner: Partitioner,
mapSideCombine: Boolean = true,
serializer: Serializer = null)
• RDD [ (K, V) ] => RDD[ (K, C) ]
• createCombiner called per partition when a new key is found
• mergeValue combines a new value to an existing accumulator
• mergeCombiners with results from different partitions
• Sometimes map-size combine not useful e.g. groupByKey
• groupByKey, aggregateByKey and reduceByKey all implemented
using combineByKey
map vs mapValues
• map takes a function that transforms each element of a
collection: map(f: T => U)
• RDD[T] => RDD[U]
• When T is a tuple we may want to only act on the values – not
the keys
• mapValues takes a function that maps the values in the inputs
to the values in the output: mapValues(f: V => W)
• Where RDD[ (K, V) ] => RDD[ (K, W) ]
• NB: use mapValues when you can: avoids reshuffle when data
is partitioned by key
Two RDD Transformations
Pseudo-set: union, intersection, subtract,
cartesian
• rdd.union(otherRdd): RRD containing elements from both
• rdd.intersection(otherRdd): RDD containing only elements
found in both
• rdd.subtract(otherRdd): remove content of one from the other
e.g. removing training data
• rdd.cartesian(otherRdd): Cartesian product of two RDDs e.g.
similarity of pairs: RDD[T] RDD[U] => RDD[ (T, U) ]
Two Pair RDD Transformations
join, rightOuterJoin, leftOuterJoin, cogroup
• Join: RDD[ ( K, V) ] and RDD[ (K, W) ] => RDD[ ( K, (V,W) ) ]
• Cogroup: RDD[ ( K, V) ] and RDD[ (K, W) ] => RDD[ ( K, ( Seq[V],
Seq[W] ) ) ]
• rightOuterJoin and leftRightJoin when keys must be present in
left / right RDD
Partition-specific
Transformations and Actions
mapPartitions, mapPartitionsWithIndex, and
foreachPartition
• Same as map and foreach except they operate on a per
partition basis
• Useful for when you have setup code (DB, RNG etc.) but don’t
want to call it for each partition
• You can set preservesPartitioning when you are not altering
the keys used for partitioning to avoid unnecessary shuffling
– As with mapValues in the last slide
Data Frames
Data Frames & Catalyst Optimizer
DataFrame creation and operations
val sc: SparkContext // An existing SparkContext.
val sqlContext = new org.apache.spark.sql.SQLContext(sc)
// Create the DataFrame
val df = sqlContext.jsonFile("examples/src/main/resources/people.json”)
// Show the content of the DataFrame
df.show()
// Print the schema in a tree format
df.printSchema()
// Select only the "name" column
df.select("name”)
// Select everybody, but increment the age
by 1
df.select("name", df("age") + 1)
// Select people older than 21
df.filter(df("name") > 21)
// Count people by age
df.groupBy("age").count()
Spark Streaming
Introduction
Alternatives
Apache Storm
Trident
Programming
Model
Micro-Batch One at a time Micro-batch
Stream Primitive
DStream Stream Tuple, Tuple Batch,
Partition
Distributed Stream
Dataflow
Stream Source
ReceiverInputDStr
eam
Container Spouts, Trident
Spouts
Data Stream
Computation
Maps/windows/op
erations on
Dstream
StreamTask,
Window, join
Filters, functions,
aggregations, joins
Maps/windows/op
erations on Data
Stream
Resource mgmt YARN/Mesos YARN YARN/Mesos YARN
Resilience
Require WAL to
DFS (HDFS/S3)
Checkpointing
(Kafka)
Nimbus reassigns
and failed batch
replayed
Lightweight
Distributed
Snapshots
Scala collections programming model, map, flatMap, window,
reduce (fold)
share code between batch and streaming, both share the same
programming model (although different semantics)
microbatches allow aggregation on the batches, improved
throughput with a latency cost
Why Spark Streaming
Spark Streaming Execution
Driver
Spark
Context
Worker Node
Executor
Task
Task
Worker Node
Executor
Task
Task
Worker Node
Executor
Task
Task
Streaming
Producer
Example overview
Code
val metaStream = stream.map { case (k, v) => (k,
DocumentMetadata.fromMutable(recordDecoder.decode(v).asInstanceOf[GenericRecord])) }
private val pdfFiles = metaStream.filter(_._2.contentType == "application/pdf")
.map { case (k, meta) => (meta, fetchFileFromMessage(k, meta)) }
val pdfDocs = pdfFiles.map { case (meta, file) => (meta, TextExtractor.parseFile(file)) }
val texts = pdfDocs.map { case (meta, doc) => (meta, TextExtractor.extractText(doc)) }.cache()
val wordStream = texts.map { case (meta, text) => (meta, text.split("""[
nrtu00a0]+""").toList.map(_.replaceAll("""[,;.]$""", "").trim.toLowerCase()).filter(_.length >
1)) }
texts.foreachRDD( rdd => rdd.foreach { case (meta,text) => indexText(meta.id, text) } )
val wordCountStream = wordStream.flatMap(_._2).map(word => (word, 1)).reduceByKey(_ + _)
val totalWordCountStream = wordStream.map(_._2.size)
val totalWords = totalWordCountStream.reduce(_+_)
val sortedWordCount = wordCountStream.transform(rdd => rdd.sortBy(_._2, ascending = false))
sortedWordCount.foreachRDD(rdd => println(rdd.toDebugString))
sortedWordCount.print(30)
totalWords.print()
Q & A

Weitere ähnliche Inhalte

Was ist angesagt?

SparkSQL and Dataframe
SparkSQL and DataframeSparkSQL and Dataframe
SparkSQL and DataframeNamgee Lee
 
Spark Dataframe - Mr. Jyotiska
Spark Dataframe - Mr. JyotiskaSpark Dataframe - Mr. Jyotiska
Spark Dataframe - Mr. JyotiskaSigmoid
 
Apache Spark: What? Why? When?
Apache Spark: What? Why? When?Apache Spark: What? Why? When?
Apache Spark: What? Why? When?Massimo Schenone
 
Spark + Clojure for Topic Discovery - Zalando Tech Clojure/Conj Talk
Spark + Clojure for Topic Discovery - Zalando Tech Clojure/Conj TalkSpark + Clojure for Topic Discovery - Zalando Tech Clojure/Conj Talk
Spark + Clojure for Topic Discovery - Zalando Tech Clojure/Conj TalkZalando Technology
 
Tulsa techfest Spark Core Aug 5th 2016
Tulsa techfest Spark Core Aug 5th 2016Tulsa techfest Spark Core Aug 5th 2016
Tulsa techfest Spark Core Aug 5th 2016Mark Smith
 
Big Data Analytics with Scala at SCALA.IO 2013
Big Data Analytics with Scala at SCALA.IO 2013Big Data Analytics with Scala at SCALA.IO 2013
Big Data Analytics with Scala at SCALA.IO 2013Samir Bessalah
 
Big Data Analytics with Hadoop with @techmilind
Big Data Analytics with Hadoop with @techmilindBig Data Analytics with Hadoop with @techmilind
Big Data Analytics with Hadoop with @techmilindEMC
 
Engineering Fast Indexes for Big-Data Applications: Spark Summit East talk by...
Engineering Fast Indexes for Big-Data Applications: Spark Summit East talk by...Engineering Fast Indexes for Big-Data Applications: Spark Summit East talk by...
Engineering Fast Indexes for Big-Data Applications: Spark Summit East talk by...Spark Summit
 
A Scalable Hierarchical Clustering Algorithm Using Spark: Spark Summit East t...
A Scalable Hierarchical Clustering Algorithm Using Spark: Spark Summit East t...A Scalable Hierarchical Clustering Algorithm Using Spark: Spark Summit East t...
A Scalable Hierarchical Clustering Algorithm Using Spark: Spark Summit East t...Spark Summit
 
MongoDB Chicago - MapReduce, Geospatial, & Other Cool Features
MongoDB Chicago - MapReduce, Geospatial, & Other Cool FeaturesMongoDB Chicago - MapReduce, Geospatial, & Other Cool Features
MongoDB Chicago - MapReduce, Geospatial, & Other Cool Featuresajhannan
 
Lecture 2: Data-Intensive Computing for Text Analysis (Fall 2011)
Lecture 2: Data-Intensive Computing for Text Analysis (Fall 2011)Lecture 2: Data-Intensive Computing for Text Analysis (Fall 2011)
Lecture 2: Data-Intensive Computing for Text Analysis (Fall 2011)Matthew Lease
 
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 StoryDatabricks
 
Data Source API in Spark
Data Source API in SparkData Source API in Spark
Data Source API in SparkDatabricks
 
Machine learning using spark
Machine learning using sparkMachine learning using spark
Machine learning using sparkRan Silberman
 
Mapreduce Algorithms
Mapreduce AlgorithmsMapreduce Algorithms
Mapreduce AlgorithmsAmund Tveit
 
BDAS Shark study report 03 v1.1
BDAS Shark study report  03 v1.1BDAS Shark study report  03 v1.1
BDAS Shark study report 03 v1.1Stefanie Zhao
 
Introduction to spatial data analysis in r
Introduction to spatial data analysis in rIntroduction to spatial data analysis in r
Introduction to spatial data analysis in rRichard Wamalwa
 

Was ist angesagt? (20)

SparkSQL and Dataframe
SparkSQL and DataframeSparkSQL and Dataframe
SparkSQL and Dataframe
 
Spark Dataframe - Mr. Jyotiska
Spark Dataframe - Mr. JyotiskaSpark Dataframe - Mr. Jyotiska
Spark Dataframe - Mr. Jyotiska
 
Distributed computing with spark
Distributed computing with sparkDistributed computing with spark
Distributed computing with spark
 
Raster package jacob
Raster package jacobRaster package jacob
Raster package jacob
 
Apache Spark: What? Why? When?
Apache Spark: What? Why? When?Apache Spark: What? Why? When?
Apache Spark: What? Why? When?
 
Spark + Clojure for Topic Discovery - Zalando Tech Clojure/Conj Talk
Spark + Clojure for Topic Discovery - Zalando Tech Clojure/Conj TalkSpark + Clojure for Topic Discovery - Zalando Tech Clojure/Conj Talk
Spark + Clojure for Topic Discovery - Zalando Tech Clojure/Conj Talk
 
Tulsa techfest Spark Core Aug 5th 2016
Tulsa techfest Spark Core Aug 5th 2016Tulsa techfest Spark Core Aug 5th 2016
Tulsa techfest Spark Core Aug 5th 2016
 
Big Data Analytics with Scala at SCALA.IO 2013
Big Data Analytics with Scala at SCALA.IO 2013Big Data Analytics with Scala at SCALA.IO 2013
Big Data Analytics with Scala at SCALA.IO 2013
 
Big Data Analytics with Hadoop with @techmilind
Big Data Analytics with Hadoop with @techmilindBig Data Analytics with Hadoop with @techmilind
Big Data Analytics with Hadoop with @techmilind
 
Engineering Fast Indexes for Big-Data Applications: Spark Summit East talk by...
Engineering Fast Indexes for Big-Data Applications: Spark Summit East talk by...Engineering Fast Indexes for Big-Data Applications: Spark Summit East talk by...
Engineering Fast Indexes for Big-Data Applications: Spark Summit East talk by...
 
A Scalable Hierarchical Clustering Algorithm Using Spark: Spark Summit East t...
A Scalable Hierarchical Clustering Algorithm Using Spark: Spark Summit East t...A Scalable Hierarchical Clustering Algorithm Using Spark: Spark Summit East t...
A Scalable Hierarchical Clustering Algorithm Using Spark: Spark Summit East t...
 
MongoDB Chicago - MapReduce, Geospatial, & Other Cool Features
MongoDB Chicago - MapReduce, Geospatial, & Other Cool FeaturesMongoDB Chicago - MapReduce, Geospatial, & Other Cool Features
MongoDB Chicago - MapReduce, Geospatial, & Other Cool Features
 
Spark training-in-bangalore
Spark training-in-bangaloreSpark training-in-bangalore
Spark training-in-bangalore
 
Lecture 2: Data-Intensive Computing for Text Analysis (Fall 2011)
Lecture 2: Data-Intensive Computing for Text Analysis (Fall 2011)Lecture 2: Data-Intensive Computing for Text Analysis (Fall 2011)
Lecture 2: Data-Intensive Computing for Text Analysis (Fall 2011)
 
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
 
Data Source API in Spark
Data Source API in SparkData Source API in Spark
Data Source API in Spark
 
Machine learning using spark
Machine learning using sparkMachine learning using spark
Machine learning using spark
 
Mapreduce Algorithms
Mapreduce AlgorithmsMapreduce Algorithms
Mapreduce Algorithms
 
BDAS Shark study report 03 v1.1
BDAS Shark study report  03 v1.1BDAS Shark study report  03 v1.1
BDAS Shark study report 03 v1.1
 
Introduction to spatial data analysis in r
Introduction to spatial data analysis in rIntroduction to spatial data analysis in r
Introduction to spatial data analysis in r
 

Andere mochten auch

Functional Programming for OO Programmers (part 2)
Functional Programming for OO Programmers (part 2)Functional Programming for OO Programmers (part 2)
Functional Programming for OO Programmers (part 2)Calvin Cheng
 
Data Science with Spark
Data Science with SparkData Science with Spark
Data Science with SparkKrishna Sankar
 
Advanced Data Science on Spark-(Reza Zadeh, Stanford)
Advanced Data Science on Spark-(Reza Zadeh, Stanford)Advanced Data Science on Spark-(Reza Zadeh, Stanford)
Advanced Data Science on Spark-(Reza Zadeh, Stanford)Spark Summit
 
Spark tutorial py con 2016 part 2
Spark tutorial py con 2016   part 2Spark tutorial py con 2016   part 2
Spark tutorial py con 2016 part 2David Taieb
 
Data Science with Spark - Training at SparkSummit (East)
Data Science with Spark - Training at SparkSummit (East)Data Science with Spark - Training at SparkSummit (East)
Data Science with Spark - Training at SparkSummit (East)Krishna Sankar
 
Spark tutorial pycon 2016 part 1
Spark tutorial pycon 2016   part 1Spark tutorial pycon 2016   part 1
Spark tutorial pycon 2016 part 1David Taieb
 
Apache Spark Tutorial
Apache Spark TutorialApache Spark Tutorial
Apache Spark TutorialAhmet Bulut
 
Towards Aggregate Programming in Scala
Towards Aggregate Programming in ScalaTowards Aggregate Programming in Scala
Towards Aggregate Programming in ScalaRoberto Casadei
 
CabMe(intercity travels)
CabMe(intercity travels)CabMe(intercity travels)
CabMe(intercity travels)SHUBHAM GUPTA
 
CabMe|intercity travels
CabMe|intercity travelsCabMe|intercity travels
CabMe|intercity travelsSHUBHAM GUPTA
 
Spark tutorial @ KCC 2015
Spark tutorial @ KCC 2015Spark tutorial @ KCC 2015
Spark tutorial @ KCC 2015Jongwook Woo
 
How To Use Higher Order Functions in Scala
How To Use Higher Order Functions in ScalaHow To Use Higher Order Functions in Scala
How To Use Higher Order Functions in ScalaBoldRadius Solutions
 
An Introduct to Spark - Atlanta Spark Meetup
An Introduct to Spark - Atlanta Spark MeetupAn Introduct to Spark - Atlanta Spark Meetup
An Introduct to Spark - Atlanta Spark Meetupjlacefie
 

Andere mochten auch (20)

ScalaTrainings
ScalaTrainingsScalaTrainings
ScalaTrainings
 
Scala+RDD
Scala+RDDScala+RDD
Scala+RDD
 
Functional Programming for OO Programmers (part 2)
Functional Programming for OO Programmers (part 2)Functional Programming for OO Programmers (part 2)
Functional Programming for OO Programmers (part 2)
 
Scala+spark 2nd
Scala+spark 2ndScala+spark 2nd
Scala+spark 2nd
 
Data Science with Spark
Data Science with SparkData Science with Spark
Data Science with Spark
 
Advanced Data Science on Spark-(Reza Zadeh, Stanford)
Advanced Data Science on Spark-(Reza Zadeh, Stanford)Advanced Data Science on Spark-(Reza Zadeh, Stanford)
Advanced Data Science on Spark-(Reza Zadeh, Stanford)
 
Spark tutorial py con 2016 part 2
Spark tutorial py con 2016   part 2Spark tutorial py con 2016   part 2
Spark tutorial py con 2016 part 2
 
Data Science with Spark - Training at SparkSummit (East)
Data Science with Spark - Training at SparkSummit (East)Data Science with Spark - Training at SparkSummit (East)
Data Science with Spark - Training at SparkSummit (East)
 
Spark tutorial pycon 2016 part 1
Spark tutorial pycon 2016   part 1Spark tutorial pycon 2016   part 1
Spark tutorial pycon 2016 part 1
 
Apache Spark Tutorial
Apache Spark TutorialApache Spark Tutorial
Apache Spark Tutorial
 
Workshop Scala
Workshop ScalaWorkshop Scala
Workshop Scala
 
[Start] Scala
[Start] Scala[Start] Scala
[Start] Scala
 
Towards Aggregate Programming in Scala
Towards Aggregate Programming in ScalaTowards Aggregate Programming in Scala
Towards Aggregate Programming in Scala
 
CabMe(intercity travels)
CabMe(intercity travels)CabMe(intercity travels)
CabMe(intercity travels)
 
CabMe|intercity travels
CabMe|intercity travelsCabMe|intercity travels
CabMe|intercity travels
 
Spark Jobserver
Spark JobserverSpark Jobserver
Spark Jobserver
 
Spark tutorial @ KCC 2015
Spark tutorial @ KCC 2015Spark tutorial @ KCC 2015
Spark tutorial @ KCC 2015
 
NYC_2016_slides
NYC_2016_slidesNYC_2016_slides
NYC_2016_slides
 
How To Use Higher Order Functions in Scala
How To Use Higher Order Functions in ScalaHow To Use Higher Order Functions in Scala
How To Use Higher Order Functions in Scala
 
An Introduct to Spark - Atlanta Spark Meetup
An Introduct to Spark - Atlanta Spark MeetupAn Introduct to Spark - Atlanta Spark Meetup
An Introduct to Spark - Atlanta Spark Meetup
 

Ähnlich wie Scala meetup - Intro to spark

Transformations and actions a visual guide training
Transformations and actions a visual guide trainingTransformations and actions a visual guide training
Transformations and actions a visual guide trainingSpark Summit
 
Introduction to spark
Introduction to sparkIntroduction to spark
Introduction to sparkDuyhai Doan
 
Tuning and Debugging in Apache Spark
Tuning and Debugging in Apache SparkTuning and Debugging in Apache Spark
Tuning and Debugging in Apache SparkDatabricks
 
Tuning and Debugging in Apache Spark
Tuning and Debugging in Apache SparkTuning and Debugging in Apache Spark
Tuning and Debugging in Apache SparkPatrick Wendell
 
Apache Spark — Fundamentals and MLlib
Apache Spark — Fundamentals and MLlibApache Spark — Fundamentals and MLlib
Apache Spark — Fundamentals and MLlibJens Fisseler, Dr.
 
Apache Spark - Basics of RDD | Big Data Hadoop Spark Tutorial | CloudxLab
Apache Spark - Basics of RDD | Big Data Hadoop Spark Tutorial | CloudxLabApache Spark - Basics of RDD | Big Data Hadoop Spark Tutorial | CloudxLab
Apache Spark - Basics of RDD | Big Data Hadoop Spark Tutorial | CloudxLabCloudxLab
 
Introduction to Apache Spark
Introduction to Apache SparkIntroduction to Apache Spark
Introduction to Apache SparkDatio Big Data
 
Dive into spark2
Dive into spark2Dive into spark2
Dive into spark2Gal Marder
 
AI與大數據數據處理 Spark實戰(20171216)
AI與大數據數據處理 Spark實戰(20171216)AI與大數據數據處理 Spark實戰(20171216)
AI與大數據數據處理 Spark實戰(20171216)Paul Chao
 
Introduction to Apache Spark
Introduction to Apache SparkIntroduction to Apache Spark
Introduction to Apache SparkVincent Poncet
 
Big Data Day LA 2016/ Hadoop/ Spark/ Kafka track - Iterative Spark Developmen...
Big Data Day LA 2016/ Hadoop/ Spark/ Kafka track - Iterative Spark Developmen...Big Data Day LA 2016/ Hadoop/ Spark/ Kafka track - Iterative Spark Developmen...
Big Data Day LA 2016/ Hadoop/ Spark/ Kafka track - Iterative Spark Developmen...Data Con LA
 
11. From Hadoop to Spark 2/2
11. From Hadoop to Spark 2/211. From Hadoop to Spark 2/2
11. From Hadoop to Spark 2/2Fabio Fumarola
 
Apache Spark and DataStax Enablement
Apache Spark and DataStax EnablementApache Spark and DataStax Enablement
Apache Spark and DataStax EnablementVincent Poncet
 
Spark as the Gateway Drug to Typed Functional Programming: Spark Summit East ...
Spark as the Gateway Drug to Typed Functional Programming: Spark Summit East ...Spark as the Gateway Drug to Typed Functional Programming: Spark Summit East ...
Spark as the Gateway Drug to Typed Functional Programming: Spark Summit East ...Spark Summit
 
Structuring Spark: DataFrames, Datasets, and Streaming
Structuring Spark: DataFrames, Datasets, and StreamingStructuring Spark: DataFrames, Datasets, and Streaming
Structuring Spark: DataFrames, Datasets, and StreamingDatabricks
 
Think Like Spark
Think Like SparkThink Like Spark
Think Like SparkAlpine Data
 

Ähnlich wie Scala meetup - Intro to spark (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
 
Tuning and Debugging in Apache Spark
Tuning and Debugging in Apache SparkTuning and Debugging in Apache Spark
Tuning and Debugging in Apache Spark
 
Tuning and Debugging in Apache Spark
Tuning and Debugging in Apache SparkTuning and Debugging in Apache Spark
Tuning and Debugging in Apache Spark
 
Apache Spark — Fundamentals and MLlib
Apache Spark — Fundamentals and MLlibApache Spark — Fundamentals and MLlib
Apache Spark — Fundamentals and MLlib
 
Apache Spark - Basics of RDD | Big Data Hadoop Spark Tutorial | CloudxLab
Apache Spark - Basics of RDD | Big Data Hadoop Spark Tutorial | CloudxLabApache Spark - Basics of RDD | Big Data Hadoop Spark Tutorial | CloudxLab
Apache Spark - Basics of RDD | Big Data Hadoop Spark Tutorial | CloudxLab
 
Introduction to Apache Spark
Introduction to Apache SparkIntroduction to Apache Spark
Introduction to Apache Spark
 
Dive into spark2
Dive into spark2Dive into spark2
Dive into spark2
 
AI與大數據數據處理 Spark實戰(20171216)
AI與大數據數據處理 Spark實戰(20171216)AI與大數據數據處理 Spark實戰(20171216)
AI與大數據數據處理 Spark實戰(20171216)
 
Introduction to Apache Spark
Introduction to Apache SparkIntroduction to Apache Spark
Introduction to Apache Spark
 
Apache Spark & Streaming
Apache Spark & StreamingApache Spark & Streaming
Apache Spark & Streaming
 
Scala and spark
Scala and sparkScala and spark
Scala and spark
 
Big Data Day LA 2016/ Hadoop/ Spark/ Kafka track - Iterative Spark Developmen...
Big Data Day LA 2016/ Hadoop/ Spark/ Kafka track - Iterative Spark Developmen...Big Data Day LA 2016/ Hadoop/ Spark/ Kafka track - Iterative Spark Developmen...
Big Data Day LA 2016/ Hadoop/ Spark/ Kafka track - Iterative Spark Developmen...
 
11. From Hadoop to Spark 2/2
11. From Hadoop to Spark 2/211. From Hadoop to Spark 2/2
11. From Hadoop to Spark 2/2
 
Operations on rdd
Operations on rddOperations on rdd
Operations on rdd
 
Apache Spark and DataStax Enablement
Apache Spark and DataStax EnablementApache Spark and DataStax Enablement
Apache Spark and DataStax Enablement
 
hadoop
hadoophadoop
hadoop
 
Spark as the Gateway Drug to Typed Functional Programming: Spark Summit East ...
Spark as the Gateway Drug to Typed Functional Programming: Spark Summit East ...Spark as the Gateway Drug to Typed Functional Programming: Spark Summit East ...
Spark as the Gateway Drug to Typed Functional Programming: Spark Summit East ...
 
Structuring Spark: DataFrames, Datasets, and Streaming
Structuring Spark: DataFrames, Datasets, and StreamingStructuring Spark: DataFrames, Datasets, and Streaming
Structuring Spark: DataFrames, Datasets, and Streaming
 
Think Like Spark
Think Like SparkThink Like Spark
Think Like Spark
 

Kürzlich hochgeladen

Long journey of Ruby standard library at RubyConf AU 2024
Long journey of Ruby standard library at RubyConf AU 2024Long journey of Ruby standard library at RubyConf AU 2024
Long journey of Ruby standard library at RubyConf AU 2024Hiroshi SHIBATA
 
How AI, OpenAI, and ChatGPT impact business and software.
How AI, OpenAI, and ChatGPT impact business and software.How AI, OpenAI, and ChatGPT impact business and software.
How AI, OpenAI, and ChatGPT impact business and software.Curtis Poe
 
The Ultimate Guide to Choosing WordPress Pros and Cons
The Ultimate Guide to Choosing WordPress Pros and ConsThe Ultimate Guide to Choosing WordPress Pros and Cons
The Ultimate Guide to Choosing WordPress Pros and ConsPixlogix Infotech
 
The Fit for Passkeys for Employee and Consumer Sign-ins: FIDO Paris Seminar.pptx
The Fit for Passkeys for Employee and Consumer Sign-ins: FIDO Paris Seminar.pptxThe Fit for Passkeys for Employee and Consumer Sign-ins: FIDO Paris Seminar.pptx
The Fit for Passkeys for Employee and Consumer Sign-ins: FIDO Paris Seminar.pptxLoriGlavin3
 
Use of FIDO in the Payments and Identity Landscape: FIDO Paris Seminar.pptx
Use of FIDO in the Payments and Identity Landscape: FIDO Paris Seminar.pptxUse of FIDO in the Payments and Identity Landscape: FIDO Paris Seminar.pptx
Use of FIDO in the Payments and Identity Landscape: FIDO Paris Seminar.pptxLoriGlavin3
 
Digital Identity is Under Attack: FIDO Paris Seminar.pptx
Digital Identity is Under Attack: FIDO Paris Seminar.pptxDigital Identity is Under Attack: FIDO Paris Seminar.pptx
Digital Identity is Under Attack: FIDO Paris Seminar.pptxLoriGlavin3
 
Generative Artificial Intelligence: How generative AI works.pdf
Generative Artificial Intelligence: How generative AI works.pdfGenerative Artificial Intelligence: How generative AI works.pdf
Generative Artificial Intelligence: How generative AI works.pdfIngrid Airi González
 
The State of Passkeys with FIDO Alliance.pptx
The State of Passkeys with FIDO Alliance.pptxThe State of Passkeys with FIDO Alliance.pptx
The State of Passkeys with FIDO Alliance.pptxLoriGlavin3
 
So einfach geht modernes Roaming fuer Notes und Nomad.pdf
So einfach geht modernes Roaming fuer Notes und Nomad.pdfSo einfach geht modernes Roaming fuer Notes und Nomad.pdf
So einfach geht modernes Roaming fuer Notes und Nomad.pdfpanagenda
 
Arizona Broadband Policy Past, Present, and Future Presentation 3/25/24
Arizona Broadband Policy Past, Present, and Future Presentation 3/25/24Arizona Broadband Policy Past, Present, and Future Presentation 3/25/24
Arizona Broadband Policy Past, Present, and Future Presentation 3/25/24Mark Goldstein
 
Time Series Foundation Models - current state and future directions
Time Series Foundation Models - current state and future directionsTime Series Foundation Models - current state and future directions
Time Series Foundation Models - current state and future directionsNathaniel Shimoni
 
Decarbonising Buildings: Making a net-zero built environment a reality
Decarbonising Buildings: Making a net-zero built environment a realityDecarbonising Buildings: Making a net-zero built environment a reality
Decarbonising Buildings: Making a net-zero built environment a realityIES VE
 
Take control of your SAP testing with UiPath Test Suite
Take control of your SAP testing with UiPath Test SuiteTake control of your SAP testing with UiPath Test Suite
Take control of your SAP testing with UiPath Test SuiteDianaGray10
 
Generative AI for Technical Writer or Information Developers
Generative AI for Technical Writer or Information DevelopersGenerative AI for Technical Writer or Information Developers
Generative AI for Technical Writer or Information DevelopersRaghuram Pandurangan
 
The Role of FIDO in a Cyber Secure Netherlands: FIDO Paris Seminar.pptx
The Role of FIDO in a Cyber Secure Netherlands: FIDO Paris Seminar.pptxThe Role of FIDO in a Cyber Secure Netherlands: FIDO Paris Seminar.pptx
The Role of FIDO in a Cyber Secure Netherlands: FIDO Paris Seminar.pptxLoriGlavin3
 
Assure Ecommerce and Retail Operations Uptime with ThousandEyes
Assure Ecommerce and Retail Operations Uptime with ThousandEyesAssure Ecommerce and Retail Operations Uptime with ThousandEyes
Assure Ecommerce and Retail Operations Uptime with ThousandEyesThousandEyes
 
TrustArc Webinar - How to Build Consumer Trust Through Data Privacy
TrustArc Webinar - How to Build Consumer Trust Through Data PrivacyTrustArc Webinar - How to Build Consumer Trust Through Data Privacy
TrustArc Webinar - How to Build Consumer Trust Through Data PrivacyTrustArc
 
Manual 508 Accessibility Compliance Audit
Manual 508 Accessibility Compliance AuditManual 508 Accessibility Compliance Audit
Manual 508 Accessibility Compliance AuditSkynet Technologies
 
New from BookNet Canada for 2024: Loan Stars - Tech Forum 2024
New from BookNet Canada for 2024: Loan Stars - Tech Forum 2024New from BookNet Canada for 2024: Loan Stars - Tech Forum 2024
New from BookNet Canada for 2024: Loan Stars - Tech Forum 2024BookNet Canada
 
Emixa Mendix Meetup 11 April 2024 about Mendix Native development
Emixa Mendix Meetup 11 April 2024 about Mendix Native developmentEmixa Mendix Meetup 11 April 2024 about Mendix Native development
Emixa Mendix Meetup 11 April 2024 about Mendix Native developmentPim van der Noll
 

Kürzlich hochgeladen (20)

Long journey of Ruby standard library at RubyConf AU 2024
Long journey of Ruby standard library at RubyConf AU 2024Long journey of Ruby standard library at RubyConf AU 2024
Long journey of Ruby standard library at RubyConf AU 2024
 
How AI, OpenAI, and ChatGPT impact business and software.
How AI, OpenAI, and ChatGPT impact business and software.How AI, OpenAI, and ChatGPT impact business and software.
How AI, OpenAI, and ChatGPT impact business and software.
 
The Ultimate Guide to Choosing WordPress Pros and Cons
The Ultimate Guide to Choosing WordPress Pros and ConsThe Ultimate Guide to Choosing WordPress Pros and Cons
The Ultimate Guide to Choosing WordPress Pros and Cons
 
The Fit for Passkeys for Employee and Consumer Sign-ins: FIDO Paris Seminar.pptx
The Fit for Passkeys for Employee and Consumer Sign-ins: FIDO Paris Seminar.pptxThe Fit for Passkeys for Employee and Consumer Sign-ins: FIDO Paris Seminar.pptx
The Fit for Passkeys for Employee and Consumer Sign-ins: FIDO Paris Seminar.pptx
 
Use of FIDO in the Payments and Identity Landscape: FIDO Paris Seminar.pptx
Use of FIDO in the Payments and Identity Landscape: FIDO Paris Seminar.pptxUse of FIDO in the Payments and Identity Landscape: FIDO Paris Seminar.pptx
Use of FIDO in the Payments and Identity Landscape: FIDO Paris Seminar.pptx
 
Digital Identity is Under Attack: FIDO Paris Seminar.pptx
Digital Identity is Under Attack: FIDO Paris Seminar.pptxDigital Identity is Under Attack: FIDO Paris Seminar.pptx
Digital Identity is Under Attack: FIDO Paris Seminar.pptx
 
Generative Artificial Intelligence: How generative AI works.pdf
Generative Artificial Intelligence: How generative AI works.pdfGenerative Artificial Intelligence: How generative AI works.pdf
Generative Artificial Intelligence: How generative AI works.pdf
 
The State of Passkeys with FIDO Alliance.pptx
The State of Passkeys with FIDO Alliance.pptxThe State of Passkeys with FIDO Alliance.pptx
The State of Passkeys with FIDO Alliance.pptx
 
So einfach geht modernes Roaming fuer Notes und Nomad.pdf
So einfach geht modernes Roaming fuer Notes und Nomad.pdfSo einfach geht modernes Roaming fuer Notes und Nomad.pdf
So einfach geht modernes Roaming fuer Notes und Nomad.pdf
 
Arizona Broadband Policy Past, Present, and Future Presentation 3/25/24
Arizona Broadband Policy Past, Present, and Future Presentation 3/25/24Arizona Broadband Policy Past, Present, and Future Presentation 3/25/24
Arizona Broadband Policy Past, Present, and Future Presentation 3/25/24
 
Time Series Foundation Models - current state and future directions
Time Series Foundation Models - current state and future directionsTime Series Foundation Models - current state and future directions
Time Series Foundation Models - current state and future directions
 
Decarbonising Buildings: Making a net-zero built environment a reality
Decarbonising Buildings: Making a net-zero built environment a realityDecarbonising Buildings: Making a net-zero built environment a reality
Decarbonising Buildings: Making a net-zero built environment a reality
 
Take control of your SAP testing with UiPath Test Suite
Take control of your SAP testing with UiPath Test SuiteTake control of your SAP testing with UiPath Test Suite
Take control of your SAP testing with UiPath Test Suite
 
Generative AI for Technical Writer or Information Developers
Generative AI for Technical Writer or Information DevelopersGenerative AI for Technical Writer or Information Developers
Generative AI for Technical Writer or Information Developers
 
The Role of FIDO in a Cyber Secure Netherlands: FIDO Paris Seminar.pptx
The Role of FIDO in a Cyber Secure Netherlands: FIDO Paris Seminar.pptxThe Role of FIDO in a Cyber Secure Netherlands: FIDO Paris Seminar.pptx
The Role of FIDO in a Cyber Secure Netherlands: FIDO Paris Seminar.pptx
 
Assure Ecommerce and Retail Operations Uptime with ThousandEyes
Assure Ecommerce and Retail Operations Uptime with ThousandEyesAssure Ecommerce and Retail Operations Uptime with ThousandEyes
Assure Ecommerce and Retail Operations Uptime with ThousandEyes
 
TrustArc Webinar - How to Build Consumer Trust Through Data Privacy
TrustArc Webinar - How to Build Consumer Trust Through Data PrivacyTrustArc Webinar - How to Build Consumer Trust Through Data Privacy
TrustArc Webinar - How to Build Consumer Trust Through Data Privacy
 
Manual 508 Accessibility Compliance Audit
Manual 508 Accessibility Compliance AuditManual 508 Accessibility Compliance Audit
Manual 508 Accessibility Compliance Audit
 
New from BookNet Canada for 2024: Loan Stars - Tech Forum 2024
New from BookNet Canada for 2024: Loan Stars - Tech Forum 2024New from BookNet Canada for 2024: Loan Stars - Tech Forum 2024
New from BookNet Canada for 2024: Loan Stars - Tech Forum 2024
 
Emixa Mendix Meetup 11 April 2024 about Mendix Native development
Emixa Mendix Meetup 11 April 2024 about Mendix Native developmentEmixa Mendix Meetup 11 April 2024 about Mendix Native development
Emixa Mendix Meetup 11 April 2024 about Mendix Native development
 

Scala meetup - Intro to spark

  • 1. Introduction to Apache Spark Brendan Dillon Javier Arrieta
  • 2. Spark Core Your Applications The Stack Spark SQL MLLib GraphX Spark Streaming Mesos YARN Standalone sqlContext.sql("SELECT name FROM people WHERE age >= 13 AND age <= 19").map(t => "Name: " + t(0)).collect().foreach(println)
  • 4. Resilient Distributed Datasets (RDD) • Immutable: never modified – just transformed to new RDDs • Distributed: split into multiple partitions and spread across multiple servers in a cluster • Resilient: can be re-computed if they get destroyed • Created by: – Loading external data – Distributing a collection of objects in the driver program
  • 5. RDD Implementation • Array of partitions • List of dependencies on parent RDDs • Function to compute a partition given its parents – Returns Iterator over a partition • Preferred locations: list of strings for each partition (Nil by default) • Partitioner (None by default)
  • 6. Persistence / Caching • By default RDDs (and all of their dependencies) are recomputed every time an action is called on them! • Need to explicitly tell Spark when to persist • Options: – Default: stored in heap as unserialized objects (pickled objects for Python) – Memory only: serialized or not – Memory and disk: spills to disk, option to serialize in memory – Disk only • Tachyon: off-heap distributed caching – Aims to make Spark more resilient – Avoid GC overheads
  • 7. Dependency Types: Narrow E.g. map, filter E.g. union E.g. join with co-partitioned input Each partition of parent is used by at most one partition of the child
  • 8. Dependency Types: Wide E.g. groupByKey E.g. join with inputs non co-partitioned Each partition of the parent is used by more than one partition of the child
  • 9. Transformations • Return a new RDD • Lazy evaluation • Single RDD transformations: map, flatMap, filter, distinct • Pair RDDs: keyBy, reduceByKey, groupByKey, combineByKey, mapValues, flatMapValues, sortByKey • Two RDD transformations: union, intersection, subtract, cartesian • Two pair RDDs: join, rightOuterJoin, leftOuterJoin, cogroup
  • 10. Actions • Force evaluation of the transformations and return a value to the driver program or write to external storage • Actions on RDDs: – reduce, fold, aggregate – foreach(func), collect – count, countByValue – top(num) – take(num), takeOrdered(num)(ordering) • Actions on pair RDDs: – countByKey – collectAsMap – lookup(key)
  • 12. map and flatMap • map takes a function that transforms each element of a collection: map(f: T => U) • RDD[T] => RDD[U] • flatMap takes a function that transforms a single element of a collection into a sequence of elements: flatMap(f: T => Seq[U]) • Flattens out the output into a single sequence • RDD[T] => RDD[U]
  • 13. filter, distinct • filter takes a (predicate) function that returns true if an element should be in the output collection: map(f: T => Bool) • distinct removes duplicates from the RDD • Both filter and distinct transform from RDD[T] => RDD[T]
  • 15. reduce, fold & aggregate • reduce takes a function that combines pairwise element of a collection: reduce(f: (T, T) => T) • fold is like reduce except it takes a zero value i.e. fold(zero: T) (f: (T, T) => T) • reduce and fold: RDD[T] => T • aggregate is the most general form • aggregate(zero: U)(seqOp: (U, T) => U, combOp: (U, U) => U) • aggregate: RDD[T] => U
  • 17. keyBy, reduceByKey • keyBy creates tuples of the elements in an RDD by applying a function: keyBy(f: T => K) • RDD[ T ] => RDD[ (K, T) ] • reduceByKey takes a function that takes a two values and returns a single value: reduceByKey(f: (V,V) => V) • RDD[ (K, V) ] => RDD[ (K, V) ]
  • 18. groupByKey • Takes a collection of key-value pairs and no parameters • Returns a sequence of values associated with each key • RDD[ ( K, V ) ] => RDD[ ( K, Iterable[V] ) ] • Results must fit in memory • Can be slow – use aggregateByKey or reduceByKey where possible • Ordering of values not guaranteed and can vary on every evaluation
  • 19. combineByKey • def combineByKey[C](createCombiner: V => C, mergeValue: (C, V) => C, mergeCombiners: (C, C) => C, partitioner: Partitioner, mapSideCombine: Boolean = true, serializer: Serializer = null) • RDD [ (K, V) ] => RDD[ (K, C) ] • createCombiner called per partition when a new key is found • mergeValue combines a new value to an existing accumulator • mergeCombiners with results from different partitions • Sometimes map-size combine not useful e.g. groupByKey • groupByKey, aggregateByKey and reduceByKey all implemented using combineByKey
  • 20. map vs mapValues • map takes a function that transforms each element of a collection: map(f: T => U) • RDD[T] => RDD[U] • When T is a tuple we may want to only act on the values – not the keys • mapValues takes a function that maps the values in the inputs to the values in the output: mapValues(f: V => W) • Where RDD[ (K, V) ] => RDD[ (K, W) ] • NB: use mapValues when you can: avoids reshuffle when data is partitioned by key
  • 22. Pseudo-set: union, intersection, subtract, cartesian • rdd.union(otherRdd): RRD containing elements from both • rdd.intersection(otherRdd): RDD containing only elements found in both • rdd.subtract(otherRdd): remove content of one from the other e.g. removing training data • rdd.cartesian(otherRdd): Cartesian product of two RDDs e.g. similarity of pairs: RDD[T] RDD[U] => RDD[ (T, U) ]
  • 23. Two Pair RDD Transformations
  • 24. join, rightOuterJoin, leftOuterJoin, cogroup • Join: RDD[ ( K, V) ] and RDD[ (K, W) ] => RDD[ ( K, (V,W) ) ] • Cogroup: RDD[ ( K, V) ] and RDD[ (K, W) ] => RDD[ ( K, ( Seq[V], Seq[W] ) ) ] • rightOuterJoin and leftRightJoin when keys must be present in left / right RDD
  • 26. mapPartitions, mapPartitionsWithIndex, and foreachPartition • Same as map and foreach except they operate on a per partition basis • Useful for when you have setup code (DB, RNG etc.) but don’t want to call it for each partition • You can set preservesPartitioning when you are not altering the keys used for partitioning to avoid unnecessary shuffling – As with mapValues in the last slide
  • 28. Data Frames & Catalyst Optimizer
  • 29. DataFrame creation and operations val sc: SparkContext // An existing SparkContext. val sqlContext = new org.apache.spark.sql.SQLContext(sc) // Create the DataFrame val df = sqlContext.jsonFile("examples/src/main/resources/people.json”) // Show the content of the DataFrame df.show() // Print the schema in a tree format df.printSchema() // Select only the "name" column df.select("name”) // Select everybody, but increment the age by 1 df.select("name", df("age") + 1) // Select people older than 21 df.filter(df("name") > 21) // Count people by age df.groupBy("age").count()
  • 32. Alternatives Apache Storm Trident Programming Model Micro-Batch One at a time Micro-batch Stream Primitive DStream Stream Tuple, Tuple Batch, Partition Distributed Stream Dataflow Stream Source ReceiverInputDStr eam Container Spouts, Trident Spouts Data Stream Computation Maps/windows/op erations on Dstream StreamTask, Window, join Filters, functions, aggregations, joins Maps/windows/op erations on Data Stream Resource mgmt YARN/Mesos YARN YARN/Mesos YARN Resilience Require WAL to DFS (HDFS/S3) Checkpointing (Kafka) Nimbus reassigns and failed batch replayed Lightweight Distributed Snapshots
  • 33. Scala collections programming model, map, flatMap, window, reduce (fold) share code between batch and streaming, both share the same programming model (although different semantics) microbatches allow aggregation on the batches, improved throughput with a latency cost Why Spark Streaming
  • 34. Spark Streaming Execution Driver Spark Context Worker Node Executor Task Task Worker Node Executor Task Task Worker Node Executor Task Task Streaming Producer
  • 36. Code val metaStream = stream.map { case (k, v) => (k, DocumentMetadata.fromMutable(recordDecoder.decode(v).asInstanceOf[GenericRecord])) } private val pdfFiles = metaStream.filter(_._2.contentType == "application/pdf") .map { case (k, meta) => (meta, fetchFileFromMessage(k, meta)) } val pdfDocs = pdfFiles.map { case (meta, file) => (meta, TextExtractor.parseFile(file)) } val texts = pdfDocs.map { case (meta, doc) => (meta, TextExtractor.extractText(doc)) }.cache() val wordStream = texts.map { case (meta, text) => (meta, text.split("""[ nrtu00a0]+""").toList.map(_.replaceAll("""[,;.]$""", "").trim.toLowerCase()).filter(_.length > 1)) } texts.foreachRDD( rdd => rdd.foreach { case (meta,text) => indexText(meta.id, text) } ) val wordCountStream = wordStream.flatMap(_._2).map(word => (word, 1)).reduceByKey(_ + _) val totalWordCountStream = wordStream.map(_._2.size) val totalWords = totalWordCountStream.reduce(_+_) val sortedWordCount = wordCountStream.transform(rdd => rdd.sortBy(_._2, ascending = false)) sortedWordCount.foreachRDD(rdd => println(rdd.toDebugString)) sortedWordCount.print(30) totalWords.print()
  • 37. Q & A