SlideShare uma empresa Scribd logo
1 de 38
Baixar para ler 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

Mais conteúdo relacionado

Mais procurados

Optimizing Delta/Parquet Data Lakes for Apache Spark
Optimizing Delta/Parquet Data Lakes for Apache SparkOptimizing Delta/Parquet Data Lakes for Apache Spark
Optimizing Delta/Parquet Data Lakes for Apache SparkDatabricks
 
Bighead: Airbnb’s End-to-End Machine Learning Platform with Krishna Puttaswa...
 Bighead: Airbnb’s End-to-End Machine Learning Platform with Krishna Puttaswa... Bighead: Airbnb’s End-to-End Machine Learning Platform with Krishna Puttaswa...
Bighead: Airbnb’s End-to-End Machine Learning Platform with Krishna Puttaswa...Databricks
 
Spark with Delta Lake
Spark with Delta LakeSpark with Delta Lake
Spark with Delta LakeKnoldus Inc.
 
A Deep Dive into Stateful Stream Processing in Structured Streaming with Tath...
A Deep Dive into Stateful Stream Processing in Structured Streaming with Tath...A Deep Dive into Stateful Stream Processing in Structured Streaming with Tath...
A Deep Dive into Stateful Stream Processing in Structured Streaming with Tath...Databricks
 
Introduction to Apache Beam
Introduction to Apache BeamIntroduction to Apache Beam
Introduction to Apache BeamKnoldus Inc.
 
Getting Started with Apache Spark on Kubernetes
Getting Started with Apache Spark on KubernetesGetting Started with Apache Spark on Kubernetes
Getting Started with Apache Spark on KubernetesDatabricks
 
Building Data Quality pipelines with Apache Spark and Delta Lake
Building Data Quality pipelines with Apache Spark and Delta LakeBuilding Data Quality pipelines with Apache Spark and Delta Lake
Building Data Quality pipelines with Apache Spark and Delta LakeDatabricks
 
Web-Scale Graph Analytics with Apache® Spark™
Web-Scale Graph Analytics with Apache® Spark™Web-Scale Graph Analytics with Apache® Spark™
Web-Scale Graph Analytics with Apache® Spark™Databricks
 
Building a MLOps Platform Around MLflow to Enable Model Productionalization i...
Building a MLOps Platform Around MLflow to Enable Model Productionalization i...Building a MLOps Platform Around MLflow to Enable Model Productionalization i...
Building a MLOps Platform Around MLflow to Enable Model Productionalization i...Databricks
 
Scaling up uber's real time data analytics
Scaling up uber's real time data analyticsScaling up uber's real time data analytics
Scaling up uber's real time data analyticsXiang Fu
 
Common issues with Apache Kafka® Producer
Common issues with Apache Kafka® ProducerCommon issues with Apache Kafka® Producer
Common issues with Apache Kafka® Producerconfluent
 
Designing ETL Pipelines with Structured Streaming and Delta Lake—How to Archi...
Designing ETL Pipelines with Structured Streaming and Delta Lake—How to Archi...Designing ETL Pipelines with Structured Streaming and Delta Lake—How to Archi...
Designing ETL Pipelines with Structured Streaming and Delta Lake—How to Archi...Databricks
 
Data Science for Dummies - Data Engineering with Titanic dataset + Databricks...
Data Science for Dummies - Data Engineering with Titanic dataset + Databricks...Data Science for Dummies - Data Engineering with Titanic dataset + Databricks...
Data Science for Dummies - Data Engineering with Titanic dataset + Databricks...Rodney Joyce
 
Google Cloud Dataproc - Easier, faster, more cost-effective Spark and Hadoop
Google Cloud Dataproc - Easier, faster, more cost-effective Spark and HadoopGoogle Cloud Dataproc - Easier, faster, more cost-effective Spark and Hadoop
Google Cloud Dataproc - Easier, faster, more cost-effective Spark and Hadoophuguk
 
Data engineering in 10 years.pdf
Data engineering in 10 years.pdfData engineering in 10 years.pdf
Data engineering in 10 years.pdfLars Albertsson
 
Google Cloud Composer
Google Cloud ComposerGoogle Cloud Composer
Google Cloud ComposerPierre Coste
 
Data pipelines from zero to solid
Data pipelines from zero to solidData pipelines from zero to solid
Data pipelines from zero to solidLars Albertsson
 
Airflow presentation
Airflow presentationAirflow presentation
Airflow presentationIlias Okacha
 
Making Apache Spark Better with Delta Lake
Making Apache Spark Better with Delta LakeMaking Apache Spark Better with Delta Lake
Making Apache Spark Better with Delta LakeDatabricks
 

Mais procurados (20)

Optimizing Delta/Parquet Data Lakes for Apache Spark
Optimizing Delta/Parquet Data Lakes for Apache SparkOptimizing Delta/Parquet Data Lakes for Apache Spark
Optimizing Delta/Parquet Data Lakes for Apache Spark
 
Bighead: Airbnb’s End-to-End Machine Learning Platform with Krishna Puttaswa...
 Bighead: Airbnb’s End-to-End Machine Learning Platform with Krishna Puttaswa... Bighead: Airbnb’s End-to-End Machine Learning Platform with Krishna Puttaswa...
Bighead: Airbnb’s End-to-End Machine Learning Platform with Krishna Puttaswa...
 
Spark with Delta Lake
Spark with Delta LakeSpark with Delta Lake
Spark with Delta Lake
 
A Deep Dive into Stateful Stream Processing in Structured Streaming with Tath...
A Deep Dive into Stateful Stream Processing in Structured Streaming with Tath...A Deep Dive into Stateful Stream Processing in Structured Streaming with Tath...
A Deep Dive into Stateful Stream Processing in Structured Streaming with Tath...
 
Introduction to Apache Beam
Introduction to Apache BeamIntroduction to Apache Beam
Introduction to Apache Beam
 
Getting Started with Apache Spark on Kubernetes
Getting Started with Apache Spark on KubernetesGetting Started with Apache Spark on Kubernetes
Getting Started with Apache Spark on Kubernetes
 
Building Data Quality pipelines with Apache Spark and Delta Lake
Building Data Quality pipelines with Apache Spark and Delta LakeBuilding Data Quality pipelines with Apache Spark and Delta Lake
Building Data Quality pipelines with Apache Spark and Delta Lake
 
Web-Scale Graph Analytics with Apache® Spark™
Web-Scale Graph Analytics with Apache® Spark™Web-Scale Graph Analytics with Apache® Spark™
Web-Scale Graph Analytics with Apache® Spark™
 
Apache Spark Crash Course
Apache Spark Crash CourseApache Spark Crash Course
Apache Spark Crash Course
 
Building a MLOps Platform Around MLflow to Enable Model Productionalization i...
Building a MLOps Platform Around MLflow to Enable Model Productionalization i...Building a MLOps Platform Around MLflow to Enable Model Productionalization i...
Building a MLOps Platform Around MLflow to Enable Model Productionalization i...
 
Scaling up uber's real time data analytics
Scaling up uber's real time data analyticsScaling up uber's real time data analytics
Scaling up uber's real time data analytics
 
Common issues with Apache Kafka® Producer
Common issues with Apache Kafka® ProducerCommon issues with Apache Kafka® Producer
Common issues with Apache Kafka® Producer
 
Designing ETL Pipelines with Structured Streaming and Delta Lake—How to Archi...
Designing ETL Pipelines with Structured Streaming and Delta Lake—How to Archi...Designing ETL Pipelines with Structured Streaming and Delta Lake—How to Archi...
Designing ETL Pipelines with Structured Streaming and Delta Lake—How to Archi...
 
Data Science for Dummies - Data Engineering with Titanic dataset + Databricks...
Data Science for Dummies - Data Engineering with Titanic dataset + Databricks...Data Science for Dummies - Data Engineering with Titanic dataset + Databricks...
Data Science for Dummies - Data Engineering with Titanic dataset + Databricks...
 
Google Cloud Dataproc - Easier, faster, more cost-effective Spark and Hadoop
Google Cloud Dataproc - Easier, faster, more cost-effective Spark and HadoopGoogle Cloud Dataproc - Easier, faster, more cost-effective Spark and Hadoop
Google Cloud Dataproc - Easier, faster, more cost-effective Spark and Hadoop
 
Data engineering in 10 years.pdf
Data engineering in 10 years.pdfData engineering in 10 years.pdf
Data engineering in 10 years.pdf
 
Google Cloud Composer
Google Cloud ComposerGoogle Cloud Composer
Google Cloud Composer
 
Data pipelines from zero to solid
Data pipelines from zero to solidData pipelines from zero to solid
Data pipelines from zero to solid
 
Airflow presentation
Airflow presentationAirflow presentation
Airflow presentation
 
Making Apache Spark Better with Delta Lake
Making Apache Spark Better with Delta LakeMaking Apache Spark Better with Delta Lake
Making Apache Spark Better with Delta Lake
 

Semelhante a 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
 

Semelhante a 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
 

Último

Dubai Call Girls Peeing O525547819 Call Girls Dubai
Dubai Call Girls Peeing O525547819 Call Girls DubaiDubai Call Girls Peeing O525547819 Call Girls Dubai
Dubai Call Girls Peeing O525547819 Call Girls Dubaikojalkojal131
 
Top profile Call Girls In Indore [ 7014168258 ] Call Me For Genuine Models We...
Top profile Call Girls In Indore [ 7014168258 ] Call Me For Genuine Models We...Top profile Call Girls In Indore [ 7014168258 ] Call Me For Genuine Models We...
Top profile Call Girls In Indore [ 7014168258 ] Call Me For Genuine Models We...gajnagarg
 
Top profile Call Girls In bhavnagar [ 7014168258 ] Call Me For Genuine Models...
Top profile Call Girls In bhavnagar [ 7014168258 ] Call Me For Genuine Models...Top profile Call Girls In bhavnagar [ 7014168258 ] Call Me For Genuine Models...
Top profile Call Girls In bhavnagar [ 7014168258 ] Call Me For Genuine Models...gajnagarg
 
20240412-SmartCityIndex-2024-Full-Report.pdf
20240412-SmartCityIndex-2024-Full-Report.pdf20240412-SmartCityIndex-2024-Full-Report.pdf
20240412-SmartCityIndex-2024-Full-Report.pdfkhraisr
 
Top profile Call Girls In Purnia [ 7014168258 ] Call Me For Genuine Models We...
Top profile Call Girls In Purnia [ 7014168258 ] Call Me For Genuine Models We...Top profile Call Girls In Purnia [ 7014168258 ] Call Me For Genuine Models We...
Top profile Call Girls In Purnia [ 7014168258 ] Call Me For Genuine Models We...nirzagarg
 
如何办理英国诺森比亚大学毕业证(NU毕业证书)成绩单原件一模一样
如何办理英国诺森比亚大学毕业证(NU毕业证书)成绩单原件一模一样如何办理英国诺森比亚大学毕业证(NU毕业证书)成绩单原件一模一样
如何办理英国诺森比亚大学毕业证(NU毕业证书)成绩单原件一模一样wsppdmt
 
Top profile Call Girls In Begusarai [ 7014168258 ] Call Me For Genuine Models...
Top profile Call Girls In Begusarai [ 7014168258 ] Call Me For Genuine Models...Top profile Call Girls In Begusarai [ 7014168258 ] Call Me For Genuine Models...
Top profile Call Girls In Begusarai [ 7014168258 ] Call Me For Genuine Models...nirzagarg
 
Jual Obat Aborsi Surabaya ( Asli No.1 ) 085657271886 Obat Penggugur Kandungan...
Jual Obat Aborsi Surabaya ( Asli No.1 ) 085657271886 Obat Penggugur Kandungan...Jual Obat Aborsi Surabaya ( Asli No.1 ) 085657271886 Obat Penggugur Kandungan...
Jual Obat Aborsi Surabaya ( Asli No.1 ) 085657271886 Obat Penggugur Kandungan...ZurliaSoop
 
Vadodara 💋 Call Girl 7737669865 Call Girls in Vadodara Escort service book now
Vadodara 💋 Call Girl 7737669865 Call Girls in Vadodara Escort service book nowVadodara 💋 Call Girl 7737669865 Call Girls in Vadodara Escort service book now
Vadodara 💋 Call Girl 7737669865 Call Girls in Vadodara Escort service book nowgargpaaro
 
Top Call Girls in Balaghat 9332606886Call Girls Advance Cash On Delivery Ser...
Top Call Girls in Balaghat  9332606886Call Girls Advance Cash On Delivery Ser...Top Call Girls in Balaghat  9332606886Call Girls Advance Cash On Delivery Ser...
Top Call Girls in Balaghat 9332606886Call Girls Advance Cash On Delivery Ser...kumargunjan9515
 
Sonagachi * best call girls in Kolkata | ₹,9500 Pay Cash 8005736733 Free Home...
Sonagachi * best call girls in Kolkata | ₹,9500 Pay Cash 8005736733 Free Home...Sonagachi * best call girls in Kolkata | ₹,9500 Pay Cash 8005736733 Free Home...
Sonagachi * best call girls in Kolkata | ₹,9500 Pay Cash 8005736733 Free Home...HyderabadDolls
 
7. Epi of Chronic respiratory diseases.ppt
7. Epi of Chronic respiratory diseases.ppt7. Epi of Chronic respiratory diseases.ppt
7. Epi of Chronic respiratory diseases.pptibrahimabdi22
 
Sealdah % High Class Call Girls Kolkata - 450+ Call Girl Cash Payment 8005736...
Sealdah % High Class Call Girls Kolkata - 450+ Call Girl Cash Payment 8005736...Sealdah % High Class Call Girls Kolkata - 450+ Call Girl Cash Payment 8005736...
Sealdah % High Class Call Girls Kolkata - 450+ Call Girl Cash Payment 8005736...HyderabadDolls
 
Discover Why Less is More in B2B Research
Discover Why Less is More in B2B ResearchDiscover Why Less is More in B2B Research
Discover Why Less is More in B2B Researchmichael115558
 
Ranking and Scoring Exercises for Research
Ranking and Scoring Exercises for ResearchRanking and Scoring Exercises for Research
Ranking and Scoring Exercises for ResearchRajesh Mondal
 
Nirala Nagar / Cheap Call Girls In Lucknow Phone No 9548273370 Elite Escort S...
Nirala Nagar / Cheap Call Girls In Lucknow Phone No 9548273370 Elite Escort S...Nirala Nagar / Cheap Call Girls In Lucknow Phone No 9548273370 Elite Escort S...
Nirala Nagar / Cheap Call Girls In Lucknow Phone No 9548273370 Elite Escort S...HyderabadDolls
 
Fun all Day Call Girls in Jaipur 9332606886 High Profile Call Girls You Ca...
Fun all Day Call Girls in Jaipur   9332606886  High Profile Call Girls You Ca...Fun all Day Call Girls in Jaipur   9332606886  High Profile Call Girls You Ca...
Fun all Day Call Girls in Jaipur 9332606886 High Profile Call Girls You Ca...kumargunjan9515
 
Gulbai Tekra * Cheap Call Girls In Ahmedabad Phone No 8005736733 Elite Escort...
Gulbai Tekra * Cheap Call Girls In Ahmedabad Phone No 8005736733 Elite Escort...Gulbai Tekra * Cheap Call Girls In Ahmedabad Phone No 8005736733 Elite Escort...
Gulbai Tekra * Cheap Call Girls In Ahmedabad Phone No 8005736733 Elite Escort...gragchanchal546
 
Top profile Call Girls In Satna [ 7014168258 ] Call Me For Genuine Models We ...
Top profile Call Girls In Satna [ 7014168258 ] Call Me For Genuine Models We ...Top profile Call Girls In Satna [ 7014168258 ] Call Me For Genuine Models We ...
Top profile Call Girls In Satna [ 7014168258 ] Call Me For Genuine Models We ...nirzagarg
 
Top profile Call Girls In Latur [ 7014168258 ] Call Me For Genuine Models We ...
Top profile Call Girls In Latur [ 7014168258 ] Call Me For Genuine Models We ...Top profile Call Girls In Latur [ 7014168258 ] Call Me For Genuine Models We ...
Top profile Call Girls In Latur [ 7014168258 ] Call Me For Genuine Models We ...gajnagarg
 

Último (20)

Dubai Call Girls Peeing O525547819 Call Girls Dubai
Dubai Call Girls Peeing O525547819 Call Girls DubaiDubai Call Girls Peeing O525547819 Call Girls Dubai
Dubai Call Girls Peeing O525547819 Call Girls Dubai
 
Top profile Call Girls In Indore [ 7014168258 ] Call Me For Genuine Models We...
Top profile Call Girls In Indore [ 7014168258 ] Call Me For Genuine Models We...Top profile Call Girls In Indore [ 7014168258 ] Call Me For Genuine Models We...
Top profile Call Girls In Indore [ 7014168258 ] Call Me For Genuine Models We...
 
Top profile Call Girls In bhavnagar [ 7014168258 ] Call Me For Genuine Models...
Top profile Call Girls In bhavnagar [ 7014168258 ] Call Me For Genuine Models...Top profile Call Girls In bhavnagar [ 7014168258 ] Call Me For Genuine Models...
Top profile Call Girls In bhavnagar [ 7014168258 ] Call Me For Genuine Models...
 
20240412-SmartCityIndex-2024-Full-Report.pdf
20240412-SmartCityIndex-2024-Full-Report.pdf20240412-SmartCityIndex-2024-Full-Report.pdf
20240412-SmartCityIndex-2024-Full-Report.pdf
 
Top profile Call Girls In Purnia [ 7014168258 ] Call Me For Genuine Models We...
Top profile Call Girls In Purnia [ 7014168258 ] Call Me For Genuine Models We...Top profile Call Girls In Purnia [ 7014168258 ] Call Me For Genuine Models We...
Top profile Call Girls In Purnia [ 7014168258 ] Call Me For Genuine Models We...
 
如何办理英国诺森比亚大学毕业证(NU毕业证书)成绩单原件一模一样
如何办理英国诺森比亚大学毕业证(NU毕业证书)成绩单原件一模一样如何办理英国诺森比亚大学毕业证(NU毕业证书)成绩单原件一模一样
如何办理英国诺森比亚大学毕业证(NU毕业证书)成绩单原件一模一样
 
Top profile Call Girls In Begusarai [ 7014168258 ] Call Me For Genuine Models...
Top profile Call Girls In Begusarai [ 7014168258 ] Call Me For Genuine Models...Top profile Call Girls In Begusarai [ 7014168258 ] Call Me For Genuine Models...
Top profile Call Girls In Begusarai [ 7014168258 ] Call Me For Genuine Models...
 
Jual Obat Aborsi Surabaya ( Asli No.1 ) 085657271886 Obat Penggugur Kandungan...
Jual Obat Aborsi Surabaya ( Asli No.1 ) 085657271886 Obat Penggugur Kandungan...Jual Obat Aborsi Surabaya ( Asli No.1 ) 085657271886 Obat Penggugur Kandungan...
Jual Obat Aborsi Surabaya ( Asli No.1 ) 085657271886 Obat Penggugur Kandungan...
 
Vadodara 💋 Call Girl 7737669865 Call Girls in Vadodara Escort service book now
Vadodara 💋 Call Girl 7737669865 Call Girls in Vadodara Escort service book nowVadodara 💋 Call Girl 7737669865 Call Girls in Vadodara Escort service book now
Vadodara 💋 Call Girl 7737669865 Call Girls in Vadodara Escort service book now
 
Top Call Girls in Balaghat 9332606886Call Girls Advance Cash On Delivery Ser...
Top Call Girls in Balaghat  9332606886Call Girls Advance Cash On Delivery Ser...Top Call Girls in Balaghat  9332606886Call Girls Advance Cash On Delivery Ser...
Top Call Girls in Balaghat 9332606886Call Girls Advance Cash On Delivery Ser...
 
Sonagachi * best call girls in Kolkata | ₹,9500 Pay Cash 8005736733 Free Home...
Sonagachi * best call girls in Kolkata | ₹,9500 Pay Cash 8005736733 Free Home...Sonagachi * best call girls in Kolkata | ₹,9500 Pay Cash 8005736733 Free Home...
Sonagachi * best call girls in Kolkata | ₹,9500 Pay Cash 8005736733 Free Home...
 
7. Epi of Chronic respiratory diseases.ppt
7. Epi of Chronic respiratory diseases.ppt7. Epi of Chronic respiratory diseases.ppt
7. Epi of Chronic respiratory diseases.ppt
 
Sealdah % High Class Call Girls Kolkata - 450+ Call Girl Cash Payment 8005736...
Sealdah % High Class Call Girls Kolkata - 450+ Call Girl Cash Payment 8005736...Sealdah % High Class Call Girls Kolkata - 450+ Call Girl Cash Payment 8005736...
Sealdah % High Class Call Girls Kolkata - 450+ Call Girl Cash Payment 8005736...
 
Discover Why Less is More in B2B Research
Discover Why Less is More in B2B ResearchDiscover Why Less is More in B2B Research
Discover Why Less is More in B2B Research
 
Ranking and Scoring Exercises for Research
Ranking and Scoring Exercises for ResearchRanking and Scoring Exercises for Research
Ranking and Scoring Exercises for Research
 
Nirala Nagar / Cheap Call Girls In Lucknow Phone No 9548273370 Elite Escort S...
Nirala Nagar / Cheap Call Girls In Lucknow Phone No 9548273370 Elite Escort S...Nirala Nagar / Cheap Call Girls In Lucknow Phone No 9548273370 Elite Escort S...
Nirala Nagar / Cheap Call Girls In Lucknow Phone No 9548273370 Elite Escort S...
 
Fun all Day Call Girls in Jaipur 9332606886 High Profile Call Girls You Ca...
Fun all Day Call Girls in Jaipur   9332606886  High Profile Call Girls You Ca...Fun all Day Call Girls in Jaipur   9332606886  High Profile Call Girls You Ca...
Fun all Day Call Girls in Jaipur 9332606886 High Profile Call Girls You Ca...
 
Gulbai Tekra * Cheap Call Girls In Ahmedabad Phone No 8005736733 Elite Escort...
Gulbai Tekra * Cheap Call Girls In Ahmedabad Phone No 8005736733 Elite Escort...Gulbai Tekra * Cheap Call Girls In Ahmedabad Phone No 8005736733 Elite Escort...
Gulbai Tekra * Cheap Call Girls In Ahmedabad Phone No 8005736733 Elite Escort...
 
Top profile Call Girls In Satna [ 7014168258 ] Call Me For Genuine Models We ...
Top profile Call Girls In Satna [ 7014168258 ] Call Me For Genuine Models We ...Top profile Call Girls In Satna [ 7014168258 ] Call Me For Genuine Models We ...
Top profile Call Girls In Satna [ 7014168258 ] Call Me For Genuine Models We ...
 
Top profile Call Girls In Latur [ 7014168258 ] Call Me For Genuine Models We ...
Top profile Call Girls In Latur [ 7014168258 ] Call Me For Genuine Models We ...Top profile Call Girls In Latur [ 7014168258 ] Call Me For Genuine Models We ...
Top profile Call Girls In Latur [ 7014168258 ] Call Me For Genuine Models We ...
 

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