SlideShare a Scribd company logo
1 of 38
Download to read offline
Effective Testing for Spark
Programs
or avoiding “I didn’t think that could happen”
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 Alpine and previously 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
● Doing traditional unit testing for Spark programs
○ Along with special considerations for SQL/Hive & Streaming
● Using counters & other job acceptance tests w/ Spark
● Cute & scary pictures
○ I promise at least one panda and one cat
● “Future Work”
○ 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
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)
}
}
A simple unit test (Java)
public class SampleJavaRDDTest
extends SharedJavaSparkContext implements Serializable {
@Test public void verifyMapTest() {
List<Integer> input = Arrays.asList(1,2);
JavaRDD<Integer> result = jsc().parallelize(input).map(
new Function<Integer, Integer>() { public Integer call(Integer x) { return
x * x;}});
assertEquals(input.size(), result.count());
}
}
A simple unit test (Python)
class SimpleTest(SparkTestingBaseTestCase):
"""A simple test."""
def test_basic(self):
"""Test a simple collect."""
input = ["hello world"]
rdd = self.sc.parallelize(input)
result = rdd.collect()
assert result == input
Making fake data
● If you have production data you can sample you are lucky!
○ If possible you can try and save in the same format
● sc.parallelize is pretty good for small tests
○ Note: that we can specify the number of partitions
● 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 RDDs
*I assume
PROtara hunt
With sscheck
def forallRDDGenOfNtoM = {
val minWords, maxWords = (50, 100)
Prop.forAll(RDDGen.ofNtoM(50, 100, arbitrary[String])) { rdd : RDD[String] =>
rdd.map(_.length()).sum must be_>=(0.0)
}
}
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
● Inside of Spark validation looks like:
def checkAnswer(df: DataFrame, expectedAnswer: Seq[Row])
● Sadly it’s not in a published package :(
def equalDataFrames(expected: DataFrame, result: DataFrame) {
def approxEqualDataFrames(e: DataFrame, r: DataFrame, tol: Double) {
We can make it easier!*
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.
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
Photo by: Bev Sykes
Running on a real* cluster
● 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 is merged & (maybe 1.6 ?)
Photo by Richard Masoner
On to validation
Why should we validate our jobs?
● Our code will most likely fail
○ Sometimes data sources fail in new & exciting ways (see Mongo)
○ That jerk on that other floor changed the meaning of a field :(
○ Our tests won’t catch all of the corner cases that the real world finds
● We should try and minimize the impact
○ Avoid making potentially embarrassing recommendations
○ Save having to be woken up at 3am to do a roll-back
○ Specifying a few simple invariants isn’t that hard
Photo of GSM intercept by Matt E
So how do we validate our jobs?
● Spark has it own counters
○ Per-stage bytes r/w, shuffle r/w, record r/w. execution time, etc.
● We can add counters for things we care about
○ invalid records, users with no recommendations, etc.
● We can write rules for if the values are expected
○ Simple rules (X > J)
○ Historic rules (X > Avg(J))
Photo by:
Paul Schadler
Simple historic validation Photo by Dvortygirl
val vc = new ValidationConf(jobHistoryPath, "1", true,
List[ValidationRule](new AvgRule("acc", 0.001, Some(200))))
val v = Validation(sc, vc)
// Some job logic
// Register an accumulator (optional)
val acc = sc.accumulator(0)
v.registerAccumulator(acc, "acc")
// More Job logic goes here
if (v.validate(jobId)) {
// Success logic goes here
} else sadness()
With a Spark internal counter...
val vc = new ValidationConf(tempPath, "1", true,
List[ValidationRule](
new AbsoluteSparkCounterValidationRule("recordsRead", Some(30), Some
(1000)))
)
val sqlCtx = new SQLContext(sc)
val v = Validation(sc, sqlCtx, vc)
//Do work here....
assert(v.validate(5) === true)
}
Photo by Dvortygirl
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
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/
“Future Work”
● Integrating into Apache Spark
○ Using their style rules to simplify future transition
● Better ScalaCheck integration (with the help of the sscheck people)
● Some reasonable prefab rules for Job validation
● Testing details in my next Spark book
● Whatever* you all want
○ Testing with Spark survey: http://bit.ly/holdenTestingSpark
Semi-likely:
● integration testing
*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 tweet results
“eventually” @holdenkarau

More Related Content

What's hot

Testing data streaming applications
Testing data streaming applicationsTesting data streaming applications
Testing data streaming applicationsLars Albertsson
 
Apache Spark Core – Practical Optimization
Apache Spark Core – Practical OptimizationApache Spark Core – Practical Optimization
Apache Spark Core – Practical OptimizationDatabricks
 
OSMC 2022 | VictoriaMetrics: scaling to 100 million metrics per second by Ali...
OSMC 2022 | VictoriaMetrics: scaling to 100 million metrics per second by Ali...OSMC 2022 | VictoriaMetrics: scaling to 100 million metrics per second by Ali...
OSMC 2022 | VictoriaMetrics: scaling to 100 million metrics per second by Ali...NETWAYS
 
The Parquet Format and Performance Optimization Opportunities
The Parquet Format and Performance Optimization OpportunitiesThe Parquet Format and Performance Optimization Opportunities
The Parquet Format and Performance Optimization OpportunitiesDatabricks
 
What's the time? ...and why? (Mattias Sax, Confluent) Kafka Summit SF 2019
What's the time? ...and why? (Mattias Sax, Confluent) Kafka Summit SF 2019What's the time? ...and why? (Mattias Sax, Confluent) Kafka Summit SF 2019
What's the time? ...and why? (Mattias Sax, Confluent) Kafka Summit SF 2019confluent
 
Airflow를 이용한 데이터 Workflow 관리
Airflow를 이용한  데이터 Workflow 관리Airflow를 이용한  데이터 Workflow 관리
Airflow를 이용한 데이터 Workflow 관리YoungHeon (Roy) Kim
 
[2D1]Elasticsearch 성능 최적화
[2D1]Elasticsearch 성능 최적화[2D1]Elasticsearch 성능 최적화
[2D1]Elasticsearch 성능 최적화NAVER D2
 
Apache Calcite: A Foundational Framework for Optimized Query Processing Over ...
Apache Calcite: A Foundational Framework for Optimized Query Processing Over ...Apache Calcite: A Foundational Framework for Optimized Query Processing Over ...
Apache Calcite: A Foundational Framework for Optimized Query Processing Over ...Julian Hyde
 
Amazon Redshift: Performance Tuning and Optimization
Amazon Redshift: Performance Tuning and OptimizationAmazon Redshift: Performance Tuning and Optimization
Amazon Redshift: Performance Tuning and OptimizationAmazon Web Services
 
Alfresco Content Modelling and Policy Behaviours
Alfresco Content Modelling and Policy BehavioursAlfresco Content Modelling and Policy Behaviours
Alfresco Content Modelling and Policy BehavioursJ V
 
Image Processing on Delta Lake
Image Processing on Delta LakeImage Processing on Delta Lake
Image Processing on Delta LakeDatabricks
 
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
 
PostgreSQL High_Performance_Cheatsheet
PostgreSQL High_Performance_CheatsheetPostgreSQL High_Performance_Cheatsheet
PostgreSQL High_Performance_CheatsheetLucian Oprea
 
Spark Operator—Deploy, Manage and Monitor Spark clusters on Kubernetes
 Spark Operator—Deploy, Manage and Monitor Spark clusters on Kubernetes Spark Operator—Deploy, Manage and Monitor Spark clusters on Kubernetes
Spark Operator—Deploy, Manage and Monitor Spark clusters on KubernetesDatabricks
 
[2019] 200만 동접 게임을 위한 MySQL 샤딩
[2019] 200만 동접 게임을 위한 MySQL 샤딩[2019] 200만 동접 게임을 위한 MySQL 샤딩
[2019] 200만 동접 게임을 위한 MySQL 샤딩NHN FORWARD
 
Morel, a Functional Query Language
Morel, a Functional Query LanguageMorel, a Functional Query Language
Morel, a Functional Query LanguageJulian Hyde
 
Apache kafka performance(latency)_benchmark_v0.3
Apache kafka performance(latency)_benchmark_v0.3Apache kafka performance(latency)_benchmark_v0.3
Apache kafka performance(latency)_benchmark_v0.3SANG WON PARK
 
Tutorial on developing a Solr search component plugin
Tutorial on developing a Solr search component pluginTutorial on developing a Solr search component plugin
Tutorial on developing a Solr search component pluginsearchbox-com
 

What's hot (20)

Testing data streaming applications
Testing data streaming applicationsTesting data streaming applications
Testing data streaming applications
 
Apache Spark Core – Practical Optimization
Apache Spark Core – Practical OptimizationApache Spark Core – Practical Optimization
Apache Spark Core – Practical Optimization
 
OSMC 2022 | VictoriaMetrics: scaling to 100 million metrics per second by Ali...
OSMC 2022 | VictoriaMetrics: scaling to 100 million metrics per second by Ali...OSMC 2022 | VictoriaMetrics: scaling to 100 million metrics per second by Ali...
OSMC 2022 | VictoriaMetrics: scaling to 100 million metrics per second by Ali...
 
The Parquet Format and Performance Optimization Opportunities
The Parquet Format and Performance Optimization OpportunitiesThe Parquet Format and Performance Optimization Opportunities
The Parquet Format and Performance Optimization Opportunities
 
Amazon DynamoDB 키 디자인 패턴
Amazon DynamoDB 키 디자인 패턴Amazon DynamoDB 키 디자인 패턴
Amazon DynamoDB 키 디자인 패턴
 
What's the time? ...and why? (Mattias Sax, Confluent) Kafka Summit SF 2019
What's the time? ...and why? (Mattias Sax, Confluent) Kafka Summit SF 2019What's the time? ...and why? (Mattias Sax, Confluent) Kafka Summit SF 2019
What's the time? ...and why? (Mattias Sax, Confluent) Kafka Summit SF 2019
 
Airflow를 이용한 데이터 Workflow 관리
Airflow를 이용한  데이터 Workflow 관리Airflow를 이용한  데이터 Workflow 관리
Airflow를 이용한 데이터 Workflow 관리
 
[2D1]Elasticsearch 성능 최적화
[2D1]Elasticsearch 성능 최적화[2D1]Elasticsearch 성능 최적화
[2D1]Elasticsearch 성능 최적화
 
Apache Calcite: A Foundational Framework for Optimized Query Processing Over ...
Apache Calcite: A Foundational Framework for Optimized Query Processing Over ...Apache Calcite: A Foundational Framework for Optimized Query Processing Over ...
Apache Calcite: A Foundational Framework for Optimized Query Processing Over ...
 
Amazon Redshift: Performance Tuning and Optimization
Amazon Redshift: Performance Tuning and OptimizationAmazon Redshift: Performance Tuning and Optimization
Amazon Redshift: Performance Tuning and Optimization
 
Alfresco Content Modelling and Policy Behaviours
Alfresco Content Modelling and Policy BehavioursAlfresco Content Modelling and Policy Behaviours
Alfresco Content Modelling and Policy Behaviours
 
Image Processing on Delta Lake
Image Processing on Delta LakeImage Processing on Delta Lake
Image Processing on Delta Lake
 
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
 
PostgreSQL High_Performance_Cheatsheet
PostgreSQL High_Performance_CheatsheetPostgreSQL High_Performance_Cheatsheet
PostgreSQL High_Performance_Cheatsheet
 
Druid deep dive
Druid deep diveDruid deep dive
Druid deep dive
 
Spark Operator—Deploy, Manage and Monitor Spark clusters on Kubernetes
 Spark Operator—Deploy, Manage and Monitor Spark clusters on Kubernetes Spark Operator—Deploy, Manage and Monitor Spark clusters on Kubernetes
Spark Operator—Deploy, Manage and Monitor Spark clusters on Kubernetes
 
[2019] 200만 동접 게임을 위한 MySQL 샤딩
[2019] 200만 동접 게임을 위한 MySQL 샤딩[2019] 200만 동접 게임을 위한 MySQL 샤딩
[2019] 200만 동접 게임을 위한 MySQL 샤딩
 
Morel, a Functional Query Language
Morel, a Functional Query LanguageMorel, a Functional Query Language
Morel, a Functional Query Language
 
Apache kafka performance(latency)_benchmark_v0.3
Apache kafka performance(latency)_benchmark_v0.3Apache kafka performance(latency)_benchmark_v0.3
Apache kafka performance(latency)_benchmark_v0.3
 
Tutorial on developing a Solr search component plugin
Tutorial on developing a Solr search component pluginTutorial on developing a Solr search component plugin
Tutorial on developing a Solr search component plugin
 

Similar to Effective testing for spark programs Strata NY 2015

Beyond Parallelize and Collect by Holden Karau
Beyond Parallelize and Collect by Holden KarauBeyond Parallelize and Collect by Holden Karau
Beyond Parallelize and Collect by Holden KarauSpark Summit
 
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 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
 
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 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
 
Cool JVM Tools to Help You Test
Cool JVM Tools to Help You TestCool JVM Tools to Help You Test
Cool JVM Tools to Help You TestSchalk Cronjé
 
Getting Started with Spark Structured Streaming - Current 22
Getting Started with Spark Structured Streaming - Current 22Getting Started with Spark Structured Streaming - Current 22
Getting Started with Spark Structured Streaming - Current 22Dustin Vannoy
 
Getting Started With Spark Structured Streaming With Dustin Vannoy | Current ...
Getting Started With Spark Structured Streaming With Dustin Vannoy | Current ...Getting Started With Spark Structured Streaming With Dustin Vannoy | Current ...
Getting Started With Spark Structured Streaming With Dustin Vannoy | Current ...HostedbyConfluent
 
Developer Test - Things to Know
Developer Test - Things to KnowDeveloper Test - Things to Know
Developer Test - Things to Knowvilniusjug
 
Pocket Talk; Spock framework
Pocket Talk; Spock frameworkPocket Talk; Spock framework
Pocket Talk; Spock frameworkInfoway
 
33rd Degree 2013, Bad Tests, Good Tests
33rd Degree 2013, Bad Tests, Good Tests33rd Degree 2013, Bad Tests, Good Tests
33rd Degree 2013, Bad Tests, Good TestsTomek Kaczanowski
 
Smart Data Conference: DL4J and DataVec
Smart Data Conference: DL4J and DataVecSmart Data Conference: DL4J and DataVec
Smart Data Conference: DL4J and DataVecJosh Patterson
 
Unit Testing - The Whys, Whens and Hows
Unit Testing - The Whys, Whens and HowsUnit Testing - The Whys, Whens and Hows
Unit Testing - The Whys, Whens and Howsatesgoral
 
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
 
ITB_2023_CommandBox_Task_Runners_Brad_Wood.pdf
ITB_2023_CommandBox_Task_Runners_Brad_Wood.pdfITB_2023_CommandBox_Task_Runners_Brad_Wood.pdf
ITB_2023_CommandBox_Task_Runners_Brad_Wood.pdfOrtus Solutions, Corp
 
Parallel Programming With Dot Net
Parallel Programming With Dot NetParallel Programming With Dot Net
Parallel Programming With Dot NetNeeraj Kaushik
 

Similar to Effective testing for spark programs Strata NY 2015 (20)

Beyond Parallelize and Collect by Holden Karau
Beyond Parallelize and Collect by Holden KarauBeyond Parallelize and Collect by Holden Karau
Beyond Parallelize and Collect by Holden 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 2016
 
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
 
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 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)
 
Testing with PostgreSQL
Testing with PostgreSQLTesting with PostgreSQL
Testing with PostgreSQL
 
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...
 
Cool JVM Tools to Help You Test
Cool JVM Tools to Help You TestCool JVM Tools to Help You Test
Cool JVM Tools to Help You Test
 
Getting Started with Spark Structured Streaming - Current 22
Getting Started with Spark Structured Streaming - Current 22Getting Started with Spark Structured Streaming - Current 22
Getting Started with Spark Structured Streaming - Current 22
 
Getting Started With Spark Structured Streaming With Dustin Vannoy | Current ...
Getting Started With Spark Structured Streaming With Dustin Vannoy | Current ...Getting Started With Spark Structured Streaming With Dustin Vannoy | Current ...
Getting Started With Spark Structured Streaming With Dustin Vannoy | Current ...
 
Developer Test - Things to Know
Developer Test - Things to KnowDeveloper Test - Things to Know
Developer Test - Things to Know
 
Pocket Talk; Spock framework
Pocket Talk; Spock frameworkPocket Talk; Spock framework
Pocket Talk; Spock framework
 
Spock framework
Spock frameworkSpock framework
Spock framework
 
33rd Degree 2013, Bad Tests, Good Tests
33rd Degree 2013, Bad Tests, Good Tests33rd Degree 2013, Bad Tests, Good Tests
33rd Degree 2013, Bad Tests, Good Tests
 
Smart Data Conference: DL4J and DataVec
Smart Data Conference: DL4J and DataVecSmart Data Conference: DL4J and DataVec
Smart Data Conference: DL4J and DataVec
 
Unit Testing - The Whys, Whens and Hows
Unit Testing - The Whys, Whens and HowsUnit Testing - The Whys, Whens and Hows
Unit Testing - The Whys, Whens and Hows
 
Spock
SpockSpock
Spock
 
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
 
ITB_2023_CommandBox_Task_Runners_Brad_Wood.pdf
ITB_2023_CommandBox_Task_Runners_Brad_Wood.pdfITB_2023_CommandBox_Task_Runners_Brad_Wood.pdf
ITB_2023_CommandBox_Task_Runners_Brad_Wood.pdf
 
Parallel Programming With Dot Net
Parallel Programming With Dot NetParallel Programming With Dot Net
Parallel Programming With Dot Net
 

Recently uploaded

Brighton SEO | April 2024 | Data Storytelling
Brighton SEO | April 2024 | Data StorytellingBrighton SEO | April 2024 | Data Storytelling
Brighton SEO | April 2024 | Data StorytellingNeil Barnes
 
RA-11058_IRR-COMPRESS Do 198 series of 1998
RA-11058_IRR-COMPRESS Do 198 series of 1998RA-11058_IRR-COMPRESS Do 198 series of 1998
RA-11058_IRR-COMPRESS Do 198 series of 1998YohFuh
 
Ukraine War presentation: KNOW THE BASICS
Ukraine War presentation: KNOW THE BASICSUkraine War presentation: KNOW THE BASICS
Ukraine War presentation: KNOW THE BASICSAishani27
 
RadioAdProWritingCinderellabyButleri.pdf
RadioAdProWritingCinderellabyButleri.pdfRadioAdProWritingCinderellabyButleri.pdf
RadioAdProWritingCinderellabyButleri.pdfgstagge
 
VIP Call Girls in Amravati Aarohi 8250192130 Independent Escort Service Amravati
VIP Call Girls in Amravati Aarohi 8250192130 Independent Escort Service AmravatiVIP Call Girls in Amravati Aarohi 8250192130 Independent Escort Service Amravati
VIP Call Girls in Amravati Aarohi 8250192130 Independent Escort Service AmravatiSuhani Kapoor
 
代办国外大学文凭《原版美国UCLA文凭证书》加州大学洛杉矶分校毕业证制作成绩单修改
代办国外大学文凭《原版美国UCLA文凭证书》加州大学洛杉矶分校毕业证制作成绩单修改代办国外大学文凭《原版美国UCLA文凭证书》加州大学洛杉矶分校毕业证制作成绩单修改
代办国外大学文凭《原版美国UCLA文凭证书》加州大学洛杉矶分校毕业证制作成绩单修改atducpo
 
(PARI) Call Girls Wanowrie ( 7001035870 ) HI-Fi Pune Escorts Service
(PARI) Call Girls Wanowrie ( 7001035870 ) HI-Fi Pune Escorts Service(PARI) Call Girls Wanowrie ( 7001035870 ) HI-Fi Pune Escorts Service
(PARI) Call Girls Wanowrie ( 7001035870 ) HI-Fi Pune Escorts Serviceranjana rawat
 
Low Rate Call Girls Bhilai Anika 8250192130 Independent Escort Service Bhilai
Low Rate Call Girls Bhilai Anika 8250192130 Independent Escort Service BhilaiLow Rate Call Girls Bhilai Anika 8250192130 Independent Escort Service Bhilai
Low Rate Call Girls Bhilai Anika 8250192130 Independent Escort Service BhilaiSuhani Kapoor
 
Call Girls in Defence Colony Delhi 💯Call Us 🔝8264348440🔝
Call Girls in Defence Colony Delhi 💯Call Us 🔝8264348440🔝Call Girls in Defence Colony Delhi 💯Call Us 🔝8264348440🔝
Call Girls in Defence Colony Delhi 💯Call Us 🔝8264348440🔝soniya singh
 
Schema on read is obsolete. Welcome metaprogramming..pdf
Schema on read is obsolete. Welcome metaprogramming..pdfSchema on read is obsolete. Welcome metaprogramming..pdf
Schema on read is obsolete. Welcome metaprogramming..pdfLars Albertsson
 
B2 Creative Industry Response Evaluation.docx
B2 Creative Industry Response Evaluation.docxB2 Creative Industry Response Evaluation.docx
B2 Creative Industry Response Evaluation.docxStephen266013
 
Customer Service Analytics - Make Sense of All Your Data.pptx
Customer Service Analytics - Make Sense of All Your Data.pptxCustomer Service Analytics - Make Sense of All Your Data.pptx
Customer Service Analytics - Make Sense of All Your Data.pptxEmmanuel Dauda
 
100-Concepts-of-AI by Anupama Kate .pptx
100-Concepts-of-AI by Anupama Kate .pptx100-Concepts-of-AI by Anupama Kate .pptx
100-Concepts-of-AI by Anupama Kate .pptxAnupama Kate
 
VIP Call Girls Service Miyapur Hyderabad Call +91-8250192130
VIP Call Girls Service Miyapur Hyderabad Call +91-8250192130VIP Call Girls Service Miyapur Hyderabad Call +91-8250192130
VIP Call Girls Service Miyapur Hyderabad Call +91-8250192130Suhani Kapoor
 
20240419 - Measurecamp Amsterdam - SAM.pdf
20240419 - Measurecamp Amsterdam - SAM.pdf20240419 - Measurecamp Amsterdam - SAM.pdf
20240419 - Measurecamp Amsterdam - SAM.pdfHuman37
 
Data Science Project: Advancements in Fetal Health Classification
Data Science Project: Advancements in Fetal Health ClassificationData Science Project: Advancements in Fetal Health Classification
Data Science Project: Advancements in Fetal Health ClassificationBoston Institute of Analytics
 
Market Analysis in the 5 Largest Economic Countries in Southeast Asia.pdf
Market Analysis in the 5 Largest Economic Countries in Southeast Asia.pdfMarket Analysis in the 5 Largest Economic Countries in Southeast Asia.pdf
Market Analysis in the 5 Largest Economic Countries in Southeast Asia.pdfRachmat Ramadhan H
 
Unveiling Insights: The Role of a Data Analyst
Unveiling Insights: The Role of a Data AnalystUnveiling Insights: The Role of a Data Analyst
Unveiling Insights: The Role of a Data AnalystSamantha Rae Coolbeth
 

Recently uploaded (20)

Brighton SEO | April 2024 | Data Storytelling
Brighton SEO | April 2024 | Data StorytellingBrighton SEO | April 2024 | Data Storytelling
Brighton SEO | April 2024 | Data Storytelling
 
RA-11058_IRR-COMPRESS Do 198 series of 1998
RA-11058_IRR-COMPRESS Do 198 series of 1998RA-11058_IRR-COMPRESS Do 198 series of 1998
RA-11058_IRR-COMPRESS Do 198 series of 1998
 
Ukraine War presentation: KNOW THE BASICS
Ukraine War presentation: KNOW THE BASICSUkraine War presentation: KNOW THE BASICS
Ukraine War presentation: KNOW THE BASICS
 
RadioAdProWritingCinderellabyButleri.pdf
RadioAdProWritingCinderellabyButleri.pdfRadioAdProWritingCinderellabyButleri.pdf
RadioAdProWritingCinderellabyButleri.pdf
 
VIP Call Girls in Amravati Aarohi 8250192130 Independent Escort Service Amravati
VIP Call Girls in Amravati Aarohi 8250192130 Independent Escort Service AmravatiVIP Call Girls in Amravati Aarohi 8250192130 Independent Escort Service Amravati
VIP Call Girls in Amravati Aarohi 8250192130 Independent Escort Service Amravati
 
VIP Call Girls Service Charbagh { Lucknow Call Girls Service 9548273370 } Boo...
VIP Call Girls Service Charbagh { Lucknow Call Girls Service 9548273370 } Boo...VIP Call Girls Service Charbagh { Lucknow Call Girls Service 9548273370 } Boo...
VIP Call Girls Service Charbagh { Lucknow Call Girls Service 9548273370 } Boo...
 
代办国外大学文凭《原版美国UCLA文凭证书》加州大学洛杉矶分校毕业证制作成绩单修改
代办国外大学文凭《原版美国UCLA文凭证书》加州大学洛杉矶分校毕业证制作成绩单修改代办国外大学文凭《原版美国UCLA文凭证书》加州大学洛杉矶分校毕业证制作成绩单修改
代办国外大学文凭《原版美国UCLA文凭证书》加州大学洛杉矶分校毕业证制作成绩单修改
 
(PARI) Call Girls Wanowrie ( 7001035870 ) HI-Fi Pune Escorts Service
(PARI) Call Girls Wanowrie ( 7001035870 ) HI-Fi Pune Escorts Service(PARI) Call Girls Wanowrie ( 7001035870 ) HI-Fi Pune Escorts Service
(PARI) Call Girls Wanowrie ( 7001035870 ) HI-Fi Pune Escorts Service
 
Low Rate Call Girls Bhilai Anika 8250192130 Independent Escort Service Bhilai
Low Rate Call Girls Bhilai Anika 8250192130 Independent Escort Service BhilaiLow Rate Call Girls Bhilai Anika 8250192130 Independent Escort Service Bhilai
Low Rate Call Girls Bhilai Anika 8250192130 Independent Escort Service Bhilai
 
Call Girls in Defence Colony Delhi 💯Call Us 🔝8264348440🔝
Call Girls in Defence Colony Delhi 💯Call Us 🔝8264348440🔝Call Girls in Defence Colony Delhi 💯Call Us 🔝8264348440🔝
Call Girls in Defence Colony Delhi 💯Call Us 🔝8264348440🔝
 
Schema on read is obsolete. Welcome metaprogramming..pdf
Schema on read is obsolete. Welcome metaprogramming..pdfSchema on read is obsolete. Welcome metaprogramming..pdf
Schema on read is obsolete. Welcome metaprogramming..pdf
 
B2 Creative Industry Response Evaluation.docx
B2 Creative Industry Response Evaluation.docxB2 Creative Industry Response Evaluation.docx
B2 Creative Industry Response Evaluation.docx
 
Customer Service Analytics - Make Sense of All Your Data.pptx
Customer Service Analytics - Make Sense of All Your Data.pptxCustomer Service Analytics - Make Sense of All Your Data.pptx
Customer Service Analytics - Make Sense of All Your Data.pptx
 
100-Concepts-of-AI by Anupama Kate .pptx
100-Concepts-of-AI by Anupama Kate .pptx100-Concepts-of-AI by Anupama Kate .pptx
100-Concepts-of-AI by Anupama Kate .pptx
 
VIP Call Girls Service Miyapur Hyderabad Call +91-8250192130
VIP Call Girls Service Miyapur Hyderabad Call +91-8250192130VIP Call Girls Service Miyapur Hyderabad Call +91-8250192130
VIP Call Girls Service Miyapur Hyderabad Call +91-8250192130
 
20240419 - Measurecamp Amsterdam - SAM.pdf
20240419 - Measurecamp Amsterdam - SAM.pdf20240419 - Measurecamp Amsterdam - SAM.pdf
20240419 - Measurecamp Amsterdam - SAM.pdf
 
Deep Generative Learning for All - The Gen AI Hype (Spring 2024)
Deep Generative Learning for All - The Gen AI Hype (Spring 2024)Deep Generative Learning for All - The Gen AI Hype (Spring 2024)
Deep Generative Learning for All - The Gen AI Hype (Spring 2024)
 
Data Science Project: Advancements in Fetal Health Classification
Data Science Project: Advancements in Fetal Health ClassificationData Science Project: Advancements in Fetal Health Classification
Data Science Project: Advancements in Fetal Health Classification
 
Market Analysis in the 5 Largest Economic Countries in Southeast Asia.pdf
Market Analysis in the 5 Largest Economic Countries in Southeast Asia.pdfMarket Analysis in the 5 Largest Economic Countries in Southeast Asia.pdf
Market Analysis in the 5 Largest Economic Countries in Southeast Asia.pdf
 
Unveiling Insights: The Role of a Data Analyst
Unveiling Insights: The Role of a Data AnalystUnveiling Insights: The Role of a Data Analyst
Unveiling Insights: The Role of a Data Analyst
 

Effective testing for spark programs Strata NY 2015

  • 1. Effective Testing for Spark Programs or avoiding “I didn’t think that could happen” 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 Alpine and previously 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 ● Doing traditional unit testing for Spark programs ○ Along with special considerations for SQL/Hive & Streaming ● Using counters & other job acceptance tests w/ Spark ● Cute & scary pictures ○ I promise at least one panda and one cat ● “Future Work” ○ 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. 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.
  • 7. Cat photo from http://galato901.deviantart.com/art/Cat-on-Work-Break-173043455
  • 8. 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
  • 9. 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
  • 10. Wait, where were the batteries? Photo by Jim Bauer
  • 11. 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
  • 12. 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) } }
  • 13. A simple unit test (Java) public class SampleJavaRDDTest extends SharedJavaSparkContext implements Serializable { @Test public void verifyMapTest() { List<Integer> input = Arrays.asList(1,2); JavaRDD<Integer> result = jsc().parallelize(input).map( new Function<Integer, Integer>() { public Integer call(Integer x) { return x * x;}}); assertEquals(input.size(), result.count()); } }
  • 14. A simple unit test (Python) class SimpleTest(SparkTestingBaseTestCase): """A simple test.""" def test_basic(self): """Test a simple collect.""" input = ["hello world"] rdd = self.sc.parallelize(input) result = rdd.collect() assert result == input
  • 15. Making fake data ● If you have production data you can sample you are lucky! ○ If possible you can try and save in the same format ● sc.parallelize is pretty good for small tests ○ Note: that we can specify the number of partitions ● Coming up with good test data can take a long time Lori Rielly
  • 16. 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 RDDs *I assume PROtara hunt
  • 17. With sscheck def forallRDDGenOfNtoM = { val minWords, maxWords = (50, 100) Prop.forAll(RDDGen.ofNtoM(50, 100, arbitrary[String])) { rdd : RDD[String] => rdd.map(_.length()).sum must be_>=(0.0) } }
  • 18. With spark-testing-base test("map should not change number of elements") { forAll(RDDGenerator.genRDD[String](sc)){ rdd => rdd.map(_.length).count() == rdd.count() } }
  • 20. // 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
  • 21. 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) } } } }
  • 22. 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() } }
  • 23. 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
  • 24. 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
  • 25. What about DataFrames? ● We can do the same as we did for RDD’s ● Inside of Spark validation looks like: def checkAnswer(df: DataFrame, expectedAnswer: Seq[Row]) ● Sadly it’s not in a published package :( def equalDataFrames(expected: DataFrame, result: DataFrame) { def approxEqualDataFrames(e: DataFrame, r: DataFrame, tol: Double) {
  • 26. We can make it easier!* 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.
  • 28. 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 Photo by: Bev Sykes
  • 29. Running on a real* cluster ● 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 is merged & (maybe 1.6 ?) Photo by Richard Masoner
  • 31. Why should we validate our jobs? ● Our code will most likely fail ○ Sometimes data sources fail in new & exciting ways (see Mongo) ○ That jerk on that other floor changed the meaning of a field :( ○ Our tests won’t catch all of the corner cases that the real world finds ● We should try and minimize the impact ○ Avoid making potentially embarrassing recommendations ○ Save having to be woken up at 3am to do a roll-back ○ Specifying a few simple invariants isn’t that hard Photo of GSM intercept by Matt E
  • 32. So how do we validate our jobs? ● Spark has it own counters ○ Per-stage bytes r/w, shuffle r/w, record r/w. execution time, etc. ● We can add counters for things we care about ○ invalid records, users with no recommendations, etc. ● We can write rules for if the values are expected ○ Simple rules (X > J) ○ Historic rules (X > Avg(J)) Photo by: Paul Schadler
  • 33. Simple historic validation Photo by Dvortygirl val vc = new ValidationConf(jobHistoryPath, "1", true, List[ValidationRule](new AvgRule("acc", 0.001, Some(200)))) val v = Validation(sc, vc) // Some job logic // Register an accumulator (optional) val acc = sc.accumulator(0) v.registerAccumulator(acc, "acc") // More Job logic goes here if (v.validate(jobId)) { // Success logic goes here } else sadness()
  • 34. With a Spark internal counter... val vc = new ValidationConf(tempPath, "1", true, List[ValidationRule]( new AbsoluteSparkCounterValidationRule("recordsRead", Some(30), Some (1000))) ) val sqlCtx = new SQLContext(sc) val v = Validation(sc, sqlCtx, vc) //Do work here.... assert(v.validate(5) === true) } Photo by Dvortygirl
  • 35. 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
  • 36. 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/
  • 37. “Future Work” ● Integrating into Apache Spark ○ Using their style rules to simplify future transition ● Better ScalaCheck integration (with the help of the sscheck people) ● Some reasonable prefab rules for Job validation ● Testing details in my next Spark book ● Whatever* you all want ○ Testing with Spark survey: http://bit.ly/holdenTestingSpark Semi-likely: ● integration testing *That I feel like doing, or you feel like making a pull request for. Photo by bullet101
  • 38. Cat wave photo by Quinn Dombrowski k thnx bye! If you want to fill out survey: http://bit. ly/holdenTestingSpark Will tweet results “eventually” @holdenkarau