SlideShare a Scribd company logo
1 of 43
Download to read offline
Beyond Parallelize & Collect
(Effective testing of Spark Programs)
Now
mostly
“works”*
*See developer for details. Does not imply warranty. :p
Who am I?
● My name is Holden Karau
● Prefered pronouns are she/her
● I’m a Software Engineer
● currently IBM and previously Alpine, Databricks, Google, Foursquare &
Amazon
● co-author of Learning Spark & Fast Data processing with Spark
● @holdenkarau
● Slide share http://www.slideshare.net/hkarau
● Linkedin https://www.linkedin.com/in/holdenkarau
● Spark Videos http://bit.ly/holdenSparkVideos
What is going to be covered:
● What I think I might know about you
● A bit about why you should test your programs
● Using parallelize & collect for unit testing (quick skim)
● Comparing datasets too large to fit in memory
● Considerations for Streaming & SQL (DataFrames & Datasets)
● Cute & scary pictures
○ I promise at least one panda and one cat
● “Future Work”
○ Integration testing lives here for now (sorry)
○ Some of this future work might even get done!
Who I think you wonderful humans are?
● Nice* people
● Like silly pictures
● Familiar with Apache Spark
○ If not, buy one of my books or watch Paco’s awesome video
● Familiar with one of Scala, Java, or Python
○ If you know R well I’d love to chat though
● Want to make better software
○ (or models, or w/e)
So why should you test?
● Makes you a better person
● Save $s
○ May help you avoid losing your employer all of their money
■ Or “users” if we were in the bay
○ AWS is expensive
● Waiting for our jobs to fail is a pretty long dev cycle
● This is really just to guilt trip you & give you flashbacks to your QA internships
So why should you test - continued
Results from: Testing with Spark survey http://bit.ly/holdenTestingSpark
So why should you test - continued
Results from: Testing with Spark survey http://bit.ly/holdenTestingSpark
Why don’t we test?
● It’s hard
○ Faking data, setting up integration tests, urgh w/e
● Our tests can get too slow
● It takes a lot of time
○ and people always want everything done yesterday
○ or I just want to go home see my partner
○ etc.
Cat photo from http://galato901.deviantart.com/art/Cat-on-Work-Break-173043455
An artisanal Spark unit test
@transient private var _sc: SparkContext = _
override def beforeAll() {
_sc = new SparkContext("local[4]")
super.beforeAll()
}
override def afterAll() {
if (sc != null)
sc.stop()
System.clearProperty("spark.driver.port") // rebind issue
_sc = null
super.afterAll()
}
Photo by morinesque
And on to the actual test...
test("really simple transformation") {
val input = List("hi", "hi holden", "bye")
val expected = List(List("hi"), List("hi", "holden"), List("bye"))
assert(tokenize(sc.parallelize(input)).collect().toList === expected)
}
def tokenize(f: RDD[String]) = {
f.map(_.split(" ").toList)
}
Photo by morinesque
Wait, where were the batteries?
Photo by Jim Bauer
Let’s get batteries!
● Spark unit testing
○ spark-testing-base - https://github.com/holdenk/spark-testing-base
○ sscheck - https://github.com/juanrh/sscheck
● Integration testing
○ spark-integration-tests (Spark internals) - https://github.com/databricks/spark-integration-tests
● Performance
○ spark-perf (also for Spark internals) - https://github.com/databricks/spark-perf
● Spark job validation
○ spark-validator - https://github.com/holdenk/spark-validator
Photo by Mike Mozart
A simple unit test re-visited (Scala)
class SampleRDDTest extends FunSuite with SharedSparkContext {
test("really simple transformation") {
val input = List("hi", "hi holden", "bye")
val expected = List(List("hi"), List("hi", "holden"), List("bye"))
assert(SampleRDD.tokenize(sc.parallelize(input)).collect().toList === expected)
}
}
Ok but what about problems @ scale
● Maybe our program works fine on our local sized input
● If we are using Spark our actual workload is probably huge
● How do we test workloads too large for a single machine?
○ we can’t just use parallelize and collect
Qfamily
Distributed “set” operations to the rescue*
● Pretty close - already built into Spark
● Doesn’t do so well with floating points :(
○ damn floating points keep showing up everywhere :p
● Doesn’t really handle duplicates very well
○ {“coffee”, “coffee”, “panda”} != {“panda”, “coffee”} but with set operations...
Matti Mattila
Or use RDDComparisions:
def compareWithOrderSamePartitioner[T: ClassTag](expected: RDD
[T], result: RDD[T]): Option[(T, T)] = {
expected.zip(result).filter{case (x, y) => x != y}.take(1).
headOption
}
Matti Mattila
Or use RDDComparisions:
def compare[T: ClassTag](expected: RDD[T], result: RDD[T]): Option
[(T, Int, Int)] = {
val expectedKeyed = expected.map(x => (x, 1)).reduceByKey(_ +
_)
val resultKeyed = result.map(x => (x, 1)).reduceByKey(_ + _)
expectedKeyed.cogroup(resultKeyed).filter{case (_, (i1, i2))
=>
i1.isEmpty || i2.isEmpty || i1.head != i2.head}.take(1).
headOption.
map{case (v, (i1, i2)) => (v, i1.headOption.getOrElse(0),
i2.headOption.getOrElse(0))}
}
Matti Mattila
But where do we get the data for those tests?
● If you have production data you can sample you are lucky!
○ If possible you can try and save in the same format
● If our data is a bunch of Vectors or Doubles Spark’s got tools :)
● Coming up with good test data can take a long time
Lori Rielly
QuickCheck / ScalaCheck
● QuickCheck generates tests data under a set of constraints
● Scala version is ScalaCheck - supported by the two unit testing libraries for
Spark
● sscheck
○ Awesome people*, supports generating DStreams too!
● spark-testing-base
○ Also Awesome people*, generates more pathological (e.g. empty partitions etc.) RDDs
*I assume
PROtara hunt
With spark-testing-base
test("map should not change number of elements") {
forAll(RDDGenerator.genRDD[String](sc)){
rdd => rdd.map(_.length).count() == rdd.count()
}
}
Testing streaming….
Photo by Steve Jurvetson
// Setup our Stream:
class TestInputStream[T: ClassTag](@transient var sc:
SparkContext,
ssc_ : StreamingContext, input: Seq[Seq[T]], numPartitions: Int)
extends FriendlyInputDStream[T](ssc_) {
def start() {}
def stop() {}
def compute(validTime: Time): Option[RDD[T]] = {
logInfo("Computing RDD for time " + validTime)
val index = ((validTime - ourZeroTime) / slideDuration - 1).
toInt
val selectedInput = if (index < input.size) input(index) else
Seq[T]()
// lets us test cases where RDDs are not created
if (selectedInput == null) {
return None
}
val rdd = sc.makeRDD(selectedInput, numPartitions)
logInfo("Created RDD " + rdd.id + " with " + selectedInput)
Some(rdd)
}
}
Artisanal Stream Testing Code
trait StreamingSuiteBase extends FunSuite with BeforeAndAfterAll with Logging
with SharedSparkContext {
// Name of the framework for Spark context
def framework: String = this.getClass.getSimpleName
// Master for Spark context
def master: String = "local[4]"
// Batch duration
def batchDuration: Duration = Seconds(1)
// Directory where the checkpoint data will be saved
lazy val checkpointDir = {
val dir = Utils.createTempDir()
logDebug(s"checkpointDir: $dir")
dir.toString
}
// Default after function for any streaming test suite. Override this
// if you want to add your stuff to "after" (i.e., don't call after { } )
override def afterAll() {
System.clearProperty("spark.streaming.clock")
super.afterAll()
}
Phot
o by
Stev
e
Jurv
etso
n
and continued….
/**
* Create an input stream for the provided input sequence. This is done using
* TestInputStream as queueStream's are not checkpointable.
*/
def createTestInputStream[T: ClassTag](sc: SparkContext, ssc_ :
TestStreamingContext,
input: Seq[Seq[T]]): TestInputStream[T] = {
new TestInputStream(sc, ssc_, input, numInputPartitions)
}
// Default before function for any streaming test suite. Override this
// if you want to add your stuff to "before" (i.e., don't call before { } )
override def beforeAll() {
if (useManualClock) {
logInfo("Using manual clock")
conf.set("spark.streaming.clock", "org.apache.spark.streaming.util.
TestManualClock") // We can specify our own clock
} else {
logInfo("Using real clock")
conf.set("spark.streaming.clock", "org.apache.spark.streaming.util.SystemClock")
}
super.beforeAll()
}
/**
* Run a block of code with the given StreamingContext and automatically
* stop the context when the block completes or when an exception is thrown.
*/
def withOutputAndStreamingContext[R](outputStreamSSC: (TestOutputStream
[R], TestStreamingContext))
(block: (TestOutputStream[R], TestStreamingContext) => Unit): Unit = {
val outputStream = outputStreamSSC._1
val ssc = outputStreamSSC._2
try {
block(outputStream, ssc)
} finally {
try {
ssc.stop(stopSparkContext = false)
} catch {
case e: Exception =>
logError("Error stopping StreamingContext", e)
}
}
}
}
and now for the clock
/*
* Allows us access to a manual clock. Note that the manual clock changed between
1.1.1 and 1.3
*/
class TestManualClock(var time: Long) extends Clock {
def this() = this(0L)
def getTime(): Long = getTimeMillis() // Compat
def currentTime(): Long = getTimeMillis() // Compat
def getTimeMillis(): Long =
synchronized {
time
}
def setTime(timeToSet: Long): Unit =
synchronized {
time = timeToSet
notifyAll()
}
def advance(timeToAdd: Long): Unit =
synchronized {
time += timeToAdd
notifyAll()
}
def addToTime(timeToAdd: Long): Unit = advance(timeToAdd) // Compat
/**
* @param targetTime block until the clock time is set or advanced to at least this
time
* @return current time reported by the clock when waiting finishes
*/
def waitTillTime(targetTime: Long): Long =
synchronized {
while (time < targetTime) {
wait(100)
}
getTimeMillis()
}
}
Testing streaming the happy panda way
● Creating test data is hard
○ ssc.queueStream works - unless you need checkpoints (1.4.1+)
● Collecting the data locally is hard
○ foreachRDD & a var
● figuring out when your test is “done”
Let’s abstract all that away into testOperation
We can hide all of that:
test("really simple transformation") {
val input = List(List("hi"), List("hi holden"), List("bye"))
val expected = List(List("hi"), List("hi", "holden"), List("bye"))
testOperation[String, String](input, tokenize _, expected, useSet = true)
}
Photo by An eye
for my mind
What about DataFrames?
● We can do the same as we did for RDD’s (.rdd)
● Inside of Spark validation looks like:
def checkAnswer(df: DataFrame, expectedAnswer: Seq[Row])
● Sadly it’s not in a published package & local only
● instead we expose:
def equalDataFrames(expected: DataFrame, result: DataFrame) {
def approxEqualDataFrames(e: DataFrame, r: DataFrame, tol: Double) {
…. and Datasets
● We can do the same as we did for RDD’s (.rdd)
● Inside of Spark validation looks like:
def checkAnswer(df: Dataset[T], expectedAnswer: T*)
● Sadly it’s not in a published package & local only
● instead we expose:
def equalDatasets(expected: Dataset[U], result: Dataset[V]) {
def approxEqualDatasets(e: Dataset[U], r: Dataset[V], tol: Double) {
This is what it looks like:
test("dataframe should be equal to its self") {
val sqlCtx = sqlContext
import sqlCtx.implicits._// Yah I know this is ugly
val input = sc.parallelize(inputList).toDF
equalDataFrames(input, input)
}
*This may or may not be easier.
Which has “built-in” large support :)
Photo by allison
Let’s talk about local mode
● It’s way better than you would expect*
● It does its best to try and catch serialization errors
● It’s still not the same as running on a “real” cluster
● Especially since if we were just local mode, parallelize and collect might be
fine
Photo by: Bev Sykes
Options beyond local mode:
● Just point at your existing cluster (set master)
● Start one with your shell scripts & change the master
○ Really easy way to plug into existing integration testing
● spark-docker - hack in our own tests
● YarnMiniCluster
○ https://github.
com/apache/spark/blob/master/yarn/src/test/scala/org/apache/spark/deploy/yarn/BaseYarnClu
sterSuite.scala
○ In Spark Testing Base extend SharedMiniCluster
■ Not recommended until after SPARK-10812 (e.g. 1.5.2+ or 1.6+)
Photo by Richard Masoner
Validation
● Validation can be really useful for catching errors before deploying a model
○ Our tests can’t catch everything
● For now checking file sizes & execution time seem like the most common best
practice (from survey)
● Accumulators have some challenges (see SPARK-12469 for progress) but
are an interesting option
● spark-validator is still in early stages and not ready for production use but
interesting proof of concept
Photo by:
Paul Schadler
Related talks & blog posts
● Testing Spark Best Practices (Spark Summit 2014)
● Every Day I’m Shuffling (Strata 2015) & slides
● Spark and Spark Streaming Unit Testing
● Making Spark Unit Testing With Spark Testing Base
Learning Spark
Fast Data
Processing with
Spark
(Out of Date)
Fast Data
Processing with
Spark
(2nd edition)
Advanced
Analytics with
Spark
Learning Spark
Fast Data
Processing with
Spark
(Out of Date)
Fast Data
Processing with
Spark
(2nd edition)
Advanced
Analytics with
Spark
Coming soon:
Spark in Action
Coming soon:
High Performance Spark
And the next book…..
Still being written - signup to be notified when it is available:
● http://www.highperformancespark.com
● https://twitter.com/highperfspark
Related packages
● spark-testing-base: https://github.com/holdenk/spark-testing-base
● sscheck: https://github.com/juanrh/sscheck
● spark-validator: https://github.com/holdenk/spark-validator *ALPHA*
● spark-perf - https://github.com/databricks/spark-perf
● spark-integration-tests - https://github.com/databricks/spark-integration-tests
● scalacheck - https://www.scalacheck.org/
And including spark-testing-base:
sbt:
"com.holdenkarau" %% "spark-testing-base" % "1.5.2_0.3.1"
maven:
<dependency>
<groupId>com.holdenkarau</groupId>
<artifactId>spark-testing-base</artifactId>
<version>${spark.version}_0.3.1</version>
<scope>test</scope>
</dependency>
“Future Work”
● Better ScalaCheck integration (ala sscheck)
● Testing details in my next Spark book
● Whatever* you all want
○ Testing with Spark survey: http://bit.ly/holdenTestingSpark
Semi-likely:
● integration testing (for now see @cfriegly’s Spark + Docker setup):
○ https://github.com/fluxcapacitor/pipeline
Pretty unlikely:
● Integrating into Apache Spark ( SPARK-12433 )
*That I feel like doing, or you feel like making a pull request for.
Photo by
bullet101
Cat wave photo by Quinn Dombrowski
k thnx bye!
If you want to fill out survey: http:
//bit.ly/holdenTestingSpark
Will use update results in
Strata Presentation & tweet
eventually at @holdenkarau

More Related Content

What's hot

파이썬 TDD 101
파이썬 TDD 101파이썬 TDD 101
파이썬 TDD 101정주 김
 
SparkSQL: A Compiler from Queries to RDDs
SparkSQL: A Compiler from Queries to RDDsSparkSQL: A Compiler from Queries to RDDs
SparkSQL: A Compiler from Queries to RDDsDatabricks
 
#살아있다 #자프링외길12년차 #코프링2개월생존기
#살아있다 #자프링외길12년차 #코프링2개월생존기#살아있다 #자프링외길12년차 #코프링2개월생존기
#살아있다 #자프링외길12년차 #코프링2개월생존기Arawn Park
 
Pyspark Tutorial | Introduction to Apache Spark with Python | PySpark Trainin...
Pyspark Tutorial | Introduction to Apache Spark with Python | PySpark Trainin...Pyspark Tutorial | Introduction to Apache Spark with Python | PySpark Trainin...
Pyspark Tutorial | Introduction to Apache Spark with Python | PySpark Trainin...Edureka!
 
Data Source API in Spark
Data Source API in SparkData Source API in Spark
Data Source API in SparkDatabricks
 
Beyond unit tests: Testing for Spark/Hadoop Workflows with Shankar Manian Ana...
Beyond unit tests: Testing for Spark/Hadoop Workflows with Shankar Manian Ana...Beyond unit tests: Testing for Spark/Hadoop Workflows with Shankar Manian Ana...
Beyond unit tests: Testing for Spark/Hadoop Workflows with Shankar Manian Ana...Spark Summit
 
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 Spark Summit
 
Streaming SQL with Apache Calcite
Streaming SQL with Apache CalciteStreaming SQL with Apache Calcite
Streaming SQL with Apache CalciteJulian Hyde
 
Introducao oracle-sqlplsql-vol2-pt
Introducao oracle-sqlplsql-vol2-ptIntroducao oracle-sqlplsql-vol2-pt
Introducao oracle-sqlplsql-vol2-ptValdinho Pereira
 
Apache Arrow Flight Overview
Apache Arrow Flight OverviewApache Arrow Flight Overview
Apache Arrow Flight OverviewJacques Nadeau
 
AngularJS: an introduction
AngularJS: an introductionAngularJS: an introduction
AngularJS: an introductionLuigi De Russis
 
Validating spark ml jobs stopping failures before production on Apache Spark ...
Validating spark ml jobs stopping failures before production on Apache Spark ...Validating spark ml jobs stopping failures before production on Apache Spark ...
Validating spark ml jobs stopping failures before production on Apache Spark ...Holden Karau
 
Nodejs functions & modules
Nodejs functions & modulesNodejs functions & modules
Nodejs functions & modulesmonikadeshmane
 
Android Jetpack Compose - Turkey 2021
Android Jetpack Compose - Turkey 2021Android Jetpack Compose - Turkey 2021
Android Jetpack Compose - Turkey 2021Nelson Glauber Leal
 
Data pipelines observability: OpenLineage & Marquez
Data pipelines observability:  OpenLineage & MarquezData pipelines observability:  OpenLineage & Marquez
Data pipelines observability: OpenLineage & MarquezJulien Le Dem
 
From Query Plan to Query Performance: Supercharging your Apache Spark Queries...
From Query Plan to Query Performance: Supercharging your Apache Spark Queries...From Query Plan to Query Performance: Supercharging your Apache Spark Queries...
From Query Plan to Query Performance: Supercharging your Apache Spark Queries...Databricks
 
Smarter Together - Bringing Relational Algebra, Powered by Apache Calcite, in...
Smarter Together - Bringing Relational Algebra, Powered by Apache Calcite, in...Smarter Together - Bringing Relational Algebra, Powered by Apache Calcite, in...
Smarter Together - Bringing Relational Algebra, Powered by Apache Calcite, in...Julian Hyde
 
Deep Dive: Memory Management in Apache Spark
Deep Dive: Memory Management in Apache SparkDeep Dive: Memory Management in Apache Spark
Deep Dive: Memory Management in Apache SparkDatabricks
 
Integration Testing in AEM
Integration Testing in AEMIntegration Testing in AEM
Integration Testing in AEMconnectwebex
 

What's hot (20)

파이썬 TDD 101
파이썬 TDD 101파이썬 TDD 101
파이썬 TDD 101
 
SparkSQL: A Compiler from Queries to RDDs
SparkSQL: A Compiler from Queries to RDDsSparkSQL: A Compiler from Queries to RDDs
SparkSQL: A Compiler from Queries to RDDs
 
#살아있다 #자프링외길12년차 #코프링2개월생존기
#살아있다 #자프링외길12년차 #코프링2개월생존기#살아있다 #자프링외길12년차 #코프링2개월생존기
#살아있다 #자프링외길12년차 #코프링2개월생존기
 
Pyspark Tutorial | Introduction to Apache Spark with Python | PySpark Trainin...
Pyspark Tutorial | Introduction to Apache Spark with Python | PySpark Trainin...Pyspark Tutorial | Introduction to Apache Spark with Python | PySpark Trainin...
Pyspark Tutorial | Introduction to Apache Spark with Python | PySpark Trainin...
 
Data Source API in Spark
Data Source API in SparkData Source API in Spark
Data Source API in Spark
 
Beyond unit tests: Testing for Spark/Hadoop Workflows with Shankar Manian Ana...
Beyond unit tests: Testing for Spark/Hadoop Workflows with Shankar Manian Ana...Beyond unit tests: Testing for Spark/Hadoop Workflows with Shankar Manian Ana...
Beyond unit tests: Testing for Spark/Hadoop Workflows with Shankar Manian Ana...
 
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
 
Streaming SQL with Apache Calcite
Streaming SQL with Apache CalciteStreaming SQL with Apache Calcite
Streaming SQL with Apache Calcite
 
Introducao oracle-sqlplsql-vol2-pt
Introducao oracle-sqlplsql-vol2-ptIntroducao oracle-sqlplsql-vol2-pt
Introducao oracle-sqlplsql-vol2-pt
 
Apache Arrow Flight Overview
Apache Arrow Flight OverviewApache Arrow Flight Overview
Apache Arrow Flight Overview
 
AngularJS: an introduction
AngularJS: an introductionAngularJS: an introduction
AngularJS: an introduction
 
Validating spark ml jobs stopping failures before production on Apache Spark ...
Validating spark ml jobs stopping failures before production on Apache Spark ...Validating spark ml jobs stopping failures before production on Apache Spark ...
Validating spark ml jobs stopping failures before production on Apache Spark ...
 
Nodejs functions & modules
Nodejs functions & modulesNodejs functions & modules
Nodejs functions & modules
 
Android Jetpack Compose - Turkey 2021
Android Jetpack Compose - Turkey 2021Android Jetpack Compose - Turkey 2021
Android Jetpack Compose - Turkey 2021
 
Data pipelines observability: OpenLineage & Marquez
Data pipelines observability:  OpenLineage & MarquezData pipelines observability:  OpenLineage & Marquez
Data pipelines observability: OpenLineage & Marquez
 
From Query Plan to Query Performance: Supercharging your Apache Spark Queries...
From Query Plan to Query Performance: Supercharging your Apache Spark Queries...From Query Plan to Query Performance: Supercharging your Apache Spark Queries...
From Query Plan to Query Performance: Supercharging your Apache Spark Queries...
 
Django Celery
Django Celery Django Celery
Django Celery
 
Smarter Together - Bringing Relational Algebra, Powered by Apache Calcite, in...
Smarter Together - Bringing Relational Algebra, Powered by Apache Calcite, in...Smarter Together - Bringing Relational Algebra, Powered by Apache Calcite, in...
Smarter Together - Bringing Relational Algebra, Powered by Apache Calcite, in...
 
Deep Dive: Memory Management in Apache Spark
Deep Dive: Memory Management in Apache SparkDeep Dive: Memory Management in Apache Spark
Deep Dive: Memory Management in Apache Spark
 
Integration Testing in AEM
Integration Testing in AEMIntegration Testing in AEM
Integration Testing in AEM
 

Viewers also liked

Operational Tips for Deploying Spark by Miklos Christine
Operational Tips for Deploying Spark by Miklos ChristineOperational Tips for Deploying Spark by Miklos Christine
Operational Tips for Deploying Spark by Miklos ChristineSpark Summit
 
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...Databricks
 
Online Security Analytics on Large Scale Video Surveillance System by Yu Cao ...
Online Security Analytics on Large Scale Video Surveillance System by Yu Cao ...Online Security Analytics on Large Scale Video Surveillance System by Yu Cao ...
Online Security Analytics on Large Scale Video Surveillance System by Yu Cao ...Spark Summit
 
Clickstream Analysis with Spark—Understanding Visitors in Realtime by Josef A...
Clickstream Analysis with Spark—Understanding Visitors in Realtime by Josef A...Clickstream Analysis with Spark—Understanding Visitors in Realtime by Josef A...
Clickstream Analysis with Spark—Understanding Visitors in Realtime by Josef A...Spark Summit
 
Enhancements on Spark SQL optimizer by Min Qiu
Enhancements on Spark SQL optimizer by Min QiuEnhancements on Spark SQL optimizer by Min Qiu
Enhancements on Spark SQL optimizer by Min QiuSpark Summit
 
Building Realtime Data Pipelines with Kafka Connect and Spark Streaming
Building Realtime Data Pipelines with Kafka Connect and Spark StreamingBuilding Realtime Data Pipelines with Kafka Connect and Spark Streaming
Building Realtime Data Pipelines with Kafka Connect and Spark StreamingJen Aman
 
Spark Summit San Francisco 2016 - Matei Zaharia Keynote: Apache Spark 2.0
Spark Summit San Francisco 2016 - Matei Zaharia Keynote: Apache Spark 2.0Spark Summit San Francisco 2016 - Matei Zaharia Keynote: Apache Spark 2.0
Spark Summit San Francisco 2016 - Matei Zaharia Keynote: Apache Spark 2.0Databricks
 
Structuring Apache Spark 2.0: SQL, DataFrames, Datasets And Streaming - by Mi...
Structuring Apache Spark 2.0: SQL, DataFrames, Datasets And Streaming - by Mi...Structuring Apache Spark 2.0: SQL, DataFrames, Datasets And Streaming - by Mi...
Structuring Apache Spark 2.0: SQL, DataFrames, Datasets And Streaming - by Mi...Databricks
 
Apache Spark 2.0: A Deep Dive Into Structured Streaming - by Tathagata Das
Apache Spark 2.0: A Deep Dive Into Structured Streaming - by Tathagata Das Apache Spark 2.0: A Deep Dive Into Structured Streaming - by Tathagata Das
Apache Spark 2.0: A Deep Dive Into Structured Streaming - by Tathagata Das Databricks
 
Profiling & Testing with Spark
Profiling & Testing with SparkProfiling & Testing with Spark
Profiling & Testing with SparkRoger Rafanell Mas
 
Top 5 Mistakes When Writing Spark Applications by Mark Grover and Ted Malaska
Top 5 Mistakes When Writing Spark Applications by Mark Grover and Ted MalaskaTop 5 Mistakes When Writing Spark Applications by Mark Grover and Ted Malaska
Top 5 Mistakes When Writing Spark Applications by Mark Grover and Ted MalaskaSpark Summit
 
Tuning tips for Apache Spark Jobs
Tuning tips for Apache Spark JobsTuning tips for Apache Spark Jobs
Tuning tips for Apache Spark JobsSamir Bessalah
 
Sparklife - Life In The Trenches With Spark
Sparklife - Life In The Trenches With SparkSparklife - Life In The Trenches With Spark
Sparklife - Life In The Trenches With SparkIan Pointer
 
Sparkler Presentation for Spark Summit East 2017
Sparkler Presentation for Spark Summit East 2017Sparkler Presentation for Spark Summit East 2017
Sparkler Presentation for Spark Summit East 2017Karanjeet Singh
 
Highlights and Challenges from Running Spark on Mesos in Production by Morri ...
Highlights and Challenges from Running Spark on Mesos in Production by Morri ...Highlights and Challenges from Running Spark on Mesos in Production by Morri ...
Highlights and Challenges from Running Spark on Mesos in Production by Morri ...Spark Summit
 
Production Readiness Testing At Salesforce Using Spark MLlib
Production Readiness Testing At Salesforce Using Spark MLlibProduction Readiness Testing At Salesforce Using Spark MLlib
Production Readiness Testing At Salesforce Using Spark MLlibSpark Summit
 
Tagging and Processing Data in Real Time-(Hari Shreedharan and Siddhartha Jai...
Tagging and Processing Data in Real Time-(Hari Shreedharan and Siddhartha Jai...Tagging and Processing Data in Real Time-(Hari Shreedharan and Siddhartha Jai...
Tagging and Processing Data in Real Time-(Hari Shreedharan and Siddhartha Jai...Spark Summit
 
Spark Summit EU 2015: SparkUI visualization: a lens into your application
Spark Summit EU 2015: SparkUI visualization: a lens into your applicationSpark Summit EU 2015: SparkUI visualization: a lens into your application
Spark Summit EU 2015: SparkUI visualization: a lens into your applicationDatabricks
 
Spark with Cassandra by Christopher Batey
Spark with Cassandra by Christopher BateySpark with Cassandra by Christopher Batey
Spark with Cassandra by Christopher BateySpark Summit
 
Apache Spark Overview
Apache Spark OverviewApache Spark Overview
Apache Spark OverviewairisData
 

Viewers also liked (20)

Operational Tips for Deploying Spark by Miklos Christine
Operational Tips for Deploying Spark by Miklos ChristineOperational Tips for Deploying Spark by Miklos Christine
Operational Tips for Deploying Spark by Miklos Christine
 
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...
 
Online Security Analytics on Large Scale Video Surveillance System by Yu Cao ...
Online Security Analytics on Large Scale Video Surveillance System by Yu Cao ...Online Security Analytics on Large Scale Video Surveillance System by Yu Cao ...
Online Security Analytics on Large Scale Video Surveillance System by Yu Cao ...
 
Clickstream Analysis with Spark—Understanding Visitors in Realtime by Josef A...
Clickstream Analysis with Spark—Understanding Visitors in Realtime by Josef A...Clickstream Analysis with Spark—Understanding Visitors in Realtime by Josef A...
Clickstream Analysis with Spark—Understanding Visitors in Realtime by Josef A...
 
Enhancements on Spark SQL optimizer by Min Qiu
Enhancements on Spark SQL optimizer by Min QiuEnhancements on Spark SQL optimizer by Min Qiu
Enhancements on Spark SQL optimizer by Min Qiu
 
Building Realtime Data Pipelines with Kafka Connect and Spark Streaming
Building Realtime Data Pipelines with Kafka Connect and Spark StreamingBuilding Realtime Data Pipelines with Kafka Connect and Spark Streaming
Building Realtime Data Pipelines with Kafka Connect and Spark Streaming
 
Spark Summit San Francisco 2016 - Matei Zaharia Keynote: Apache Spark 2.0
Spark Summit San Francisco 2016 - Matei Zaharia Keynote: Apache Spark 2.0Spark Summit San Francisco 2016 - Matei Zaharia Keynote: Apache Spark 2.0
Spark Summit San Francisco 2016 - Matei Zaharia Keynote: Apache Spark 2.0
 
Structuring Apache Spark 2.0: SQL, DataFrames, Datasets And Streaming - by Mi...
Structuring Apache Spark 2.0: SQL, DataFrames, Datasets And Streaming - by Mi...Structuring Apache Spark 2.0: SQL, DataFrames, Datasets And Streaming - by Mi...
Structuring Apache Spark 2.0: SQL, DataFrames, Datasets And Streaming - by Mi...
 
Apache Spark 2.0: A Deep Dive Into Structured Streaming - by Tathagata Das
Apache Spark 2.0: A Deep Dive Into Structured Streaming - by Tathagata Das Apache Spark 2.0: A Deep Dive Into Structured Streaming - by Tathagata Das
Apache Spark 2.0: A Deep Dive Into Structured Streaming - by Tathagata Das
 
Profiling & Testing with Spark
Profiling & Testing with SparkProfiling & Testing with Spark
Profiling & Testing with Spark
 
Top 5 Mistakes When Writing Spark Applications by Mark Grover and Ted Malaska
Top 5 Mistakes When Writing Spark Applications by Mark Grover and Ted MalaskaTop 5 Mistakes When Writing Spark Applications by Mark Grover and Ted Malaska
Top 5 Mistakes When Writing Spark Applications by Mark Grover and Ted Malaska
 
Tuning tips for Apache Spark Jobs
Tuning tips for Apache Spark JobsTuning tips for Apache Spark Jobs
Tuning tips for Apache Spark Jobs
 
Sparklife - Life In The Trenches With Spark
Sparklife - Life In The Trenches With SparkSparklife - Life In The Trenches With Spark
Sparklife - Life In The Trenches With Spark
 
Sparkler Presentation for Spark Summit East 2017
Sparkler Presentation for Spark Summit East 2017Sparkler Presentation for Spark Summit East 2017
Sparkler Presentation for Spark Summit East 2017
 
Highlights and Challenges from Running Spark on Mesos in Production by Morri ...
Highlights and Challenges from Running Spark on Mesos in Production by Morri ...Highlights and Challenges from Running Spark on Mesos in Production by Morri ...
Highlights and Challenges from Running Spark on Mesos in Production by Morri ...
 
Production Readiness Testing At Salesforce Using Spark MLlib
Production Readiness Testing At Salesforce Using Spark MLlibProduction Readiness Testing At Salesforce Using Spark MLlib
Production Readiness Testing At Salesforce Using Spark MLlib
 
Tagging and Processing Data in Real Time-(Hari Shreedharan and Siddhartha Jai...
Tagging and Processing Data in Real Time-(Hari Shreedharan and Siddhartha Jai...Tagging and Processing Data in Real Time-(Hari Shreedharan and Siddhartha Jai...
Tagging and Processing Data in Real Time-(Hari Shreedharan and Siddhartha Jai...
 
Spark Summit EU 2015: SparkUI visualization: a lens into your application
Spark Summit EU 2015: SparkUI visualization: a lens into your applicationSpark Summit EU 2015: SparkUI visualization: a lens into your application
Spark Summit EU 2015: SparkUI visualization: a lens into your application
 
Spark with Cassandra by Christopher Batey
Spark with Cassandra by Christopher BateySpark with Cassandra by Christopher Batey
Spark with Cassandra by Christopher Batey
 
Apache Spark Overview
Apache Spark OverviewApache Spark Overview
Apache Spark Overview
 

Similar to Beyond Parallelize and Collect by Holden Karau

Testing and validating spark programs - Strata SJ 2016
Testing and validating spark programs - Strata SJ 2016Testing and validating spark programs - Strata SJ 2016
Testing and validating spark programs - Strata SJ 2016Holden Karau
 
Beyond parallelize and collect - Spark Summit East 2016
Beyond parallelize and collect - Spark Summit East 2016Beyond parallelize and collect - Spark Summit East 2016
Beyond parallelize and collect - Spark Summit East 2016Holden Karau
 
Testing and validating distributed systems with Apache Spark and Apache Beam ...
Testing and validating distributed systems with Apache Spark and Apache Beam ...Testing and validating distributed systems with Apache Spark and Apache Beam ...
Testing and validating distributed systems with Apache Spark and Apache Beam ...Holden Karau
 
Effective testing for spark programs Strata NY 2015
Effective testing for spark programs   Strata NY 2015Effective testing for spark programs   Strata NY 2015
Effective testing for spark programs Strata NY 2015Holden Karau
 
Effective testing for spark programs scala bay preview (pre-strata ny 2015)
Effective testing for spark programs scala bay preview (pre-strata ny 2015)Effective testing for spark programs scala bay preview (pre-strata ny 2015)
Effective testing for spark programs scala bay preview (pre-strata ny 2015)Holden Karau
 
Ml pipelines with Apache spark and Apache beam - Ottawa Reactive meetup Augus...
Ml pipelines with Apache spark and Apache beam - Ottawa Reactive meetup Augus...Ml pipelines with Apache spark and Apache beam - Ottawa Reactive meetup Augus...
Ml pipelines with Apache spark and Apache beam - Ottawa Reactive meetup Augus...Holden Karau
 
Apache spark as a gateway drug to FP concepts taught and broken - Curry On 2018
Apache spark as a gateway drug to FP concepts taught and broken - Curry On 2018Apache spark as a gateway drug to FP concepts taught and broken - Curry On 2018
Apache spark as a gateway drug to FP concepts taught and broken - Curry On 2018Holden Karau
 
Validating Big Data Jobs—Stopping Failures Before Production on Apache Spark...
 Validating Big Data Jobs—Stopping Failures Before Production on Apache Spark... Validating Big Data Jobs—Stopping Failures Before Production on Apache Spark...
Validating Big Data Jobs—Stopping Failures Before Production on Apache Spark...Databricks
 
Beyond Shuffling, Tips and Tricks for Scaling Apache Spark updated for Spark ...
Beyond Shuffling, Tips and Tricks for Scaling Apache Spark updated for Spark ...Beyond Shuffling, Tips and Tricks for Scaling Apache Spark updated for Spark ...
Beyond Shuffling, Tips and Tricks for Scaling Apache Spark updated for Spark ...Holden Karau
 
Apache Spark Structured Streaming for Machine Learning - StrataConf 2016
Apache Spark Structured Streaming for Machine Learning - StrataConf 2016Apache Spark Structured Streaming for Machine Learning - StrataConf 2016
Apache Spark Structured Streaming for Machine Learning - StrataConf 2016Holden Karau
 
Beyond Wordcount with spark datasets (and scalaing) - Nide PDX Jan 2018
Beyond Wordcount  with spark datasets (and scalaing) - Nide PDX Jan 2018Beyond Wordcount  with spark datasets (and scalaing) - Nide PDX Jan 2018
Beyond Wordcount with spark datasets (and scalaing) - Nide PDX Jan 2018Holden Karau
 
Validating big data jobs - Spark AI Summit EU
Validating big data jobs  - Spark AI Summit EUValidating big data jobs  - Spark AI Summit EU
Validating big data jobs - Spark AI Summit EUHolden Karau
 
Introducing Apache Spark's Data Frames and Dataset APIs workshop series
Introducing Apache Spark's Data Frames and Dataset APIs workshop seriesIntroducing Apache Spark's Data Frames and Dataset APIs workshop series
Introducing Apache Spark's Data Frames and Dataset APIs workshop seriesHolden Karau
 
Introduction to Spark Datasets - Functional and relational together at last
Introduction to Spark Datasets - Functional and relational together at lastIntroduction to Spark Datasets - Functional and relational together at last
Introduction to Spark Datasets - Functional and relational together at lastHolden Karau
 
Validating big data pipelines - Scala eXchange 2018
Validating big data pipelines -  Scala eXchange 2018Validating big data pipelines -  Scala eXchange 2018
Validating big data pipelines - Scala eXchange 2018Holden Karau
 
Debugging PySpark - Spark Summit East 2017
Debugging PySpark - Spark Summit East 2017Debugging PySpark - Spark Summit East 2017
Debugging PySpark - Spark Summit East 2017Holden Karau
 
Debugging PySpark: Spark Summit East talk by Holden Karau
Debugging PySpark: Spark Summit East talk by Holden KarauDebugging PySpark: Spark Summit East talk by Holden Karau
Debugging PySpark: Spark Summit East talk by Holden KarauSpark Summit
 
Introduction to and Extending Spark ML
Introduction to and Extending Spark MLIntroduction to and Extending Spark ML
Introduction to and Extending Spark MLHolden Karau
 
Big data beyond the JVM - DDTX 2018
Big data beyond the JVM -  DDTX 2018Big data beyond the JVM -  DDTX 2018
Big data beyond the JVM - DDTX 2018Holden Karau
 
An introduction into Spark ML plus how to go beyond when you get stuck
An introduction into Spark ML plus how to go beyond when you get stuckAn introduction into Spark ML plus how to go beyond when you get stuck
An introduction into Spark ML plus how to go beyond when you get stuckData Con LA
 

Similar to Beyond Parallelize and Collect by Holden Karau (20)

Testing and validating spark programs - Strata SJ 2016
Testing and validating spark programs - Strata SJ 2016Testing and validating spark programs - Strata SJ 2016
Testing and validating spark programs - Strata SJ 2016
 
Beyond parallelize and collect - Spark Summit East 2016
Beyond parallelize and collect - Spark Summit East 2016Beyond parallelize and collect - Spark Summit East 2016
Beyond parallelize and collect - Spark Summit East 2016
 
Testing and validating distributed systems with Apache Spark and Apache Beam ...
Testing and validating distributed systems with Apache Spark and Apache Beam ...Testing and validating distributed systems with Apache Spark and Apache Beam ...
Testing and validating distributed systems with Apache Spark and Apache Beam ...
 
Effective testing for spark programs Strata NY 2015
Effective testing for spark programs   Strata NY 2015Effective testing for spark programs   Strata NY 2015
Effective testing for spark programs Strata NY 2015
 
Effective testing for spark programs scala bay preview (pre-strata ny 2015)
Effective testing for spark programs scala bay preview (pre-strata ny 2015)Effective testing for spark programs scala bay preview (pre-strata ny 2015)
Effective testing for spark programs scala bay preview (pre-strata ny 2015)
 
Ml pipelines with Apache spark and Apache beam - Ottawa Reactive meetup Augus...
Ml pipelines with Apache spark and Apache beam - Ottawa Reactive meetup Augus...Ml pipelines with Apache spark and Apache beam - Ottawa Reactive meetup Augus...
Ml pipelines with Apache spark and Apache beam - Ottawa Reactive meetup Augus...
 
Apache spark as a gateway drug to FP concepts taught and broken - Curry On 2018
Apache spark as a gateway drug to FP concepts taught and broken - Curry On 2018Apache spark as a gateway drug to FP concepts taught and broken - Curry On 2018
Apache spark as a gateway drug to FP concepts taught and broken - Curry On 2018
 
Validating Big Data Jobs—Stopping Failures Before Production on Apache Spark...
 Validating Big Data Jobs—Stopping Failures Before Production on Apache Spark... Validating Big Data Jobs—Stopping Failures Before Production on Apache Spark...
Validating Big Data Jobs—Stopping Failures Before Production on Apache Spark...
 
Beyond Shuffling, Tips and Tricks for Scaling Apache Spark updated for Spark ...
Beyond Shuffling, Tips and Tricks for Scaling Apache Spark updated for Spark ...Beyond Shuffling, Tips and Tricks for Scaling Apache Spark updated for Spark ...
Beyond Shuffling, Tips and Tricks for Scaling Apache Spark updated for Spark ...
 
Apache Spark Structured Streaming for Machine Learning - StrataConf 2016
Apache Spark Structured Streaming for Machine Learning - StrataConf 2016Apache Spark Structured Streaming for Machine Learning - StrataConf 2016
Apache Spark Structured Streaming for Machine Learning - StrataConf 2016
 
Beyond Wordcount with spark datasets (and scalaing) - Nide PDX Jan 2018
Beyond Wordcount  with spark datasets (and scalaing) - Nide PDX Jan 2018Beyond Wordcount  with spark datasets (and scalaing) - Nide PDX Jan 2018
Beyond Wordcount with spark datasets (and scalaing) - Nide PDX Jan 2018
 
Validating big data jobs - Spark AI Summit EU
Validating big data jobs  - Spark AI Summit EUValidating big data jobs  - Spark AI Summit EU
Validating big data jobs - Spark AI Summit EU
 
Introducing Apache Spark's Data Frames and Dataset APIs workshop series
Introducing Apache Spark's Data Frames and Dataset APIs workshop seriesIntroducing Apache Spark's Data Frames and Dataset APIs workshop series
Introducing Apache Spark's Data Frames and Dataset APIs workshop series
 
Introduction to Spark Datasets - Functional and relational together at last
Introduction to Spark Datasets - Functional and relational together at lastIntroduction to Spark Datasets - Functional and relational together at last
Introduction to Spark Datasets - Functional and relational together at last
 
Validating big data pipelines - Scala eXchange 2018
Validating big data pipelines -  Scala eXchange 2018Validating big data pipelines -  Scala eXchange 2018
Validating big data pipelines - Scala eXchange 2018
 
Debugging PySpark - Spark Summit East 2017
Debugging PySpark - Spark Summit East 2017Debugging PySpark - Spark Summit East 2017
Debugging PySpark - Spark Summit East 2017
 
Debugging PySpark: Spark Summit East talk by Holden Karau
Debugging PySpark: Spark Summit East talk by Holden KarauDebugging PySpark: Spark Summit East talk by Holden Karau
Debugging PySpark: Spark Summit East talk by Holden Karau
 
Introduction to and Extending Spark ML
Introduction to and Extending Spark MLIntroduction to and Extending Spark ML
Introduction to and Extending Spark ML
 
Big data beyond the JVM - DDTX 2018
Big data beyond the JVM -  DDTX 2018Big data beyond the JVM -  DDTX 2018
Big data beyond the JVM - DDTX 2018
 
An introduction into Spark ML plus how to go beyond when you get stuck
An introduction into Spark ML plus how to go beyond when you get stuckAn introduction into Spark ML plus how to go beyond when you get stuck
An introduction into Spark ML plus how to go beyond when you get stuck
 

More from Spark Summit

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...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 WuSpark 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 RaghavendraSpark Summit
 
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...Spark Summit
 
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 ...Spark Summit
 
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 DowlingSpark Summit
 
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 DowlingSpark 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
 
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 WozniakSpark Summit
 
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 KimSpark 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 RaghavendraSpark Summit
 
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—...Spark Summit
 
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...Spark Summit
 
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...Spark Summit
 
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 SimeonovSpark Summit
 
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...Spark Summit
 
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 VolkSpark 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
 
Indicium: Interactive Querying at Scale Using Apache Spark, Zeppelin, and Spa...
Indicium: Interactive Querying at Scale Using Apache Spark, Zeppelin, and Spa...Indicium: Interactive Querying at Scale Using Apache Spark, Zeppelin, and Spa...
Indicium: Interactive Querying at Scale Using Apache Spark, Zeppelin, and Spa...Spark Summit
 

More from Spark Summit (20)

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...
 
Indicium: Interactive Querying at Scale Using Apache Spark, Zeppelin, and Spa...
Indicium: Interactive Querying at Scale Using Apache Spark, Zeppelin, and Spa...Indicium: Interactive Querying at Scale Using Apache Spark, Zeppelin, and Spa...
Indicium: Interactive Querying at Scale Using Apache Spark, Zeppelin, and Spa...
 

Recently uploaded

How to Build an Experimentation Culture for Data-Driven Product Development
How to Build an Experimentation Culture for Data-Driven Product DevelopmentHow to Build an Experimentation Culture for Data-Driven Product Development
How to Build an Experimentation Culture for Data-Driven Product DevelopmentAggregage
 
Empowering Decisions A Guide to Embedded Analytics
Empowering Decisions A Guide to Embedded AnalyticsEmpowering Decisions A Guide to Embedded Analytics
Empowering Decisions A Guide to Embedded AnalyticsGain Insights
 
Data Analytics Fundamentals: data analytics types.potx
Data Analytics Fundamentals: data analytics types.potxData Analytics Fundamentals: data analytics types.potx
Data Analytics Fundamentals: data analytics types.potxEmmanuel Dauda
 
Air Con Energy Rating Info411 Presentation.pdf
Air Con Energy Rating Info411 Presentation.pdfAir Con Energy Rating Info411 Presentation.pdf
Air Con Energy Rating Info411 Presentation.pdfJasonBoboKyaw
 
Prediction Of Cryptocurrency Prices Using Lstm, Svm And Polynomial Regression...
Prediction Of Cryptocurrency Prices Using Lstm, Svm And Polynomial Regression...Prediction Of Cryptocurrency Prices Using Lstm, Svm And Polynomial Regression...
Prediction Of Cryptocurrency Prices Using Lstm, Svm And Polynomial Regression...ferisulianta.com
 
Bengaluru Tableau UG event- 2nd March 2024 Q1
Bengaluru Tableau UG event- 2nd March 2024 Q1Bengaluru Tableau UG event- 2nd March 2024 Q1
Bengaluru Tableau UG event- 2nd March 2024 Q1bengalurutug
 
The market for cross-border mortgages in Europe
The market for cross-border mortgages in EuropeThe market for cross-border mortgages in Europe
The market for cross-border mortgages in Europe321k
 
STOCK PRICE ANALYSIS Furkan Ali TASCI --.pptx
STOCK PRICE ANALYSIS  Furkan Ali TASCI --.pptxSTOCK PRICE ANALYSIS  Furkan Ali TASCI --.pptx
STOCK PRICE ANALYSIS Furkan Ali TASCI --.pptxFurkanTasci3
 
Elements of language learning - an analysis of how different elements of lang...
Elements of language learning - an analysis of how different elements of lang...Elements of language learning - an analysis of how different elements of lang...
Elements of language learning - an analysis of how different elements of lang...PrithaVashisht1
 
Data Collection from Social Media Platforms
Data Collection from Social Media PlatformsData Collection from Social Media Platforms
Data Collection from Social Media PlatformsMahmoud Yasser
 
Neo4j Graph Summit 2024 Workshop - EMEA - Breda_and_Munchen.pdf
Neo4j Graph Summit 2024 Workshop - EMEA - Breda_and_Munchen.pdfNeo4j Graph Summit 2024 Workshop - EMEA - Breda_and_Munchen.pdf
Neo4j Graph Summit 2024 Workshop - EMEA - Breda_and_Munchen.pdfNeo4j
 
Neo4j_Jesus Barrasa_The Art of the Possible with Graph.pptx.pdf
Neo4j_Jesus Barrasa_The Art of the Possible with Graph.pptx.pdfNeo4j_Jesus Barrasa_The Art of the Possible with Graph.pptx.pdf
Neo4j_Jesus Barrasa_The Art of the Possible with Graph.pptx.pdfNeo4j
 
Stochastic Dynamic Programming and You.pptx
Stochastic Dynamic Programming and You.pptxStochastic Dynamic Programming and You.pptx
Stochastic Dynamic Programming and You.pptxjkmrshll88
 
TCFPro24 Building Real-Time Generative AI Pipelines
TCFPro24 Building Real-Time Generative AI PipelinesTCFPro24 Building Real-Time Generative AI Pipelines
TCFPro24 Building Real-Time Generative AI PipelinesTimothy Spann
 
Microeconomic Group Presentation Apple.pdf
Microeconomic Group Presentation Apple.pdfMicroeconomic Group Presentation Apple.pdf
Microeconomic Group Presentation Apple.pdfmxlos0
 
PPT for Presiding Officer.pptxvvdffdfgggg
PPT for Presiding Officer.pptxvvdffdfggggPPT for Presiding Officer.pptxvvdffdfgggg
PPT for Presiding Officer.pptxvvdffdfggggbhadratanusenapati1
 
Paul Martin (Gartner) - Show Me the AI Money.pdf
Paul Martin (Gartner) - Show Me the AI Money.pdfPaul Martin (Gartner) - Show Me the AI Money.pdf
Paul Martin (Gartner) - Show Me the AI Money.pdfdcphostmaster
 
Báo cáo Social Media Benchmark 2024 cho dân Marketing
Báo cáo Social Media Benchmark 2024 cho dân MarketingBáo cáo Social Media Benchmark 2024 cho dân Marketing
Báo cáo Social Media Benchmark 2024 cho dân MarketingMarketingTrips
 
Using DAX & Time-based Analysis in Data Warehouse
Using DAX & Time-based Analysis in Data WarehouseUsing DAX & Time-based Analysis in Data Warehouse
Using DAX & Time-based Analysis in Data WarehouseThinkInnovation
 
Deloitte+RedCross_Talk to your data with Knowledge-enriched Generative AI.ppt...
Deloitte+RedCross_Talk to your data with Knowledge-enriched Generative AI.ppt...Deloitte+RedCross_Talk to your data with Knowledge-enriched Generative AI.ppt...
Deloitte+RedCross_Talk to your data with Knowledge-enriched Generative AI.ppt...Neo4j
 

Recently uploaded (20)

How to Build an Experimentation Culture for Data-Driven Product Development
How to Build an Experimentation Culture for Data-Driven Product DevelopmentHow to Build an Experimentation Culture for Data-Driven Product Development
How to Build an Experimentation Culture for Data-Driven Product Development
 
Empowering Decisions A Guide to Embedded Analytics
Empowering Decisions A Guide to Embedded AnalyticsEmpowering Decisions A Guide to Embedded Analytics
Empowering Decisions A Guide to Embedded Analytics
 
Data Analytics Fundamentals: data analytics types.potx
Data Analytics Fundamentals: data analytics types.potxData Analytics Fundamentals: data analytics types.potx
Data Analytics Fundamentals: data analytics types.potx
 
Air Con Energy Rating Info411 Presentation.pdf
Air Con Energy Rating Info411 Presentation.pdfAir Con Energy Rating Info411 Presentation.pdf
Air Con Energy Rating Info411 Presentation.pdf
 
Prediction Of Cryptocurrency Prices Using Lstm, Svm And Polynomial Regression...
Prediction Of Cryptocurrency Prices Using Lstm, Svm And Polynomial Regression...Prediction Of Cryptocurrency Prices Using Lstm, Svm And Polynomial Regression...
Prediction Of Cryptocurrency Prices Using Lstm, Svm And Polynomial Regression...
 
Bengaluru Tableau UG event- 2nd March 2024 Q1
Bengaluru Tableau UG event- 2nd March 2024 Q1Bengaluru Tableau UG event- 2nd March 2024 Q1
Bengaluru Tableau UG event- 2nd March 2024 Q1
 
The market for cross-border mortgages in Europe
The market for cross-border mortgages in EuropeThe market for cross-border mortgages in Europe
The market for cross-border mortgages in Europe
 
STOCK PRICE ANALYSIS Furkan Ali TASCI --.pptx
STOCK PRICE ANALYSIS  Furkan Ali TASCI --.pptxSTOCK PRICE ANALYSIS  Furkan Ali TASCI --.pptx
STOCK PRICE ANALYSIS Furkan Ali TASCI --.pptx
 
Elements of language learning - an analysis of how different elements of lang...
Elements of language learning - an analysis of how different elements of lang...Elements of language learning - an analysis of how different elements of lang...
Elements of language learning - an analysis of how different elements of lang...
 
Data Collection from Social Media Platforms
Data Collection from Social Media PlatformsData Collection from Social Media Platforms
Data Collection from Social Media Platforms
 
Neo4j Graph Summit 2024 Workshop - EMEA - Breda_and_Munchen.pdf
Neo4j Graph Summit 2024 Workshop - EMEA - Breda_and_Munchen.pdfNeo4j Graph Summit 2024 Workshop - EMEA - Breda_and_Munchen.pdf
Neo4j Graph Summit 2024 Workshop - EMEA - Breda_and_Munchen.pdf
 
Neo4j_Jesus Barrasa_The Art of the Possible with Graph.pptx.pdf
Neo4j_Jesus Barrasa_The Art of the Possible with Graph.pptx.pdfNeo4j_Jesus Barrasa_The Art of the Possible with Graph.pptx.pdf
Neo4j_Jesus Barrasa_The Art of the Possible with Graph.pptx.pdf
 
Stochastic Dynamic Programming and You.pptx
Stochastic Dynamic Programming and You.pptxStochastic Dynamic Programming and You.pptx
Stochastic Dynamic Programming and You.pptx
 
TCFPro24 Building Real-Time Generative AI Pipelines
TCFPro24 Building Real-Time Generative AI PipelinesTCFPro24 Building Real-Time Generative AI Pipelines
TCFPro24 Building Real-Time Generative AI Pipelines
 
Microeconomic Group Presentation Apple.pdf
Microeconomic Group Presentation Apple.pdfMicroeconomic Group Presentation Apple.pdf
Microeconomic Group Presentation Apple.pdf
 
PPT for Presiding Officer.pptxvvdffdfgggg
PPT for Presiding Officer.pptxvvdffdfggggPPT for Presiding Officer.pptxvvdffdfgggg
PPT for Presiding Officer.pptxvvdffdfgggg
 
Paul Martin (Gartner) - Show Me the AI Money.pdf
Paul Martin (Gartner) - Show Me the AI Money.pdfPaul Martin (Gartner) - Show Me the AI Money.pdf
Paul Martin (Gartner) - Show Me the AI Money.pdf
 
Báo cáo Social Media Benchmark 2024 cho dân Marketing
Báo cáo Social Media Benchmark 2024 cho dân MarketingBáo cáo Social Media Benchmark 2024 cho dân Marketing
Báo cáo Social Media Benchmark 2024 cho dân Marketing
 
Using DAX & Time-based Analysis in Data Warehouse
Using DAX & Time-based Analysis in Data WarehouseUsing DAX & Time-based Analysis in Data Warehouse
Using DAX & Time-based Analysis in Data Warehouse
 
Deloitte+RedCross_Talk to your data with Knowledge-enriched Generative AI.ppt...
Deloitte+RedCross_Talk to your data with Knowledge-enriched Generative AI.ppt...Deloitte+RedCross_Talk to your data with Knowledge-enriched Generative AI.ppt...
Deloitte+RedCross_Talk to your data with Knowledge-enriched Generative AI.ppt...
 

Beyond Parallelize and Collect by Holden Karau

  • 1. Beyond Parallelize & Collect (Effective testing of Spark Programs) Now mostly “works”* *See developer for details. Does not imply warranty. :p
  • 2. Who am I? ● My name is Holden Karau ● Prefered pronouns are she/her ● I’m a Software Engineer ● currently IBM and previously Alpine, Databricks, Google, Foursquare & Amazon ● co-author of Learning Spark & Fast Data processing with Spark ● @holdenkarau ● Slide share http://www.slideshare.net/hkarau ● Linkedin https://www.linkedin.com/in/holdenkarau ● Spark Videos http://bit.ly/holdenSparkVideos
  • 3. What is going to be covered: ● What I think I might know about you ● A bit about why you should test your programs ● Using parallelize & collect for unit testing (quick skim) ● Comparing datasets too large to fit in memory ● Considerations for Streaming & SQL (DataFrames & Datasets) ● Cute & scary pictures ○ I promise at least one panda and one cat ● “Future Work” ○ Integration testing lives here for now (sorry) ○ Some of this future work might even get done!
  • 4. Who I think you wonderful humans are? ● Nice* people ● Like silly pictures ● Familiar with Apache Spark ○ If not, buy one of my books or watch Paco’s awesome video ● Familiar with one of Scala, Java, or Python ○ If you know R well I’d love to chat though ● Want to make better software ○ (or models, or w/e)
  • 5. So why should you test? ● Makes you a better person ● Save $s ○ May help you avoid losing your employer all of their money ■ Or “users” if we were in the bay ○ AWS is expensive ● Waiting for our jobs to fail is a pretty long dev cycle ● This is really just to guilt trip you & give you flashbacks to your QA internships
  • 6. So why should you test - continued Results from: Testing with Spark survey http://bit.ly/holdenTestingSpark
  • 7. So why should you test - continued Results from: Testing with Spark survey http://bit.ly/holdenTestingSpark
  • 8. Why don’t we test? ● It’s hard ○ Faking data, setting up integration tests, urgh w/e ● Our tests can get too slow ● It takes a lot of time ○ and people always want everything done yesterday ○ or I just want to go home see my partner ○ etc.
  • 9. Cat photo from http://galato901.deviantart.com/art/Cat-on-Work-Break-173043455
  • 10. An artisanal Spark unit test @transient private var _sc: SparkContext = _ override def beforeAll() { _sc = new SparkContext("local[4]") super.beforeAll() } override def afterAll() { if (sc != null) sc.stop() System.clearProperty("spark.driver.port") // rebind issue _sc = null super.afterAll() } Photo by morinesque
  • 11. And on to the actual test... test("really simple transformation") { val input = List("hi", "hi holden", "bye") val expected = List(List("hi"), List("hi", "holden"), List("bye")) assert(tokenize(sc.parallelize(input)).collect().toList === expected) } def tokenize(f: RDD[String]) = { f.map(_.split(" ").toList) } Photo by morinesque
  • 12. Wait, where were the batteries? Photo by Jim Bauer
  • 13. Let’s get batteries! ● Spark unit testing ○ spark-testing-base - https://github.com/holdenk/spark-testing-base ○ sscheck - https://github.com/juanrh/sscheck ● Integration testing ○ spark-integration-tests (Spark internals) - https://github.com/databricks/spark-integration-tests ● Performance ○ spark-perf (also for Spark internals) - https://github.com/databricks/spark-perf ● Spark job validation ○ spark-validator - https://github.com/holdenk/spark-validator Photo by Mike Mozart
  • 14. A simple unit test re-visited (Scala) class SampleRDDTest extends FunSuite with SharedSparkContext { test("really simple transformation") { val input = List("hi", "hi holden", "bye") val expected = List(List("hi"), List("hi", "holden"), List("bye")) assert(SampleRDD.tokenize(sc.parallelize(input)).collect().toList === expected) } }
  • 15. Ok but what about problems @ scale ● Maybe our program works fine on our local sized input ● If we are using Spark our actual workload is probably huge ● How do we test workloads too large for a single machine? ○ we can’t just use parallelize and collect Qfamily
  • 16. Distributed “set” operations to the rescue* ● Pretty close - already built into Spark ● Doesn’t do so well with floating points :( ○ damn floating points keep showing up everywhere :p ● Doesn’t really handle duplicates very well ○ {“coffee”, “coffee”, “panda”} != {“panda”, “coffee”} but with set operations... Matti Mattila
  • 17. Or use RDDComparisions: def compareWithOrderSamePartitioner[T: ClassTag](expected: RDD [T], result: RDD[T]): Option[(T, T)] = { expected.zip(result).filter{case (x, y) => x != y}.take(1). headOption } Matti Mattila
  • 18. Or use RDDComparisions: def compare[T: ClassTag](expected: RDD[T], result: RDD[T]): Option [(T, Int, Int)] = { val expectedKeyed = expected.map(x => (x, 1)).reduceByKey(_ + _) val resultKeyed = result.map(x => (x, 1)).reduceByKey(_ + _) expectedKeyed.cogroup(resultKeyed).filter{case (_, (i1, i2)) => i1.isEmpty || i2.isEmpty || i1.head != i2.head}.take(1). headOption. map{case (v, (i1, i2)) => (v, i1.headOption.getOrElse(0), i2.headOption.getOrElse(0))} } Matti Mattila
  • 19. But where do we get the data for those tests? ● If you have production data you can sample you are lucky! ○ If possible you can try and save in the same format ● If our data is a bunch of Vectors or Doubles Spark’s got tools :) ● Coming up with good test data can take a long time Lori Rielly
  • 20. QuickCheck / ScalaCheck ● QuickCheck generates tests data under a set of constraints ● Scala version is ScalaCheck - supported by the two unit testing libraries for Spark ● sscheck ○ Awesome people*, supports generating DStreams too! ● spark-testing-base ○ Also Awesome people*, generates more pathological (e.g. empty partitions etc.) RDDs *I assume PROtara hunt
  • 21. With spark-testing-base test("map should not change number of elements") { forAll(RDDGenerator.genRDD[String](sc)){ rdd => rdd.map(_.length).count() == rdd.count() } }
  • 23. // Setup our Stream: class TestInputStream[T: ClassTag](@transient var sc: SparkContext, ssc_ : StreamingContext, input: Seq[Seq[T]], numPartitions: Int) extends FriendlyInputDStream[T](ssc_) { def start() {} def stop() {} def compute(validTime: Time): Option[RDD[T]] = { logInfo("Computing RDD for time " + validTime) val index = ((validTime - ourZeroTime) / slideDuration - 1). toInt val selectedInput = if (index < input.size) input(index) else Seq[T]() // lets us test cases where RDDs are not created if (selectedInput == null) { return None } val rdd = sc.makeRDD(selectedInput, numPartitions) logInfo("Created RDD " + rdd.id + " with " + selectedInput) Some(rdd) } } Artisanal Stream Testing Code trait StreamingSuiteBase extends FunSuite with BeforeAndAfterAll with Logging with SharedSparkContext { // Name of the framework for Spark context def framework: String = this.getClass.getSimpleName // Master for Spark context def master: String = "local[4]" // Batch duration def batchDuration: Duration = Seconds(1) // Directory where the checkpoint data will be saved lazy val checkpointDir = { val dir = Utils.createTempDir() logDebug(s"checkpointDir: $dir") dir.toString } // Default after function for any streaming test suite. Override this // if you want to add your stuff to "after" (i.e., don't call after { } ) override def afterAll() { System.clearProperty("spark.streaming.clock") super.afterAll() } Phot o by Stev e Jurv etso n
  • 24. and continued…. /** * Create an input stream for the provided input sequence. This is done using * TestInputStream as queueStream's are not checkpointable. */ def createTestInputStream[T: ClassTag](sc: SparkContext, ssc_ : TestStreamingContext, input: Seq[Seq[T]]): TestInputStream[T] = { new TestInputStream(sc, ssc_, input, numInputPartitions) } // Default before function for any streaming test suite. Override this // if you want to add your stuff to "before" (i.e., don't call before { } ) override def beforeAll() { if (useManualClock) { logInfo("Using manual clock") conf.set("spark.streaming.clock", "org.apache.spark.streaming.util. TestManualClock") // We can specify our own clock } else { logInfo("Using real clock") conf.set("spark.streaming.clock", "org.apache.spark.streaming.util.SystemClock") } super.beforeAll() } /** * Run a block of code with the given StreamingContext and automatically * stop the context when the block completes or when an exception is thrown. */ def withOutputAndStreamingContext[R](outputStreamSSC: (TestOutputStream [R], TestStreamingContext)) (block: (TestOutputStream[R], TestStreamingContext) => Unit): Unit = { val outputStream = outputStreamSSC._1 val ssc = outputStreamSSC._2 try { block(outputStream, ssc) } finally { try { ssc.stop(stopSparkContext = false) } catch { case e: Exception => logError("Error stopping StreamingContext", e) } } } }
  • 25. and now for the clock /* * Allows us access to a manual clock. Note that the manual clock changed between 1.1.1 and 1.3 */ class TestManualClock(var time: Long) extends Clock { def this() = this(0L) def getTime(): Long = getTimeMillis() // Compat def currentTime(): Long = getTimeMillis() // Compat def getTimeMillis(): Long = synchronized { time } def setTime(timeToSet: Long): Unit = synchronized { time = timeToSet notifyAll() } def advance(timeToAdd: Long): Unit = synchronized { time += timeToAdd notifyAll() } def addToTime(timeToAdd: Long): Unit = advance(timeToAdd) // Compat /** * @param targetTime block until the clock time is set or advanced to at least this time * @return current time reported by the clock when waiting finishes */ def waitTillTime(targetTime: Long): Long = synchronized { while (time < targetTime) { wait(100) } getTimeMillis() } }
  • 26. Testing streaming the happy panda way ● Creating test data is hard ○ ssc.queueStream works - unless you need checkpoints (1.4.1+) ● Collecting the data locally is hard ○ foreachRDD & a var ● figuring out when your test is “done” Let’s abstract all that away into testOperation
  • 27. We can hide all of that: test("really simple transformation") { val input = List(List("hi"), List("hi holden"), List("bye")) val expected = List(List("hi"), List("hi", "holden"), List("bye")) testOperation[String, String](input, tokenize _, expected, useSet = true) } Photo by An eye for my mind
  • 28. What about DataFrames? ● We can do the same as we did for RDD’s (.rdd) ● Inside of Spark validation looks like: def checkAnswer(df: DataFrame, expectedAnswer: Seq[Row]) ● Sadly it’s not in a published package & local only ● instead we expose: def equalDataFrames(expected: DataFrame, result: DataFrame) { def approxEqualDataFrames(e: DataFrame, r: DataFrame, tol: Double) {
  • 29. …. and Datasets ● We can do the same as we did for RDD’s (.rdd) ● Inside of Spark validation looks like: def checkAnswer(df: Dataset[T], expectedAnswer: T*) ● Sadly it’s not in a published package & local only ● instead we expose: def equalDatasets(expected: Dataset[U], result: Dataset[V]) { def approxEqualDatasets(e: Dataset[U], r: Dataset[V], tol: Double) {
  • 30. This is what it looks like: test("dataframe should be equal to its self") { val sqlCtx = sqlContext import sqlCtx.implicits._// Yah I know this is ugly val input = sc.parallelize(inputList).toDF equalDataFrames(input, input) } *This may or may not be easier.
  • 31. Which has “built-in” large support :)
  • 33. Let’s talk about local mode ● It’s way better than you would expect* ● It does its best to try and catch serialization errors ● It’s still not the same as running on a “real” cluster ● Especially since if we were just local mode, parallelize and collect might be fine Photo by: Bev Sykes
  • 34. Options beyond local mode: ● Just point at your existing cluster (set master) ● Start one with your shell scripts & change the master ○ Really easy way to plug into existing integration testing ● spark-docker - hack in our own tests ● YarnMiniCluster ○ https://github. com/apache/spark/blob/master/yarn/src/test/scala/org/apache/spark/deploy/yarn/BaseYarnClu sterSuite.scala ○ In Spark Testing Base extend SharedMiniCluster ■ Not recommended until after SPARK-10812 (e.g. 1.5.2+ or 1.6+) Photo by Richard Masoner
  • 35. Validation ● Validation can be really useful for catching errors before deploying a model ○ Our tests can’t catch everything ● For now checking file sizes & execution time seem like the most common best practice (from survey) ● Accumulators have some challenges (see SPARK-12469 for progress) but are an interesting option ● spark-validator is still in early stages and not ready for production use but interesting proof of concept Photo by: Paul Schadler
  • 36. Related talks & blog posts ● Testing Spark Best Practices (Spark Summit 2014) ● Every Day I’m Shuffling (Strata 2015) & slides ● Spark and Spark Streaming Unit Testing ● Making Spark Unit Testing With Spark Testing Base
  • 37. Learning Spark Fast Data Processing with Spark (Out of Date) Fast Data Processing with Spark (2nd edition) Advanced Analytics with Spark
  • 38. Learning Spark Fast Data Processing with Spark (Out of Date) Fast Data Processing with Spark (2nd edition) Advanced Analytics with Spark Coming soon: Spark in Action Coming soon: High Performance Spark
  • 39. And the next book….. Still being written - signup to be notified when it is available: ● http://www.highperformancespark.com ● https://twitter.com/highperfspark
  • 40. Related packages ● spark-testing-base: https://github.com/holdenk/spark-testing-base ● sscheck: https://github.com/juanrh/sscheck ● spark-validator: https://github.com/holdenk/spark-validator *ALPHA* ● spark-perf - https://github.com/databricks/spark-perf ● spark-integration-tests - https://github.com/databricks/spark-integration-tests ● scalacheck - https://www.scalacheck.org/
  • 41. And including spark-testing-base: sbt: "com.holdenkarau" %% "spark-testing-base" % "1.5.2_0.3.1" maven: <dependency> <groupId>com.holdenkarau</groupId> <artifactId>spark-testing-base</artifactId> <version>${spark.version}_0.3.1</version> <scope>test</scope> </dependency>
  • 42. “Future Work” ● Better ScalaCheck integration (ala sscheck) ● Testing details in my next Spark book ● Whatever* you all want ○ Testing with Spark survey: http://bit.ly/holdenTestingSpark Semi-likely: ● integration testing (for now see @cfriegly’s Spark + Docker setup): ○ https://github.com/fluxcapacitor/pipeline Pretty unlikely: ● Integrating into Apache Spark ( SPARK-12433 ) *That I feel like doing, or you feel like making a pull request for. Photo by bullet101
  • 43. Cat wave photo by Quinn Dombrowski k thnx bye! If you want to fill out survey: http: //bit.ly/holdenTestingSpark Will use update results in Strata Presentation & tweet eventually at @holdenkarau