Skip to content

Commit

Permalink
SPARK-25299: add CI infrastructure and SortShuffleWriterBenchmark (ap…
Browse files Browse the repository at this point in the history
…ache-spark-on-k8s#498)

* add initial bypass merge sort shuffle writer benchmarks

* dd unsafe shuffle writer benchmarks

* changes in bypassmergesort benchmarks

* cleanup

* add circle script

* add this branch for testing

* fix circle attempt 1

* checkout code

* add some caches?

* why is it not pull caches...

* save as artifact instead of publishing

* mkdir

* typo

* try uploading artifacts again

* try print per iteration to avoid circle erroring out on idle

* blah (apache-spark-on-k8s#495)

* make a PR comment

* actually delete files

* run benchmarks on test build branch

* oops forgot to enable upload

* add sort shuffle writer benchmarks

* add stdev

* cleanup sort a bit

* fix stdev text

* fix sort shuffle

* initial code for read side

* format

* use times and sample stdev

* add assert for at least one iteration

* cleanup shuffle write to use fewer mocks and single base interface

* shuffle read works with transport client... needs lots of cleaning

* test running in cicle

* scalastyle

* dont publish results yet

* cleanup writer code

* get only git message

* fix command to get PR number

* add SortshuffleWriterBenchmark

* writer code

* cleanup

* fix benchmark script

* use ArgumentMatchers

* also in shufflewriterbenchmarkbase

* scalastyle

* add apache license

* fix some scale stuff

* fix up tests

* only copy benchmarks we care about

* increase size for reader again

* delete two writers and reader for PR

* SPARK-25299: Add shuffle reader benchmarks (apache-spark-on-k8s#506)

* Revert "SPARK-25299: Add shuffle reader benchmarks (apache-spark-on-k8s#506)"

This reverts commit 9d46fae.

* add -e to bash script

* blah

* enable upload as a PR comment and prevent running benchmarks on this branch

* Revert "enable upload as a PR comment and prevent running benchmarks on this branch"

This reverts commit 13703fa.

* try machine execution

* try uploading benchmarks (apache-spark-on-k8s#498)

* only upload results when merging into the feature branch

* lock down machine image

* don't write input data to disk

* run benchmark test

* stop creating file cleanup threads for every block manager

* use alphanumeric again

* use a new random everytime

* close the writers -__________-

* delete branch and publish results as comment

* close in finally
  • Loading branch information
yifeih authored and mccheah committed Mar 14, 2019
1 parent 4bf2a75 commit 11a37ed
Show file tree
Hide file tree
Showing 5 changed files with 462 additions and 6 deletions.
32 changes: 32 additions & 0 deletions .circleci/config.yml
Original file line number Diff line number Diff line change
Expand Up @@ -8,6 +8,12 @@ defaults: &defaults
TERM: dumb
BUILD_SBT_CACHE: "/home/circleci/build-sbt-cache"

spark-25299-config: &spark-25299-config
machine:
image: circleci/classic:201808-01
environment: &defaults-environment
TERM: dumb
BUILD_SBT_CACHE: "/home/circleci/build-sbt-cache"

test-defaults: &test-defaults
<<: *defaults
Expand All @@ -23,6 +29,12 @@ all-branches-and-tags: &all-branches-and-tags
tags:
only: /.*/

spark-25299-branch-only: &spark-25299-branch-only
filters:
branches:
only:
- spark-25299

deployable-branches-and-tags: &deployable-branches-and-tags
filters:
tags:
Expand Down Expand Up @@ -452,6 +464,22 @@ jobs:
key: v1-maven-dependency-cache-versioned-{{ checksum "pom.xml" }}
paths: ~/.m2

run-spark-25299-benchmarks:
<<: *spark-25299-config
steps:
- *checkout-code
- attach_workspace:
at: .
- *restore-build-sbt-cache
- *link-in-build-sbt-cache
- *restore-ivy-cache
- *restore-build-binaries-cache
- *restore-home-sbt-cache
- run:
command: ./dev/run-spark-25299-benchmarks.sh -u
- store_artifacts:
path: /tmp/artifacts/

deploy-gradle:
<<: *defaults
docker:
Expand Down Expand Up @@ -512,6 +540,10 @@ workflows:
requires:
- build-sbt
<<: *all-branches-and-tags
- run-spark-25299-benchmarks:
requires:
- build-sbt
<<: *spark-25299-branch-only
- run-scala-tests:
requires:
- build-sbt
Expand Down
18 changes: 12 additions & 6 deletions core/src/test/scala/org/apache/spark/benchmark/Benchmark.scala
Original file line number Diff line number Diff line change
Expand Up @@ -111,13 +111,15 @@ private[spark] class Benchmark(
// The results are going to be processor specific so it is useful to include that.
out.println(Benchmark.getJVMOSInfo())
out.println(Benchmark.getProcessorName())
out.printf("%-40s %16s %12s %13s %10s\n", name + ":", "Best/Avg Time(ms)", "Rate(M/s)",
out.printf("%-40s %14s %14s %11s %12s %13s %10s\n", name + ":", "Best Time(ms)", "Avg Time(ms)", "Stdev(ms)", "Rate(M/s)",
"Per Row(ns)", "Relative")
out.println("-" * 96)
out.println("-" * 120)
results.zip(benchmarks).foreach { case (result, benchmark) =>
out.printf("%-40s %16s %12s %13s %10s\n",
out.printf("%-40s %14s %14s %11s %12s %13s %10s\n",
benchmark.name,
"%5.0f / %4.0f" format (result.bestMs, result.avgMs),
"%5.0f" format result.bestMs,
"%4.0f" format result.avgMs,
"%5.0f" format result.stdevMs,
"%10.1f" format result.bestRate,
"%6.1f" format (1000 / result.bestRate),
"%3.1fX" format (firstBest / result.bestMs))
Expand Down Expand Up @@ -156,9 +158,13 @@ private[spark] class Benchmark(
// scalastyle:off
println(s" Stopped after $i iterations, ${runTimes.sum / 1000000} ms")
// scalastyle:on
assert(runTimes.nonEmpty)
val best = runTimes.min
val avg = runTimes.sum / runTimes.size
Result(avg / 1000000.0, num / (best / 1000.0), best / 1000000.0)
val stdev = if (runTimes.size > 1) {
math.sqrt(runTimes.map(time => (time - avg) * (time - avg)).sum / (runTimes.size - 1))
} else 0
Result(avg / 1000000.0, num / (best / 1000.0), best / 1000000.0, stdev / 1000000.0)
}
}

Expand Down Expand Up @@ -191,7 +197,7 @@ private[spark] object Benchmark {
}

case class Case(name: String, fn: Timer => Unit, numIters: Int)
case class Result(avgMs: Double, bestRate: Double, bestMs: Double)
case class Result(avgMs: Double, bestRate: Double, bestMs: Double, stdevMs: Double)

/**
* This should return a user helpful processor information. Getting at this depends on the OS.
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,158 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/

package org.apache.spark.shuffle.sort

import java.io.{BufferedInputStream, Closeable, File, FileInputStream, FileOutputStream}
import java.util.UUID

import org.apache.commons.io.FileUtils
import org.mockito.{Mock, MockitoAnnotations}
import org.mockito.Answers.RETURNS_SMART_NULLS
import org.mockito.ArgumentMatchers.any
import org.mockito.Mockito.when
import scala.collection.mutable
import scala.collection.mutable.ArrayBuffer
import scala.util.Random

import org.apache.spark.{HashPartitioner, ShuffleDependency, SparkConf, TaskContext}
import org.apache.spark.benchmark.{Benchmark, BenchmarkBase}
import org.apache.spark.executor.TaskMetrics
import org.apache.spark.memory.{MemoryManager, TaskMemoryManager, TestMemoryManager}
import org.apache.spark.rpc.{RpcEndpoint, RpcEndpointRef, RpcEnv}
import org.apache.spark.serializer.{KryoSerializer, Serializer, SerializerManager}
import org.apache.spark.shuffle.IndexShuffleBlockResolver
import org.apache.spark.storage.{BlockManager, DiskBlockManager, TempShuffleBlockId}
import org.apache.spark.util.Utils

abstract class ShuffleWriterBenchmarkBase extends BenchmarkBase {

protected val DEFAULT_DATA_STRING_SIZE = 5

// This is only used in the writer constructors, so it's ok to mock
@Mock(answer = RETURNS_SMART_NULLS) protected var dependency:
ShuffleDependency[String, String, String] = _
// This is only used in the stop() function, so we can safely mock this without affecting perf
@Mock(answer = RETURNS_SMART_NULLS) protected var taskContext: TaskContext = _
@Mock(answer = RETURNS_SMART_NULLS) protected var rpcEnv: RpcEnv = _
@Mock(answer = RETURNS_SMART_NULLS) protected var rpcEndpointRef: RpcEndpointRef = _

protected val defaultConf: SparkConf = new SparkConf(loadDefaults = false)
protected val serializer: Serializer = new KryoSerializer(defaultConf)
protected val partitioner: HashPartitioner = new HashPartitioner(10)
protected val serializerManager: SerializerManager =
new SerializerManager(serializer, defaultConf)
protected val shuffleMetrics: TaskMetrics = new TaskMetrics

protected val tempFilesCreated: ArrayBuffer[File] = new ArrayBuffer[File]
protected val filenameToFile: mutable.Map[String, File] = new mutable.HashMap[String, File]

class TestDiskBlockManager(tempDir: File) extends DiskBlockManager(defaultConf, false) {
override def getFile(filename: String): File = {
if (filenameToFile.contains(filename)) {
filenameToFile(filename)
} else {
val outputFile = File.createTempFile("shuffle", null, tempDir)
filenameToFile(filename) = outputFile
outputFile
}
}

override def createTempShuffleBlock(): (TempShuffleBlockId, File) = {
var blockId = new TempShuffleBlockId(UUID.randomUUID())
val file = getFile(blockId)
tempFilesCreated += file
(blockId, file)
}
}

class TestBlockManager(tempDir: File, memoryManager: MemoryManager) extends BlockManager("0",
rpcEnv,
null,
serializerManager,
defaultConf,
memoryManager,
null,
null,
null,
null,
1) {
override val diskBlockManager = new TestDiskBlockManager(tempDir)
override val remoteBlockTempFileManager = null
}

protected var tempDir: File = _

protected var blockManager: BlockManager = _
protected var blockResolver: IndexShuffleBlockResolver = _

protected var memoryManager: TestMemoryManager = _
protected var taskMemoryManager: TaskMemoryManager = _

MockitoAnnotations.initMocks(this)
when(dependency.partitioner).thenReturn(partitioner)
when(dependency.serializer).thenReturn(serializer)
when(dependency.shuffleId).thenReturn(0)
when(taskContext.taskMetrics()).thenReturn(shuffleMetrics)
when(rpcEnv.setupEndpoint(any[String], any[RpcEndpoint])).thenReturn(rpcEndpointRef)

def setup(): Unit = {
memoryManager = new TestMemoryManager(defaultConf)
memoryManager.limit(PackedRecordPointer.MAXIMUM_PAGE_SIZE_BYTES)
taskMemoryManager = new TaskMemoryManager(memoryManager, 0)
tempDir = Utils.createTempDir()
blockManager = new TestBlockManager(tempDir, memoryManager)
blockResolver = new IndexShuffleBlockResolver(
defaultConf,
blockManager)
}

def addBenchmarkCase(benchmark: Benchmark, name: String)(func: Benchmark.Timer => Unit): Unit = {
benchmark.addTimerCase(name) { timer =>
setup()
func(timer)
teardown()
}
}

def teardown(): Unit = {
FileUtils.deleteDirectory(tempDir)
tempFilesCreated.clear()
filenameToFile.clear()
}

protected class DataIterator (size: Int)
extends Iterator[Product2[String, String]] {
val random = new Random(123)
var count = 0
override def hasNext: Boolean = {
count < size
}

override def next(): Product2[String, String] = {
count+=1
val string = random.alphanumeric.take(5).mkString
(string, string)
}
}


def createDataIterator(size: Int): DataIterator = {
new DataIterator(size)
}

}
Loading

0 comments on commit 11a37ed

Please sign in to comment.