Testing and validating spark programs - Strata SJ 2016

50
Testing and validating Spark programs Now mostly “works”* *See developer for details. Does not imply warranty. :p Does not apply to libraries

Transcript of Testing and validating spark programs - Strata SJ 2016

Page 1: Testing and validating spark programs - Strata SJ 2016

Testing and validatingSpark programs

Now mostly

“works”*

*See developer for details. Does not imply warranty. :p Does not apply to libraries

Page 2: Testing and validating spark programs - Strata SJ 2016

Who am I?● My name is Holden Karau● Prefered pronouns are she/her● I’m a Software Engineer● currently IBM and previously Alpine, Databricks, Google, Foursquare &

Amazon● co-author of Learning Spark & Fast Data processing with Spark● @holdenkarau● Slide share http://www.slideshare.net/hkarau ● Linkedin https://www.linkedin.com/in/holdenkarau ● Spark Videos http://bit.ly/holdenSparkVideos

Page 3: Testing and validating spark programs - Strata SJ 2016

What is going to be covered:● What I think I might know about you● A bit about why you should test & validate your programs● “Normal” unit testing● Testing at scale(ish)● Considerations for Streaming & SQL (DataFrames & Datasets)● Validation - how to make simple validation rules & our current limitations● Cute & scary pictures

○ I promise at least one panda and one cat○ Yes they are mostly the same pandas & cats as always - try and spot the new ones :p

Andrew

Page 4: Testing and validating spark programs - Strata SJ 2016

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)

Page 5: Testing and validating spark programs - Strata SJ 2016

So why should you test?● Makes you a better person● Avoid making your users angry● Save $s

○ AWS is expensive○ I’ve heard raising money is hard right now, test locally for savings

● 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

Page 6: Testing and validating spark programs - Strata SJ 2016

So why should you validate?● You want to know when you're aboard the failboat● Halt deployment, roll-back● 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 all that hard

Page 7: Testing and validating spark programs - Strata SJ 2016

So why should you test & validate:

Results from: Testing with Spark survey http://bit.ly/holdenTestingSpark

Page 8: Testing and validating spark programs - Strata SJ 2016

So why should you test & validate - cont

Results from: Testing with Spark survey http://bit.ly/holdenTestingSpark

Page 9: Testing and validating spark programs - Strata SJ 2016

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.

Page 10: Testing and validating spark programs - Strata SJ 2016

Why don’t we validate?● We already tested our code● What could go wrong?

Page 11: Testing and validating spark programs - Strata SJ 2016

Cat photo from http://galato901.deviantart.com/art/Cat-on-Work-Break-173043455

Page 12: Testing and validating spark programs - Strata SJ 2016

Testing libraries:● 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

Page 13: Testing and validating spark programs - Strata SJ 2016

A simple unit test with spark-testing-baseclass 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) }}

Page 14: Testing and validating spark programs - Strata SJ 2016

Ok but what about problems @ scale● Maybe our program works fine on our local sized input● If we are using Spark our actual workload is probably huge● How do we test workloads too large for a single machine?

○ we can’t just use parallelize and collect

Qfamily

Page 15: Testing and validating spark programs - Strata SJ 2016

Distributed “set” operations to the rescue*● Pretty close - already built into Spark● Doesn’t do so well with floating points :(

○ damn floating points keep showing up everywhere :p

● Doesn’t really handle duplicates very well ○ {“coffee”, “coffee”, “panda”} != {“panda”, “coffee”} but with set operations...

Matti Mattila

Page 16: Testing and validating spark programs - Strata SJ 2016

Or use RDDComparisions: def compareWithOrderSamePartitioner[T: ClassTag](expected: RDD

[T], result: RDD[T]): Option[(T, T)] = {

expected.zip(result).filter{case (x, y) => x != y}.take(1).

headOption

}

Matti Mattila

Page 17: Testing and validating spark programs - Strata SJ 2016

Or use RDDComparisions:def compare[T: ClassTag](expected: RDD[T], result: RDD[T]): Option

[(T, Int, Int)] = {

val expectedKeyed = expected.map(x => (x, 1)).reduceByKey(_ +

_)

val resultKeyed = result.map(x => (x, 1)).reduceByKey(_ + _)

expectedKeyed.cogroup(resultKeyed).filter{case (_, (i1, i2))

=>

i1.isEmpty || i2.isEmpty || i1.head != i2.head}.take(1).

headOption.

map{case (v, (i1, i2)) => (v, i1.headOption.getOrElse(0),

i2.headOption.getOrElse(0))}

}

Matti Mattila

Page 18: Testing and validating spark programs - Strata SJ 2016

But where do we get the data for those tests?● If you have production data you can sample you are lucky!

○ If possible you can try and save in the same format

● If our data is a bunch of Vectors or Doubles Spark’s got tools :)● Coming up with good test data can take a long time

Lori Rielly

Page 19: Testing and validating spark programs - Strata SJ 2016

QuickCheck / ScalaCheck● QuickCheck generates tests data under a set of constraints● Scala version is ScalaCheck - supported by the two unit testing libraries for

Spark● sscheck

○ Awesome people*, supports generating DStreams too!

● spark-testing-base○ Also Awesome people*, generates more pathological (e.g. empty partitions etc.) RDDs

*I assume

PROtara hunt

Page 20: Testing and validating spark programs - Strata SJ 2016

With spark-testing-basetest("map should not change number of elements") { forAll(RDDGenerator.genRDD[String](sc)){ rdd => rdd.map(_.length).count() == rdd.count() }}

Page 21: Testing and validating spark programs - Strata SJ 2016

With spark-testing-base & a million entriestest("map should not change number of elements") { implicit val generatorDrivenConfig = PropertyCheckConfig(minSize = 0, maxSize = 1000000) forAll(RDDGenerator.genRDD[String](sc)){ rdd => rdd.map(_.length).count() == rdd.count() }}

Page 22: Testing and validating spark programs - Strata SJ 2016

Testing streaming….

Photo by Steve Jurvetson

Page 23: Testing and validating spark programs - Strata SJ 2016

// 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 Codetrait 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() }

Photo by Steve Jurvetson

Page 24: Testing and validating spark programs - Strata SJ 2016

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) } } }

}

Page 25: Testing and validating spark programs - Strata SJ 2016

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() }

}

Page 26: Testing and validating spark programs - Strata SJ 2016

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 ugly○ foreachRDD & a var

● figuring out when your test is “done”

Let’s abstract all that away into testOperation

Page 27: Testing and validating spark programs - Strata SJ 2016

A simple (non-scalable) stream test: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

Page 28: Testing and validating spark programs - Strata SJ 2016

What about DataFrames?● We can do the same as we did for RDD’s (.rdd)● Inside of Spark validation looks like:

def checkAnswer(df: DataFrame, expectedAnswer: Seq[Row])

● Sadly it’s not in a published package & local only● instead we expose:

def equalDataFrames(expected: DataFrame, result: DataFrame) {def approxEqualDataFrames(e: DataFrame, r: DataFrame, tol: Double) {

yoppy

Page 29: Testing and validating spark programs - Strata SJ 2016

…. and Datasets● We can do the same as we did for RDD’s (.rdd)● Inside of Spark validation looks like:

def checkAnswer(df: Dataset[T], expectedAnswer: T*)

● Sadly it’s not in a published package & local only● instead we expose:

def equalDatasets(expected: Dataset[U], result: Dataset[V]) {def approxEqualDatasets(e: Dataset[U], r: Dataset[V], tol: Double) {

Page 30: Testing and validating spark programs - Strata SJ 2016

This is what it looks like: test("dataframe should be equal to its self") { val sqlCtx = sqlContext import sqlCtx.implicits._// Yah I know this is ugly val input = sc.parallelize(inputList).toDF equalDataFrames(input, input) }

*This may or may not be easier.

Page 31: Testing and validating spark programs - Strata SJ 2016

Or with a generator based on Schema*: test("assert rows' types like schema type") {

val schema = StructType(List(StructField("name", StringType)))

val rowGen: Gen[Row] = DataframeGenerator.getRowGenerator

(schema)

val property =

forAll(rowGen) {

row => row.get(0).isInstanceOf[String]

}

check(property)

}

*For simple schemas, complex types in future versions

thelittleone417

Page 32: Testing and validating spark programs - Strata SJ 2016

Which has “built-in” large support :)

Page 33: Testing and validating spark programs - Strata SJ 2016

Other options for generating data:● mapPartitions + Random + custom code● RandomRDDs in mllib

○ Uniform, Normal, Possion, Exponential, Gamma, logNormal & Vector versions○ Different type: implement the RandomDataGenerator interface

Page 34: Testing and validating spark programs - Strata SJ 2016

RandomRDDsval zipRDD = RandomRDDs.exponentialRDD(sc, mean = 1000, size =

rows).map(_.toInt.toString)

val valuesRDD = RandomRDDs.normalVectorRDD(sc, numRows = rows,

numCols = numCols).repartition(zipRDD.partitions.size)

val keyRDD = sc.parallelize(1L.to(rows), zipRDD.getNumPartitions)

keyRDD.zipPartitions(zipRDD, valuesRDD){

(i1, i2, i3) =>

new Iterator[(Long, String, Vector)] {

...

Page 35: Testing and validating spark programs - Strata SJ 2016

Photo by allison

Page 36: Testing and validating spark programs - Strata SJ 2016

Let’s talk about local mode● It’s way better than you would expect*● It does its best to try and catch serialization errors● It’s still not the same as running on a “real” cluster● Especially since if we were just local mode, parallelize and collect might be

fine

Photo by: Bev Sykes

Page 37: Testing and validating spark programs - Strata SJ 2016

Options beyond local mode:● Just point at your existing cluster (set master)● Start one with your shell scripts & change the master

○ Really easy way to plug into existing integration testing

● spark-docker - hack in our own tests● YarnMiniCluster

○ https://github.

com/apache/spark/blob/master/yarn/src/test/scala/org/apache/spark/deploy/yarn/BaseYarnClusterSuite.scala

○ In Spark Testing Base extend SharedMiniCluster■ Not recommended until after SPARK-10812 (e.g. 1.5.2+ or 1.6+)

Photo by Richard Masoner

Page 38: Testing and validating spark programs - Strata SJ 2016

On to validation

Page 39: Testing and validating spark programs - Strata SJ 2016

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.

○ Accumulators have some challenges (see SPARK-12469 for progress) but are an interesting option

● We can write rules for if the values are expected○ Simple rules (X > J)

■ The number of records should be greater than 0○ Historic rules (X > Avg(last(10, J)))

■ We need to keep track of our previous values - but this can be great for debugging & performance investigation too.

Photo by:Paul Schadler

Page 40: Testing and validating spark programs - Strata SJ 2016

Validation● For now checking file sizes & execution time seem like the most common best

practice (from survey)● spark-validator is still in early stages and not ready for production use but

interesting proof of concept● Doesn’t need to be done in your Spark job (can be done in your scripting

language of choice with whatever job control system you are using)● Sometimes your rules will miss-fire and you’ll need to manually approve a job

- that is ok!

Photo by:Paul Schadler

Page 41: Testing and validating spark programs - Strata SJ 2016

Validating records read matches our expectations: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

Page 43: Testing and validating spark programs - Strata SJ 2016

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/

Becky Lai

Page 44: Testing and validating spark programs - Strata SJ 2016

And including spark-testing-base:sbt:

"com.holdenkarau" %% "spark-testing-base" % "1.6.1_0.3.2"

maven:

<dependency>

<groupId>com.holdenkarau</groupId>

<artifactId>spark-testing-base</artifactId>

<version>${spark.version}_0.3.2</version>

<scope>test</scope>

</dependency>

Vladimir Pustovit

Page 45: Testing and validating spark programs - Strata SJ 2016

“Future Work”● Better ScalaCheck integration (ala sscheck)● Testing details in my next Spark book● Whatever* you all want

○ Testing with Spark survey: http://bit.ly/holdenTestingSpark

Semi-likely:

● integration testing (for now see @cfriegly’s Spark + Docker setup):○ https://github.com/fluxcapacitor/pipeline

Pretty unlikely:

● Integrating into Apache Spark ( SPARK-12433 )*That I feel like doing, or you feel like making a pull request for.

Photo by bullet101

Page 48: Testing and validating spark programs - Strata SJ 2016

And the next book…..

First four chapters are available in “Early Release”*:● Buy from O’Reilly - http://bit.ly/highPerfSparkGet notified when updated & finished:● http://www.highperformancespark.com ● https://twitter.com/highperfspark

* Early Release means extra mistakes, but also a chance to help us make a more awesome book. Also “should” means we were a bit late with the deadline for the conference so maybe a few days after...

Page 49: Testing and validating spark programs - Strata SJ 2016

And some upcoming talks & office hours

● Office hours tomorrow at 1:50 PM● April

○ Local workshops in the city (Spark SQL) & south bay (Intro to Spark)○ More office hours (TBD)

● May○ Apache Con Big Data (Vancouver)

● June○ Spark Summit SF & Strata London○ Datapalooza Tokyo & Scala Days Berlin

● July○ Data Day Seattle

Page 50: Testing and validating spark programs - Strata SJ 2016

Cat wave photo by Quinn Dombrowski

k thnx bye!

If you want to fill out survey: http://bit.ly/holdenTestingSpark

Will use update results in & tweet eventually at @holdenkarau

Office hours:Tomorrow @ O’Reilly booth 1:50pm