forked from apache-spark-on-k8s/spark
-
Notifications
You must be signed in to change notification settings - Fork 5
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
SPARK-25299: add CI infrastructure and SortShuffleWriterBenchmark (ap…
…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
1 parent
e4b36df
commit cb3cfb4
Showing
3 changed files
with
418 additions
and
0 deletions.
There are no files selected for viewing
158 changes: 158 additions & 0 deletions
158
core/src/test/scala/org/apache/spark/shuffle/sort/ShuffleWriterBenchmarkBase.scala
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
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) | ||
} | ||
|
||
} |
172 changes: 172 additions & 0 deletions
172
core/src/test/scala/org/apache/spark/shuffle/sort/SortShuffleWriterBenchmark.scala
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,172 @@ | ||
/* | ||
* 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 org.mockito.Mockito.when | ||
|
||
import org.apache.spark.{Aggregator, SparkEnv} | ||
import org.apache.spark.benchmark.Benchmark | ||
import org.apache.spark.shuffle.BaseShuffleHandle | ||
import org.apache.spark.util.Utils | ||
|
||
/** | ||
* Benchmark to measure performance for aggregate primitives. | ||
* {{{ | ||
* To run this benchmark: | ||
* 1. without sbt: bin/spark-submit --class <this class> <spark sql test jar> | ||
* 2. build/sbt "sql/test:runMain <this class>" | ||
* 3. generate result: SPARK_GENERATE_BENCHMARK_FILES=1 build/sbt "sql/test:runMain <this class>" | ||
* Results will be written to "benchmarks/<this class>-results.txt". | ||
* }}} | ||
*/ | ||
object SortShuffleWriterBenchmark extends ShuffleWriterBenchmarkBase { | ||
|
||
private val shuffleHandle: BaseShuffleHandle[String, String, String] = | ||
new BaseShuffleHandle( | ||
shuffleId = 0, | ||
numMaps = 1, | ||
dependency = dependency) | ||
|
||
private val MIN_NUM_ITERS = 10 | ||
private val DATA_SIZE_SMALL = 1000 | ||
private val DATA_SIZE_LARGE = | ||
PackedRecordPointer.MAXIMUM_PAGE_SIZE_BYTES/4/DEFAULT_DATA_STRING_SIZE | ||
|
||
def getWriter(aggregator: Option[Aggregator[String, String, String]], | ||
sorter: Option[Ordering[String]]): SortShuffleWriter[String, String, String] = { | ||
// we need this since SortShuffleWriter uses SparkEnv to get lots of its private vars | ||
SparkEnv.set(new SparkEnv( | ||
"0", | ||
null, | ||
serializer, | ||
null, | ||
serializerManager, | ||
null, | ||
null, | ||
null, | ||
blockManager, | ||
null, | ||
null, | ||
null, | ||
null, | ||
defaultConf | ||
)) | ||
|
||
if (aggregator.isEmpty && sorter.isEmpty) { | ||
when(dependency.mapSideCombine).thenReturn(false) | ||
} else { | ||
when(dependency.mapSideCombine).thenReturn(false) | ||
when(dependency.aggregator).thenReturn(aggregator) | ||
when(dependency.keyOrdering).thenReturn(sorter) | ||
} | ||
|
||
when(taskContext.taskMemoryManager()).thenReturn(taskMemoryManager) | ||
|
||
val shuffleWriter = new SortShuffleWriter[String, String, String]( | ||
blockResolver, | ||
shuffleHandle, | ||
0, | ||
taskContext | ||
) | ||
shuffleWriter | ||
} | ||
|
||
def writeBenchmarkWithSmallDataset(): Unit = { | ||
val size = DATA_SIZE_SMALL | ||
val benchmark = new Benchmark("SortShuffleWriter without spills", | ||
size, | ||
minNumIters = MIN_NUM_ITERS, | ||
output = output) | ||
addBenchmarkCase(benchmark, "small dataset without spills") { timer => | ||
val shuffleWriter = getWriter(Option.empty, Option.empty) | ||
val dataIterator = createDataIterator(size) | ||
try { | ||
timer.startTiming() | ||
shuffleWriter.write(dataIterator) | ||
timer.stopTiming() | ||
assert(tempFilesCreated.isEmpty) | ||
} finally { | ||
shuffleWriter.stop(true) | ||
} | ||
} | ||
benchmark.run() | ||
} | ||
|
||
def writeBenchmarkWithSpill(): Unit = { | ||
val size = DATA_SIZE_LARGE | ||
|
||
val benchmark = new Benchmark("SortShuffleWriter with spills", | ||
size, | ||
minNumIters = MIN_NUM_ITERS, | ||
output = output, | ||
outputPerIteration = true) | ||
addBenchmarkCase(benchmark, "no map side combine") { timer => | ||
val shuffleWriter = getWriter(Option.empty, Option.empty) | ||
val dataIterator = createDataIterator(size) | ||
try { | ||
timer.startTiming() | ||
shuffleWriter.write(dataIterator) | ||
timer.stopTiming() | ||
assert(tempFilesCreated.length == 7) | ||
} finally { | ||
shuffleWriter.stop(true) | ||
} | ||
} | ||
|
||
def createCombiner(i: String): String = i | ||
def mergeValue(i: String, j: String): String = if (Ordering.String.compare(i, j) > 0) i else j | ||
def mergeCombiners(i: String, j: String): String = | ||
if (Ordering.String.compare(i, j) > 0) i else j | ||
val aggregator = | ||
new Aggregator[String, String, String](createCombiner, mergeValue, mergeCombiners) | ||
addBenchmarkCase(benchmark, "with map side aggregation") { timer => | ||
val shuffleWriter = getWriter(Some(aggregator), Option.empty) | ||
val dataIterator = createDataIterator(size) | ||
try { | ||
timer.startTiming() | ||
shuffleWriter.write(dataIterator) | ||
timer.stopTiming() | ||
assert(tempFilesCreated.length == 7) | ||
} finally { | ||
shuffleWriter.stop(true) | ||
} | ||
} | ||
|
||
val sorter = Ordering.String | ||
addBenchmarkCase(benchmark, "with map side sort") { timer => | ||
val shuffleWriter = getWriter(Option.empty, Some(sorter)) | ||
val dataIterator = createDataIterator(size) | ||
try { | ||
timer.startTiming() | ||
shuffleWriter.write(dataIterator) | ||
timer.stopTiming() | ||
assert(tempFilesCreated.length == 7) | ||
} finally { | ||
shuffleWriter.stop(true) | ||
} | ||
} | ||
benchmark.run() | ||
} | ||
|
||
override def runBenchmarkSuite(mainArgs: Array[String]): Unit = { | ||
runBenchmark("SortShuffleWriter writer") { | ||
writeBenchmarkWithSmallDataset() | ||
writeBenchmarkWithSpill() | ||
} | ||
} | ||
} |
Oops, something went wrong.