From c7abec6b997706a43f9020b602a63491b17f5b20 Mon Sep 17 00:00:00 2001 From: Yifei Huang Date: Fri, 22 Feb 2019 17:42:19 -0800 Subject: [PATCH 01/68] add initial bypass merge sort shuffle writer benchmarks --- ...ypassMergeSortShuffleWriterBenchmark.scala | 209 ++++++++++++++++++ 1 file changed, 209 insertions(+) create mode 100644 core/src/test/scala/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriterBenchmark.scala diff --git a/core/src/test/scala/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriterBenchmark.scala b/core/src/test/scala/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriterBenchmark.scala new file mode 100644 index 0000000000000..89c51e9fa8e16 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriterBenchmark.scala @@ -0,0 +1,209 @@ +/* + * 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.File +import java.util.UUID + +import org.mockito.{Mock, MockitoAnnotations} +import org.mockito.Answers.RETURNS_SMART_NULLS +import org.mockito.Matchers.{any, anyInt} +import org.mockito.Mockito.{doAnswer, when} +import org.mockito.invocation.InvocationOnMock +import org.mockito.stubbing.Answer +import scala.collection.mutable +import scala.util.Random + +import org.apache.spark.{HashPartitioner, ShuffleDependency, SparkConf, TaskContext} +import org.apache.spark.benchmark.{Benchmark, BenchmarkBase} +import org.apache.spark.executor.{ShuffleWriteMetrics, TaskMetrics} +import org.apache.spark.serializer.{JavaSerializer, SerializerInstance, SerializerManager} +import org.apache.spark.shuffle.IndexShuffleBlockResolver +import org.apache.spark.storage.{BlockId, BlockManager, DiskBlockManager, DiskBlockObjectWriter, TempShuffleBlockId} +import org.apache.spark.util.Utils + +/** + * Benchmark to measure performance for aggregate primitives. + * {{{ + * To run this benchmark: + * 1. without sbt: bin/spark-submit --class + * 2. build/sbt "sql/test:runMain " + * 3. generate result: SPARK_GENERATE_BENCHMARK_FILES=1 build/sbt "sql/test:runMain " + * Results will be written to "benchmarks/-results.txt". + * }}} + */ +object BypassMergeSortShuffleWriterBenchmark extends BenchmarkBase { + + @Mock(answer = RETURNS_SMART_NULLS) private var blockManager: BlockManager = _ + @Mock(answer = RETURNS_SMART_NULLS) private var diskBlockManager: DiskBlockManager = _ + @Mock(answer = RETURNS_SMART_NULLS) private var taskContext: TaskContext = _ + @Mock(answer = RETURNS_SMART_NULLS) private var blockResolver: IndexShuffleBlockResolver = _ + @Mock(answer = RETURNS_SMART_NULLS) private var dependency: + ShuffleDependency[String, String, String] = _ + + private var tempDir: File = _ + private val blockIdToFileMap: mutable.Map[BlockId, File] = new mutable.HashMap[BlockId, File] + private var shuffleHandle: BypassMergeSortShuffleHandle[String, String] = _ + + def setup(transferTo: Boolean): BypassMergeSortShuffleWriter[String, String] = { + MockitoAnnotations.initMocks(this) + val conf = new SparkConf(loadDefaults = false) + conf.set("spark.file.transferTo", String.valueOf(transferTo)) + conf.set("spark.shuffle.file.buffer", "32k") + + if (shuffleHandle == null) { + shuffleHandle = new BypassMergeSortShuffleHandle[String, String]( + shuffleId = 0, + numMaps = 1, + dependency = dependency + ) + } + + val taskMetrics = new TaskMetrics + when(dependency.partitioner).thenReturn(new HashPartitioner(10)) + when(dependency.serializer).thenReturn(new JavaSerializer(conf)) + when(dependency.shuffleId).thenReturn(0) + + // Create the temporary directory to write local shuffle and temp files + tempDir = Utils.createTempDir() + val outputFile = File.createTempFile("shuffle", null, tempDir) + // Final mapper data file output + when(blockResolver.getDataFile(0, 0)).thenReturn(outputFile) + + // Create the temporary writers (backed by files), one for each partition. + when(blockManager.diskBlockManager).thenReturn(diskBlockManager) + when(diskBlockManager.createTempShuffleBlock()).thenAnswer( + (invocation: InvocationOnMock) => { + val blockId = new TempShuffleBlockId(UUID.randomUUID) + val file = new File(tempDir, blockId.name) + blockIdToFileMap.put(blockId, file) + (blockId, file) + }) + when(blockManager.getDiskWriter( + any[BlockId], + any[File], + any[SerializerInstance], + anyInt(), + any[ShuffleWriteMetrics] + )).thenAnswer(new Answer[DiskBlockObjectWriter] { + override def answer(invocation: InvocationOnMock): DiskBlockObjectWriter = { + val args = invocation.getArguments + val manager = new SerializerManager(new JavaSerializer(conf), conf) + new DiskBlockObjectWriter( + args(1).asInstanceOf[File], + manager, + args(2).asInstanceOf[SerializerInstance], + args(3).asInstanceOf[Int], + syncWrites = false, + args(4).asInstanceOf[ShuffleWriteMetrics], + blockId = args(0).asInstanceOf[BlockId] + ) + } + }) + + // writing the index file + doAnswer(new Answer[Void] { + def answer(invocationOnMock: InvocationOnMock): Void = { + val tmp: File = invocationOnMock.getArguments()(3).asInstanceOf[File] + if (tmp != null) { + outputFile.delete + tmp.renameTo(outputFile) + } + null + } + }).when(blockResolver) + .writeIndexFileAndCommit(anyInt, anyInt, any(classOf[Array[Long]]), any(classOf[File])) + when(blockManager.diskBlockManager).thenReturn(diskBlockManager) + + val shuffleWriter = new BypassMergeSortShuffleWriter[String, String]( + blockManager, + blockResolver, + shuffleHandle, + 0, + conf, + taskMetrics.shuffleWriteMetrics + ) + + shuffleWriter + } + + def write(writer: BypassMergeSortShuffleWriter[String, String], + records: Array[(String, String)]): Unit = { + writer.write(records.iterator) + } + + def cleanupTempFiles(): Unit = { + tempDir.delete() + } + + def writeBenchmarkWithLargeDataset(): Unit = { + val size = 10000000 + val minNumIters = 10 + val random = new Random(123) + val data = (1 to size).map { i => { + val x = random.alphanumeric.take(5).mkString + Tuple2(x, x) + } }.toArray + val benchmark = new Benchmark( + "BypassMergeSortShuffleWrite (with spill) " + size, + size, + minNumIters = minNumIters, + output = output) + benchmark.addTimerCase("without transferTo") { timer => + val shuffleWriter = setup(false) + timer.startTiming() + write(shuffleWriter, data) + timer.stopTiming() + cleanupTempFiles() + } + benchmark.addTimerCase("with transferTo") { timer => + val shuffleWriter = setup(true) + timer.startTiming() + write(shuffleWriter, data) + timer.stopTiming() + cleanupTempFiles() + } + benchmark.run() + } + + def writeBenchmarkWithSmallDataset(): Unit = { + val size = 10000 + val random = new Random(123) + val data = (1 to size).map { i => { + val x = random.alphanumeric.take(5).mkString + Tuple2(x, x) + } }.toArray + val benchmark = new Benchmark("BypassMergeSortShuffleWrite (in memory buffer) " + size, + size, output = output) + benchmark.addTimerCase("small dataset without spills on disk") { timer => + val shuffleWriter = setup(false) + timer.startTiming() + write(shuffleWriter, data) + timer.stopTiming() + cleanupTempFiles() + } + benchmark.run() + } + + override def runBenchmarkSuite(mainArgs: Array[String]): Unit = { + runBenchmark("BypassMergeSortShuffleWriter write") { + writeBenchmarkWithSmallDataset() + writeBenchmarkWithLargeDataset() + } + } +} From 22ef648c206a6de9a49b83c45c3ba3cdfa03f32e Mon Sep 17 00:00:00 2001 From: Yifei Huang Date: Mon, 25 Feb 2019 14:52:25 -0800 Subject: [PATCH 02/68] dd unsafe shuffle writer benchmarks --- .../sort/UnsafeShuffleWriterBenchmark.scala | 257 ++++++++++++++++++ 1 file changed, 257 insertions(+) create mode 100644 core/src/test/scala/org/apache/spark/shuffle/sort/UnsafeShuffleWriterBenchmark.scala diff --git a/core/src/test/scala/org/apache/spark/shuffle/sort/UnsafeShuffleWriterBenchmark.scala b/core/src/test/scala/org/apache/spark/shuffle/sort/UnsafeShuffleWriterBenchmark.scala new file mode 100644 index 0000000000000..eb362574e1ace --- /dev/null +++ b/core/src/test/scala/org/apache/spark/shuffle/sort/UnsafeShuffleWriterBenchmark.scala @@ -0,0 +1,257 @@ +/* + * 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, File, FileInputStream, FileOutputStream} +import java.util +import java.util.{LinkedList, UUID} + +import org.mockito.{Mock, MockitoAnnotations} +import org.mockito.Answers.RETURNS_SMART_NULLS +import org.mockito.Matchers.{any, anyInt} +import org.mockito.Mockito.{doAnswer, when} +import org.mockito.invocation.InvocationOnMock +import scala.util.Random + +import org.apache.spark.{HashPartitioner, ShuffleDependency, SparkConf, TaskContext} +import org.apache.spark.benchmark.{Benchmark, BenchmarkBase} +import org.apache.spark.executor.{ShuffleWriteMetrics, TaskMetrics} +import org.apache.spark.memory.{TaskMemoryManager, TestMemoryManager} +import org.apache.spark.serializer.{KryoSerializer, SerializerInstance, SerializerManager} +import org.apache.spark.shuffle.IndexShuffleBlockResolver +import org.apache.spark.storage.{BlockId, BlockManager, DiskBlockManager, DiskBlockObjectWriter, TempShuffleBlockId} +import org.apache.spark.util.Utils + +/** + * Benchmark to measure performance for aggregate primitives. + * {{{ + * To run this benchmark: + * 1. without sbt: bin/spark-submit --class + * 2. build/sbt "sql/test:runMain " + * 3. generate result: SPARK_GENERATE_BENCHMARK_FILES=1 build/sbt "sql/test:runMain " + * Results will be written to "benchmarks/-results.txt". + * }}} + */ +object UnsafeShuffleWriterBenchmark extends BenchmarkBase { + + @Mock(answer = RETURNS_SMART_NULLS) private var blockManager: BlockManager = _ + @Mock(answer = RETURNS_SMART_NULLS) private var shuffleBlockResolver: + IndexShuffleBlockResolver = _ + @Mock(answer = RETURNS_SMART_NULLS) private var dependency: + ShuffleDependency[String, String, String] = _ + @Mock(answer = RETURNS_SMART_NULLS) private[sort] val taskContext: TaskContext = null + @Mock(answer = RETURNS_SMART_NULLS) private[sort] val diskBlockManager: DiskBlockManager = null + + private[sort] val serializer = new KryoSerializer(new SparkConf) + private[sort] val hashPartitioner = new HashPartitioner(10) + private[sort] val spillFilesCreated: util.LinkedList[File] = new util.LinkedList[File] + private var tempDataFile: File = File.createTempFile("test-data", "") + private var tempDir: File = _ + private var shuffleHandle: SerializedShuffleHandle[String, String] = _ + private var memoryManager: TestMemoryManager = _ + private var taskMemoryManager: TaskMemoryManager = _ + + private val DEFAULT_DATA_STRING_SIZE = 5 + private val MIN_NUM_ITERS = 10 + + def setup(transferTo: Boolean): UnsafeShuffleWriter[String, String] = { + MockitoAnnotations.initMocks(this) + val conf = new SparkConf(loadDefaults = false) + conf.set("spark.file.transferTo", String.valueOf(transferTo)) + memoryManager = new TestMemoryManager(conf) + memoryManager.limit(PackedRecordPointer.MAXIMUM_PAGE_SIZE_BYTES) + taskMemoryManager = new TaskMemoryManager(memoryManager, 0) + if (shuffleHandle == null) { + shuffleHandle = new SerializedShuffleHandle[String, String](0, 0, dependency) + } + val taskMetrics = new TaskMetrics + + + tempDir = Utils.createTempDir(null, "test") + val mergedOutputFile = File.createTempFile("shuffle", "", tempDir) + + // copied from UnsafeShuffleWriterSuite + // Some tests will override this manager because they change the configuration. This is a + // default for tests that don't need a specific one. + val manager: SerializerManager = new SerializerManager(serializer, conf) + when(blockManager.serializerManager).thenReturn(manager) + + when(blockManager.diskBlockManager).thenReturn(diskBlockManager) + when(blockManager.getDiskWriter( + any[BlockId], + any[File], + any[SerializerInstance], + any[Int], + any[ShuffleWriteMetrics])) + .thenAnswer((invocationOnMock: InvocationOnMock) => { + val args = invocationOnMock.getArguments + new DiskBlockObjectWriter( + args(1).asInstanceOf[File], + blockManager.serializerManager, + args(2).asInstanceOf[SerializerInstance], + args(3).asInstanceOf[java.lang.Integer], + false, + args(4).asInstanceOf[ShuffleWriteMetrics], + args(0).asInstanceOf[BlockId]) + }) + + when(shuffleBlockResolver.getDataFile(anyInt, anyInt)).thenReturn(mergedOutputFile) + doAnswer((invocationOnMock: InvocationOnMock) => { + val tmp: File = invocationOnMock.getArguments()(3).asInstanceOf[File] + mergedOutputFile.delete + tmp.renameTo(mergedOutputFile) + }).when(shuffleBlockResolver).writeIndexFileAndCommit( + any[Int], + any[Int], + any[Array[Long]], + any[File]) + + when(diskBlockManager.createTempShuffleBlock) + .thenAnswer((invocationOnMock: InvocationOnMock) => { + val blockId: TempShuffleBlockId = new TempShuffleBlockId(UUID.randomUUID) + val file: File = File.createTempFile("spillFile", ".spill", tempDir) + spillFilesCreated.add(file) + // scalastyle:off println + println("spill file created") + // scalastyle:on println + (blockId, file) + }) + + when(taskContext.taskMetrics()).thenReturn(taskMetrics) + when(dependency.serializer).thenReturn(serializer) + when(dependency.partitioner).thenReturn(hashPartitioner) + + spillFilesCreated.clear() + new UnsafeShuffleWriter[String, String]( + blockManager, + shuffleBlockResolver, + taskMemoryManager, + shuffleHandle, + 0, + taskContext, + conf, + taskMetrics.shuffleWriteMetrics + ) + } + + def cleanupTempFiles(): Unit = { + tempDir.delete() + } + + def createDataInMemory(size: Int): Array[(String, String)] = { + val random = new Random(123) + (1 to size).map { i => { + val x = random.alphanumeric.take(DEFAULT_DATA_STRING_SIZE).mkString + Tuple2(x, x) + } }.toArray + } + + def createDataOnDisk(size: Int): File = { + // scalastyle:off println + println("Generating test data with num records: " + size) + val random = new Random(123) + val dataOutput = new FileOutputStream(tempDataFile) + try { + (1 to size).foreach { i => { + if (i % 1000000 == 0) { + println("Wrote " + i + " test data points") + } + val x = random.alphanumeric.take(DEFAULT_DATA_STRING_SIZE).mkString + dataOutput.write(x.getBytes) + }} + } + finally { + dataOutput.close() + } + tempDataFile + // scalastyle:off println + } + + private class DataIterator(val inputFile: File, val bufferSize: Int) + extends Iterator[Product2[String, String]] { + val inputStream = new BufferedInputStream( + new FileInputStream(tempDataFile), DEFAULT_DATA_STRING_SIZE) + val buffer = new Array[Byte](DEFAULT_DATA_STRING_SIZE) + + override def hasNext: Boolean = { + if (inputStream.available() == 0) { + inputStream.close() + } + inputStream.available() > 0 + } + + override def next(): Product2[String, String] = { + val read = inputStream.read(buffer) + assert(read == 5) + val string = buffer.mkString + (string, string) + } + } + + def writeBenchmarkWithSmallDataset(): Unit = { + val size = 1000 + val benchmark = new Benchmark("UnsafeShuffleWriter with spills", + size, + minNumIters = MIN_NUM_ITERS, + output = output) + benchmark.addTimerCase("small dataset without spills") { timer => + val writer = setup(false) + val array = createDataInMemory(1000) + timer.startTiming() + writer.write(array.iterator) + timer.stopTiming() + assert(spillFilesCreated.size() == 1) // The single temp file is for the temp index file + cleanupTempFiles() + } + benchmark.run() + } + + def writeBenchmarkWithSpill(): Unit = { + val size = PackedRecordPointer.MAXIMUM_PAGE_SIZE_BYTES/2/DEFAULT_DATA_STRING_SIZE + val minNumIters = 10 + createDataOnDisk(size) + val benchmark = new Benchmark("UnsafeShuffleWriter with spills", + size, + minNumIters = minNumIters, + output = output) + benchmark.addTimerCase("without transferTo") { timer => + val shuffleWriter = setup(false) + timer.startTiming() + shuffleWriter.write(new DataIterator(inputFile = tempDataFile, DEFAULT_DATA_STRING_SIZE)) + timer.stopTiming() + assert(spillFilesCreated.size() == 7) + cleanupTempFiles() + } + benchmark.addTimerCase("with transferTo") { timer => + val shuffleWriter = setup(false) + timer.startTiming() + shuffleWriter.write(new DataIterator(inputFile = tempDataFile, DEFAULT_DATA_STRING_SIZE)) + timer.stopTiming() + assert(spillFilesCreated.size() == 7) + cleanupTempFiles() + } + benchmark.run() + tempDataFile.delete() + } + + override def runBenchmarkSuite(mainArgs: Array[String]): Unit = { + runBenchmark("UnsafeShuffleWriter write") { + writeBenchmarkWithSmallDataset() + writeBenchmarkWithSpill() + } + } +} From 4084e278531a9a4d7ba3d50557cc2a0a656ece6f Mon Sep 17 00:00:00 2001 From: Yifei Huang Date: Mon, 25 Feb 2019 14:54:39 -0800 Subject: [PATCH 03/68] changes in bypassmergesort benchmarks --- ...ypassMergeSortShuffleWriterBenchmark.scala | 21 ++++++++++++------- 1 file changed, 14 insertions(+), 7 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriterBenchmark.scala b/core/src/test/scala/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriterBenchmark.scala index 89c51e9fa8e16..a50df6475088a 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriterBenchmark.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriterBenchmark.scala @@ -57,8 +57,13 @@ object BypassMergeSortShuffleWriterBenchmark extends BenchmarkBase { ShuffleDependency[String, String, String] = _ private var tempDir: File = _ - private val blockIdToFileMap: mutable.Map[BlockId, File] = new mutable.HashMap[BlockId, File] private var shuffleHandle: BypassMergeSortShuffleHandle[String, String] = _ + private val blockIdToFileMap: mutable.Map[BlockId, File] = new mutable.HashMap[BlockId, File] + private val partitioner: HashPartitioner = new HashPartitioner(10) + private val defaultConf: SparkConf = new SparkConf(loadDefaults = false) + private val javaSerializer: JavaSerializer = new JavaSerializer(defaultConf) + + private val MIN_NUM_ITERS = 10 def setup(transferTo: Boolean): BypassMergeSortShuffleWriter[String, String] = { MockitoAnnotations.initMocks(this) @@ -75,8 +80,8 @@ object BypassMergeSortShuffleWriterBenchmark extends BenchmarkBase { } val taskMetrics = new TaskMetrics - when(dependency.partitioner).thenReturn(new HashPartitioner(10)) - when(dependency.serializer).thenReturn(new JavaSerializer(conf)) + when(dependency.partitioner).thenReturn(partitioner) + when(dependency.serializer).thenReturn(javaSerializer) when(dependency.shuffleId).thenReturn(0) // Create the temporary directory to write local shuffle and temp files @@ -103,7 +108,7 @@ object BypassMergeSortShuffleWriterBenchmark extends BenchmarkBase { )).thenAnswer(new Answer[DiskBlockObjectWriter] { override def answer(invocation: InvocationOnMock): DiskBlockObjectWriter = { val args = invocation.getArguments - val manager = new SerializerManager(new JavaSerializer(conf), conf) + val manager = new SerializerManager(javaSerializer, conf) new DiskBlockObjectWriter( args(1).asInstanceOf[File], manager, @@ -152,8 +157,8 @@ object BypassMergeSortShuffleWriterBenchmark extends BenchmarkBase { } def writeBenchmarkWithLargeDataset(): Unit = { + // TODO: assert the spill happened val size = 10000000 - val minNumIters = 10 val random = new Random(123) val data = (1 to size).map { i => { val x = random.alphanumeric.take(5).mkString @@ -162,7 +167,7 @@ object BypassMergeSortShuffleWriterBenchmark extends BenchmarkBase { val benchmark = new Benchmark( "BypassMergeSortShuffleWrite (with spill) " + size, size, - minNumIters = minNumIters, + minNumIters = MIN_NUM_ITERS, output = output) benchmark.addTimerCase("without transferTo") { timer => val shuffleWriter = setup(false) @@ -189,7 +194,9 @@ object BypassMergeSortShuffleWriterBenchmark extends BenchmarkBase { Tuple2(x, x) } }.toArray val benchmark = new Benchmark("BypassMergeSortShuffleWrite (in memory buffer) " + size, - size, output = output) + size, + minNumIters = MIN_NUM_ITERS, + output = output) benchmark.addTimerCase("small dataset without spills on disk") { timer => val shuffleWriter = setup(false) timer.startTiming() From fb8266d1fe59073d879526469b8ec29e88013f0d Mon Sep 17 00:00:00 2001 From: Yifei Huang Date: Tue, 26 Feb 2019 12:13:14 -0800 Subject: [PATCH 04/68] cleanup --- ...ypassMergeSortShuffleWriterBenchmark.scala | 12 ++----- .../sort/UnsafeShuffleWriterBenchmark.scala | 32 +++++++++---------- 2 files changed, 19 insertions(+), 25 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriterBenchmark.scala b/core/src/test/scala/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriterBenchmark.scala index a50df6475088a..ff856dd86cc5f 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriterBenchmark.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriterBenchmark.scala @@ -147,17 +147,11 @@ object BypassMergeSortShuffleWriterBenchmark extends BenchmarkBase { shuffleWriter } - def write(writer: BypassMergeSortShuffleWriter[String, String], - records: Array[(String, String)]): Unit = { - writer.write(records.iterator) - } - def cleanupTempFiles(): Unit = { tempDir.delete() } def writeBenchmarkWithLargeDataset(): Unit = { - // TODO: assert the spill happened val size = 10000000 val random = new Random(123) val data = (1 to size).map { i => { @@ -172,14 +166,14 @@ object BypassMergeSortShuffleWriterBenchmark extends BenchmarkBase { benchmark.addTimerCase("without transferTo") { timer => val shuffleWriter = setup(false) timer.startTiming() - write(shuffleWriter, data) + shuffleWriter.write(data.iterator) timer.stopTiming() cleanupTempFiles() } benchmark.addTimerCase("with transferTo") { timer => val shuffleWriter = setup(true) timer.startTiming() - write(shuffleWriter, data) + shuffleWriter.write(data.iterator) timer.stopTiming() cleanupTempFiles() } @@ -200,7 +194,7 @@ object BypassMergeSortShuffleWriterBenchmark extends BenchmarkBase { benchmark.addTimerCase("small dataset without spills on disk") { timer => val shuffleWriter = setup(false) timer.startTiming() - write(shuffleWriter, data) + shuffleWriter.write(data.iterator) timer.stopTiming() cleanupTempFiles() } diff --git a/core/src/test/scala/org/apache/spark/shuffle/sort/UnsafeShuffleWriterBenchmark.scala b/core/src/test/scala/org/apache/spark/shuffle/sort/UnsafeShuffleWriterBenchmark.scala index eb362574e1ace..61614a04917fe 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/sort/UnsafeShuffleWriterBenchmark.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/sort/UnsafeShuffleWriterBenchmark.scala @@ -66,7 +66,7 @@ object UnsafeShuffleWriterBenchmark extends BenchmarkBase { private var taskMemoryManager: TaskMemoryManager = _ private val DEFAULT_DATA_STRING_SIZE = 5 - private val MIN_NUM_ITERS = 10 + private val MIN_NUM_ITERS = 5 def setup(transferTo: Boolean): UnsafeShuffleWriter[String, String] = { MockitoAnnotations.initMocks(this) @@ -125,9 +125,6 @@ object UnsafeShuffleWriterBenchmark extends BenchmarkBase { val blockId: TempShuffleBlockId = new TempShuffleBlockId(UUID.randomUUID) val file: File = File.createTempFile("spillFile", ".spill", tempDir) spillFilesCreated.add(file) - // scalastyle:off println - println("spill file created") - // scalastyle:on println (blockId, file) }) @@ -181,27 +178,30 @@ object UnsafeShuffleWriterBenchmark extends BenchmarkBase { // scalastyle:off println } - private class DataIterator(val inputFile: File, val bufferSize: Int) - extends Iterator[Product2[String, String]] { - val inputStream = new BufferedInputStream( - new FileInputStream(tempDataFile), DEFAULT_DATA_STRING_SIZE) - val buffer = new Array[Byte](DEFAULT_DATA_STRING_SIZE) - + private class DataIterator private ( + private val inputStream: BufferedInputStream, + private val buffer: Array[Byte]) extends Iterator[Product2[String, String]] { override def hasNext: Boolean = { - if (inputStream.available() == 0) { - inputStream.close() - } inputStream.available() > 0 } override def next(): Product2[String, String] = { val read = inputStream.read(buffer) - assert(read == 5) + assert(read == buffer.length) val string = buffer.mkString (string, string) } } + private object DataIterator { + def apply(inputFile: File, bufferSize: Int): DataIterator = { + val inputStream = new BufferedInputStream( + new FileInputStream(inputFile), DEFAULT_DATA_STRING_SIZE) + val buffer = new Array[Byte](DEFAULT_DATA_STRING_SIZE) + new DataIterator(inputStream, buffer) + } + } + def writeBenchmarkWithSmallDataset(): Unit = { val size = 1000 val benchmark = new Benchmark("UnsafeShuffleWriter with spills", @@ -231,7 +231,7 @@ object UnsafeShuffleWriterBenchmark extends BenchmarkBase { benchmark.addTimerCase("without transferTo") { timer => val shuffleWriter = setup(false) timer.startTiming() - shuffleWriter.write(new DataIterator(inputFile = tempDataFile, DEFAULT_DATA_STRING_SIZE)) + shuffleWriter.write(DataIterator(inputFile = tempDataFile, DEFAULT_DATA_STRING_SIZE)) timer.stopTiming() assert(spillFilesCreated.size() == 7) cleanupTempFiles() @@ -239,7 +239,7 @@ object UnsafeShuffleWriterBenchmark extends BenchmarkBase { benchmark.addTimerCase("with transferTo") { timer => val shuffleWriter = setup(false) timer.startTiming() - shuffleWriter.write(new DataIterator(inputFile = tempDataFile, DEFAULT_DATA_STRING_SIZE)) + shuffleWriter.write(DataIterator(inputFile = tempDataFile, DEFAULT_DATA_STRING_SIZE)) timer.stopTiming() assert(spillFilesCreated.size() == 7) cleanupTempFiles() From 89104e2d69999e25726450cda3a02bbed57982ff Mon Sep 17 00:00:00 2001 From: Yifei Huang Date: Tue, 26 Feb 2019 12:15:40 -0800 Subject: [PATCH 05/68] add circle script --- .circleci/config.yml | 17 ++++++++++++ dev/run-spark-25299-benchmarks.sh | 44 +++++++++++++++++++++++++++++++ 2 files changed, 61 insertions(+) create mode 100755 dev/run-spark-25299-benchmarks.sh diff --git a/.circleci/config.yml b/.circleci/config.yml index 717c90b0362ea..bb0f8f873fa82 100644 --- a/.circleci/config.yml +++ b/.circleci/config.yml @@ -23,6 +23,11 @@ all-branches-and-tags: &all-branches-and-tags tags: only: /.*/ +spark-25299-branch-only: &spark-25299-branch-only + filters: + branches: + spark-25299 + deployable-branches-and-tags: &deployable-branches-and-tags filters: tags: @@ -452,6 +457,14 @@ jobs: key: v1-maven-dependency-cache-versioned-{{ checksum "pom.xml" }} paths: ~/.m2 + run-spark-25299-benchmarks: + <<: *defaults + docker: + - image: palantirtechnologies/circle-spark-r:0.1.3 + steps: + - run: + command: ./dev/run-spark-25299-benchmarks.sh + deploy-gradle: <<: *defaults docker: @@ -512,6 +525,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 diff --git a/dev/run-spark-25299-benchmarks.sh b/dev/run-spark-25299-benchmarks.sh new file mode 100755 index 0000000000000..958da819d8920 --- /dev/null +++ b/dev/run-spark-25299-benchmarks.sh @@ -0,0 +1,44 @@ +#!/usr/bin/env bash + +set -e + +echo "Running SPARK-25299 benchmarks" + + +SPARK_GENERATE_BENCHMARK_FILES=1 ./build/sbt "sql/test:runMain org.apache.spark.shuffle.sort.BypassMergeSortShuffleWriterBenchmark" +SPARK_GENERATE_BENCHMARK_FILES=1 ./build/sbt "sql/test:runMain org.apache.spark.shuffle.sort.UnsafeShuffleWriterBenchmark" + +echo "Uploading files to remote store" + +SPARK_DIR=`pwd` +BENCHMARK_REPO="spark-25299-benchmark-results" +SVC_USER="svc-spark-25299" +SVC_PASSWORD=${SVC_SPARK_25299_PASSWORD} +cd ~ +if [ ! -d $BENCHMARK_REPO ]; then + git clone https://${SVC_USER}:${SVC_PASSWORD}@github.com/yifeih/${BENCHMARK_REPO}.git +fi +cd $BENCHMARK_REPO +COMMIT_HASH=$(echo $CIRCLE_SHA1 | awk '{print substr($0,0,10)}') +git checkout -b perf-tests-$COMMIT_HASH + +mkdir $COMMIT_HASH && cd $COMMIT_HASH +cp $SPARK_DIR/sql/core/benchmarks/BypassMergeSortShuffleWriterBenchmark-results.txt . +cp $SPARK_DIR/sql/core/benchmarks/UnsafeShuffleWriterBenchmark-results.txt . + +git add * +git commit -m "Benchmark tests for commit $COMMIT_HASH" +git push origin perf-tests-$COMMIT_HASH + +message='{"title": "Benchmark tests for ' +message+=$COMMIT_HASH +message+='.", ' +message+='"body": "Benchmark tests", ' +message+='"head": "perf-tests-' +message+=$COMMIT_HASH +message+='", ' +message+='"base": "master" }' + +curl -XPOST https://${SVC_USER}:${SVC_PASSWORD}@api.github.com/repos/yifeih/spark-25299-benchmark-results/pulls -d \'$message\' + + From b90b381cd339ad4c7e5a953b0394f6eb9bfbaa30 Mon Sep 17 00:00:00 2001 From: Yifei Huang Date: Tue, 26 Feb 2019 12:17:33 -0800 Subject: [PATCH 06/68] add this branch for testing --- .circleci/config.yml | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/.circleci/config.yml b/.circleci/config.yml index bb0f8f873fa82..ec9d16b8b21ed 100644 --- a/.circleci/config.yml +++ b/.circleci/config.yml @@ -26,7 +26,9 @@ all-branches-and-tags: &all-branches-and-tags spark-25299-branch-only: &spark-25299-branch-only filters: branches: - spark-25299 + only: + - spark-25299 + - yh/add-benchmarks-and-ci deployable-branches-and-tags: &deployable-branches-and-tags filters: From 5e13dd85dc4889ea89a25af6ee274c2457a291aa Mon Sep 17 00:00:00 2001 From: Yifei Huang Date: Tue, 26 Feb 2019 12:20:02 -0800 Subject: [PATCH 07/68] fix circle attempt 1 --- .circleci/config.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.circleci/config.yml b/.circleci/config.yml index ec9d16b8b21ed..fd3a227454677 100644 --- a/.circleci/config.yml +++ b/.circleci/config.yml @@ -530,7 +530,7 @@ workflows: - run-spark-25299-benchmarks: requires: - build-sbt - <<: *spark-25299-branch-only + <<: *spark-25299-branch-only - run-scala-tests: requires: - build-sbt From 845e6454486c1edd8e16a384e1526f139e0ad972 Mon Sep 17 00:00:00 2001 From: Yifei Huang Date: Tue, 26 Feb 2019 12:45:50 -0800 Subject: [PATCH 08/68] checkout code --- .circleci/config.yml | 1 + 1 file changed, 1 insertion(+) diff --git a/.circleci/config.yml b/.circleci/config.yml index fd3a227454677..f8eccc6a2facc 100644 --- a/.circleci/config.yml +++ b/.circleci/config.yml @@ -464,6 +464,7 @@ jobs: docker: - image: palantirtechnologies/circle-spark-r:0.1.3 steps: + - *checkout-code - run: command: ./dev/run-spark-25299-benchmarks.sh From a68f45992b01f094ca483296b8674a1b7576e029 Mon Sep 17 00:00:00 2001 From: Yifei Huang Date: Tue, 26 Feb 2019 13:01:56 -0800 Subject: [PATCH 09/68] add some caches? --- .circleci/config.yml | 3 +++ dev/run-spark-25299-benchmarks.sh | 2 +- 2 files changed, 4 insertions(+), 1 deletion(-) diff --git a/.circleci/config.yml b/.circleci/config.yml index f8eccc6a2facc..fd0b370ebc9d5 100644 --- a/.circleci/config.yml +++ b/.circleci/config.yml @@ -465,6 +465,9 @@ jobs: - image: palantirtechnologies/circle-spark-r:0.1.3 steps: - *checkout-code + - *restore-ivy-cache + - *restore-build-binaries-cache + - *restore-home-sbt-cache - run: command: ./dev/run-spark-25299-benchmarks.sh diff --git a/dev/run-spark-25299-benchmarks.sh b/dev/run-spark-25299-benchmarks.sh index 958da819d8920..67cf394b86546 100755 --- a/dev/run-spark-25299-benchmarks.sh +++ b/dev/run-spark-25299-benchmarks.sh @@ -1,6 +1,6 @@ #!/usr/bin/env bash -set -e +set -ou pipefail echo "Running SPARK-25299 benchmarks" From 757f6fea8438d9f836537b5eb8048b1c9dd97603 Mon Sep 17 00:00:00 2001 From: Yifei Huang Date: Tue, 26 Feb 2019 13:11:14 -0800 Subject: [PATCH 10/68] why is it not pull caches... --- .circleci/config.yml | 2 ++ 1 file changed, 2 insertions(+) diff --git a/.circleci/config.yml b/.circleci/config.yml index fd0b370ebc9d5..878b706db6a3b 100644 --- a/.circleci/config.yml +++ b/.circleci/config.yml @@ -465,6 +465,8 @@ jobs: - image: palantirtechnologies/circle-spark-r:0.1.3 steps: - *checkout-code + - *restore-build-sbt-cache + - *link-in-build-sbt-cache - *restore-ivy-cache - *restore-build-binaries-cache - *restore-home-sbt-cache From 0bcd5d95cf45afadacf38877f52414fe3dfe2cfd Mon Sep 17 00:00:00 2001 From: Yifei Huang Date: Tue, 26 Feb 2019 14:32:43 -0800 Subject: [PATCH 11/68] save as artifact instead of publishing --- .circleci/config.yml | 4 +++ .../sort/UnsafeShuffleWriterBenchmark.scala | 2 +- dev/run-spark-25299-benchmarks.sh | 32 ++----------------- 3 files changed, 7 insertions(+), 31 deletions(-) diff --git a/.circleci/config.yml b/.circleci/config.yml index 878b706db6a3b..ab92c48d01595 100644 --- a/.circleci/config.yml +++ b/.circleci/config.yml @@ -465,6 +465,8 @@ jobs: - image: palantirtechnologies/circle-spark-r:0.1.3 steps: - *checkout-code + - attach_workspace: + at: . - *restore-build-sbt-cache - *link-in-build-sbt-cache - *restore-ivy-cache @@ -472,6 +474,8 @@ jobs: - *restore-home-sbt-cache - run: command: ./dev/run-spark-25299-benchmarks.sh + - store_artifacts: + path: /tmp/artifact deploy-gradle: <<: *defaults diff --git a/core/src/test/scala/org/apache/spark/shuffle/sort/UnsafeShuffleWriterBenchmark.scala b/core/src/test/scala/org/apache/spark/shuffle/sort/UnsafeShuffleWriterBenchmark.scala index 61614a04917fe..e179bc228fe4e 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/sort/UnsafeShuffleWriterBenchmark.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/sort/UnsafeShuffleWriterBenchmark.scala @@ -222,7 +222,7 @@ object UnsafeShuffleWriterBenchmark extends BenchmarkBase { def writeBenchmarkWithSpill(): Unit = { val size = PackedRecordPointer.MAXIMUM_PAGE_SIZE_BYTES/2/DEFAULT_DATA_STRING_SIZE - val minNumIters = 10 + val minNumIters = 5 createDataOnDisk(size) val benchmark = new Benchmark("UnsafeShuffleWriter with spills", size, diff --git a/dev/run-spark-25299-benchmarks.sh b/dev/run-spark-25299-benchmarks.sh index 67cf394b86546..e844958d73514 100755 --- a/dev/run-spark-25299-benchmarks.sh +++ b/dev/run-spark-25299-benchmarks.sh @@ -4,41 +4,13 @@ set -ou pipefail echo "Running SPARK-25299 benchmarks" - SPARK_GENERATE_BENCHMARK_FILES=1 ./build/sbt "sql/test:runMain org.apache.spark.shuffle.sort.BypassMergeSortShuffleWriterBenchmark" SPARK_GENERATE_BENCHMARK_FILES=1 ./build/sbt "sql/test:runMain org.apache.spark.shuffle.sort.UnsafeShuffleWriterBenchmark" echo "Uploading files to remote store" SPARK_DIR=`pwd` -BENCHMARK_REPO="spark-25299-benchmark-results" -SVC_USER="svc-spark-25299" -SVC_PASSWORD=${SVC_SPARK_25299_PASSWORD} -cd ~ -if [ ! -d $BENCHMARK_REPO ]; then - git clone https://${SVC_USER}:${SVC_PASSWORD}@github.com/yifeih/${BENCHMARK_REPO}.git -fi -cd $BENCHMARK_REPO -COMMIT_HASH=$(echo $CIRCLE_SHA1 | awk '{print substr($0,0,10)}') -git checkout -b perf-tests-$COMMIT_HASH - -mkdir $COMMIT_HASH && cd $COMMIT_HASH -cp $SPARK_DIR/sql/core/benchmarks/BypassMergeSortShuffleWriterBenchmark-results.txt . -cp $SPARK_DIR/sql/core/benchmarks/UnsafeShuffleWriterBenchmark-results.txt . - -git add * -git commit -m "Benchmark tests for commit $COMMIT_HASH" -git push origin perf-tests-$COMMIT_HASH - -message='{"title": "Benchmark tests for ' -message+=$COMMIT_HASH -message+='.", ' -message+='"body": "Benchmark tests", ' -message+='"head": "perf-tests-' -message+=$COMMIT_HASH -message+='", ' -message+='"base": "master" }' - -curl -XPOST https://${SVC_USER}:${SVC_PASSWORD}@api.github.com/repos/yifeih/spark-25299-benchmark-results/pulls -d \'$message\' +cp $SPARK_DIR/sql/core/benchmarks/BypassMergeSortShuffleWriterBenchmark-results.txt /tmp/artifacts/ +cp $SPARK_DIR/sql/core/benchmarks/UnsafeShuffleWriterBenchmark-results.txt /tmp/artifacts/ From 26c01ec5fba48ed8b6f28fd63d30a6c9cf03eb06 Mon Sep 17 00:00:00 2001 From: Yifei Huang Date: Tue, 26 Feb 2019 15:10:44 -0800 Subject: [PATCH 12/68] mkdir --- dev/run-spark-25299-benchmarks.sh | 1 + 1 file changed, 1 insertion(+) diff --git a/dev/run-spark-25299-benchmarks.sh b/dev/run-spark-25299-benchmarks.sh index e844958d73514..ceef855bdef0c 100755 --- a/dev/run-spark-25299-benchmarks.sh +++ b/dev/run-spark-25299-benchmarks.sh @@ -11,6 +11,7 @@ echo "Uploading files to remote store" SPARK_DIR=`pwd` +mkdir -p /tmp/artifacts cp $SPARK_DIR/sql/core/benchmarks/BypassMergeSortShuffleWriterBenchmark-results.txt /tmp/artifacts/ cp $SPARK_DIR/sql/core/benchmarks/UnsafeShuffleWriterBenchmark-results.txt /tmp/artifacts/ From 0d7a0363261fb19cb45599002d210fc9c212757a Mon Sep 17 00:00:00 2001 From: Yifei Huang Date: Tue, 26 Feb 2019 15:30:04 -0800 Subject: [PATCH 13/68] typo --- .circleci/config.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.circleci/config.yml b/.circleci/config.yml index ab92c48d01595..a7a9735c709c4 100644 --- a/.circleci/config.yml +++ b/.circleci/config.yml @@ -475,7 +475,7 @@ jobs: - run: command: ./dev/run-spark-25299-benchmarks.sh - store_artifacts: - path: /tmp/artifact + path: /tmp/artifacts deploy-gradle: <<: *defaults From 3fc5331e574756fa13e29d1c3cb2fd1add566c7e Mon Sep 17 00:00:00 2001 From: Yifei Huang Date: Tue, 26 Feb 2019 15:56:04 -0800 Subject: [PATCH 14/68] try uploading artifacts again --- .circleci/config.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.circleci/config.yml b/.circleci/config.yml index a7a9735c709c4..9d489dccace34 100644 --- a/.circleci/config.yml +++ b/.circleci/config.yml @@ -475,7 +475,7 @@ jobs: - run: command: ./dev/run-spark-25299-benchmarks.sh - store_artifacts: - path: /tmp/artifacts + path: /tmp/artifacts/ deploy-gradle: <<: *defaults From 8c337018cfd67b6107b47dea8813a000281dc7d6 Mon Sep 17 00:00:00 2001 From: Yifei Huang Date: Tue, 26 Feb 2019 15:56:28 -0800 Subject: [PATCH 15/68] try print per iteration to avoid circle erroring out on idle --- .../spark/shuffle/sort/UnsafeShuffleWriterBenchmark.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/core/src/test/scala/org/apache/spark/shuffle/sort/UnsafeShuffleWriterBenchmark.scala b/core/src/test/scala/org/apache/spark/shuffle/sort/UnsafeShuffleWriterBenchmark.scala index e179bc228fe4e..bb493a3e9e972 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/sort/UnsafeShuffleWriterBenchmark.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/sort/UnsafeShuffleWriterBenchmark.scala @@ -227,7 +227,8 @@ object UnsafeShuffleWriterBenchmark extends BenchmarkBase { val benchmark = new Benchmark("UnsafeShuffleWriter with spills", size, minNumIters = minNumIters, - output = output) + output = output, + outputPerIteration = true) benchmark.addTimerCase("without transferTo") { timer => val shuffleWriter = setup(false) timer.startTiming() From 9546397040ef77fa35579b34ae95108c8092e04b Mon Sep 17 00:00:00 2001 From: Yifei Huang Date: Tue, 26 Feb 2019 16:29:26 -0800 Subject: [PATCH 16/68] blah (#495) --- dev/run-spark-25299-benchmarks.sh | 16 ++++++++++++++++ 1 file changed, 16 insertions(+) diff --git a/dev/run-spark-25299-benchmarks.sh b/dev/run-spark-25299-benchmarks.sh index ceef855bdef0c..77723017399d4 100755 --- a/dev/run-spark-25299-benchmarks.sh +++ b/dev/run-spark-25299-benchmarks.sh @@ -15,3 +15,19 @@ mkdir -p /tmp/artifacts cp $SPARK_DIR/sql/core/benchmarks/BypassMergeSortShuffleWriterBenchmark-results.txt /tmp/artifacts/ cp $SPARK_DIR/sql/core/benchmarks/UnsafeShuffleWriterBenchmark-results.txt /tmp/artifacts/ + +RESULTS="" +for benchmark_file in /tmp/artifacts/*.txt; do + RESULTS+=$(cat $benchmark_file) +done + +PULL_REQUEST_NUM=$(git log -1 | sed "5q;d" | awk '{print $NF}' | sed 's/(//g' | sed 's/)//g' | sed 's/#//g') + + +USERNAME=svc-spark-25299 +PASSWORD=$SVC_SPARK_25299_PASSWORD +message='{"body": "```' +message+=$results +message+='```", "event":"COMMENT"}' +echo $message +curl -XPOST https://${USERNAME}:${PASSWORD}@api.github.com/repos/palantir/spark/pulls/${PULL_REQUEST_NUM}/reviews -d \'$message\' From d72ba73ed89c15124432d6228546b475ec93c443 Mon Sep 17 00:00:00 2001 From: Yifei Huang Date: Tue, 26 Feb 2019 17:11:01 -0800 Subject: [PATCH 17/68] make a PR comment --- dev/run-spark-25299-benchmarks.sh | 45 ++++++++++++++++++++++++++----- 1 file changed, 39 insertions(+), 6 deletions(-) diff --git a/dev/run-spark-25299-benchmarks.sh b/dev/run-spark-25299-benchmarks.sh index 77723017399d4..09a5d466ea43c 100755 --- a/dev/run-spark-25299-benchmarks.sh +++ b/dev/run-spark-25299-benchmarks.sh @@ -2,32 +2,65 @@ set -ou pipefail + +function usage { + echo "Usage: $(basename $0) [-h] [-u]" + echo "" + echo "Runs the perf tests and optionally uploads the results as a comment to a PR" + echo "" + echo " -h help" + echo " -u Upload the perf results as a comment" + # Exit as error for nesting scripts + exit 1 +} + +UPLOAD=false +while getopts "hu" opt; do + case $opt in + h) + usage + exit 0;; + u) + UPLOAD=true;; + esac +done + echo "Running SPARK-25299 benchmarks" SPARK_GENERATE_BENCHMARK_FILES=1 ./build/sbt "sql/test:runMain org.apache.spark.shuffle.sort.BypassMergeSortShuffleWriterBenchmark" SPARK_GENERATE_BENCHMARK_FILES=1 ./build/sbt "sql/test:runMain org.apache.spark.shuffle.sort.UnsafeShuffleWriterBenchmark" -echo "Uploading files to remote store" - SPARK_DIR=`pwd` mkdir -p /tmp/artifacts cp $SPARK_DIR/sql/core/benchmarks/BypassMergeSortShuffleWriterBenchmark-results.txt /tmp/artifacts/ cp $SPARK_DIR/sql/core/benchmarks/UnsafeShuffleWriterBenchmark-results.txt /tmp/artifacts/ +if [ "$UPLOAD" = false ]; then + exit 0 +fi +IFS= RESULTS="" for benchmark_file in /tmp/artifacts/*.txt; do RESULTS+=$(cat $benchmark_file) + RESULTS+=$'\n\n' done +echo $RESULTS PULL_REQUEST_NUM=$(git log -1 | sed "5q;d" | awk '{print $NF}' | sed 's/(//g' | sed 's/)//g' | sed 's/#//g') USERNAME=svc-spark-25299 PASSWORD=$SVC_SPARK_25299_PASSWORD message='{"body": "```' -message+=$results -message+='```", "event":"COMMENT"}' -echo $message -curl -XPOST https://${USERNAME}:${PASSWORD}@api.github.com/repos/palantir/spark/pulls/${PULL_REQUEST_NUM}/reviews -d \'$message\' +message+=$'\n' +message+=$RESULTS +message+=$'\n' +json_message=$(echo $message | awk '{printf "%s\\n", $0}') +json_message+='```", "event":"COMMENT"}' +echo "$json_message" > benchmark_results.json + +echo "Sending benchmark requests to PR $PULL_REQUEST_NUM" +curl -XPOST https://${USERNAME}:${PASSWORD}@api.github.com/repos/palantir/spark/pulls/${PULL_REQUEST_NUM}/reviews -d @benchmark_results.json +rm benchmark_results.json From 18598058b4d66fd76a0969d5c44ead9c822b4e6d Mon Sep 17 00:00:00 2001 From: Yifei Huang Date: Tue, 26 Feb 2019 17:30:03 -0800 Subject: [PATCH 18/68] actually delete files --- .../shuffle/sort/BypassMergeSortShuffleWriterBenchmark.scala | 3 ++- .../spark/shuffle/sort/UnsafeShuffleWriterBenchmark.scala | 5 +++-- 2 files changed, 5 insertions(+), 3 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriterBenchmark.scala b/core/src/test/scala/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriterBenchmark.scala index ff856dd86cc5f..846d6773de51e 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriterBenchmark.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriterBenchmark.scala @@ -20,6 +20,7 @@ package org.apache.spark.shuffle.sort import java.io.File 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.Matchers.{any, anyInt} @@ -148,7 +149,7 @@ object BypassMergeSortShuffleWriterBenchmark extends BenchmarkBase { } def cleanupTempFiles(): Unit = { - tempDir.delete() + FileUtils.deleteDirectory(tempDir) } def writeBenchmarkWithLargeDataset(): Unit = { diff --git a/core/src/test/scala/org/apache/spark/shuffle/sort/UnsafeShuffleWriterBenchmark.scala b/core/src/test/scala/org/apache/spark/shuffle/sort/UnsafeShuffleWriterBenchmark.scala index bb493a3e9e972..7f41b44407474 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/sort/UnsafeShuffleWriterBenchmark.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/sort/UnsafeShuffleWriterBenchmark.scala @@ -18,8 +18,9 @@ package org.apache.spark.shuffle.sort import java.io.{BufferedInputStream, File, FileInputStream, FileOutputStream} import java.util -import java.util.{LinkedList, UUID} +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.Matchers.{any, anyInt} @@ -146,7 +147,7 @@ object UnsafeShuffleWriterBenchmark extends BenchmarkBase { } def cleanupTempFiles(): Unit = { - tempDir.delete() + FileUtils.deleteDirectory(tempDir) } def createDataInMemory(size: Int): Array[(String, String)] = { From c20f0befb21a30a8663d2c66730b900c92c9e113 Mon Sep 17 00:00:00 2001 From: Yifei Huang Date: Tue, 26 Feb 2019 17:41:24 -0800 Subject: [PATCH 19/68] run benchmarks on test build branch --- .circleci/config.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.circleci/config.yml b/.circleci/config.yml index 9d489dccace34..a5e1e3480986c 100644 --- a/.circleci/config.yml +++ b/.circleci/config.yml @@ -28,7 +28,7 @@ spark-25299-branch-only: &spark-25299-branch-only branches: only: - spark-25299 - - yh/add-benchmarks-and-ci + - spark-25299-test-build deployable-branches-and-tags: &deployable-branches-and-tags filters: From 444d46a87f00a10036abdc93911075b94cefe849 Mon Sep 17 00:00:00 2001 From: Yifei Huang Date: Tue, 26 Feb 2019 18:40:16 -0800 Subject: [PATCH 20/68] oops forgot to enable upload --- .circleci/config.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.circleci/config.yml b/.circleci/config.yml index a5e1e3480986c..8fd23e9bad5e3 100644 --- a/.circleci/config.yml +++ b/.circleci/config.yml @@ -473,7 +473,7 @@ jobs: - *restore-build-binaries-cache - *restore-home-sbt-cache - run: - command: ./dev/run-spark-25299-benchmarks.sh + command: ./dev/run-spark-25299-benchmarks.sh -u - store_artifacts: path: /tmp/artifacts/ From 23229332071b8f19467abd6ac035ae23cc9f9d0c Mon Sep 17 00:00:00 2001 From: Yifei Huang Date: Wed, 27 Feb 2019 13:18:19 -0800 Subject: [PATCH 21/68] add sort shuffle writer benchmarks --- ...ypassMergeSortShuffleWriterBenchmark.scala | 1 - .../sort/SortShuffleWriterBenchmark.scala | 305 ++++++++++++++++++ 2 files changed, 305 insertions(+), 1 deletion(-) create mode 100644 core/src/test/scala/org/apache/spark/shuffle/sort/SortShuffleWriterBenchmark.scala diff --git a/core/src/test/scala/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriterBenchmark.scala b/core/src/test/scala/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriterBenchmark.scala index 846d6773de51e..c003dbca0429a 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriterBenchmark.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriterBenchmark.scala @@ -134,7 +134,6 @@ object BypassMergeSortShuffleWriterBenchmark extends BenchmarkBase { } }).when(blockResolver) .writeIndexFileAndCommit(anyInt, anyInt, any(classOf[Array[Long]]), any(classOf[File])) - when(blockManager.diskBlockManager).thenReturn(diskBlockManager) val shuffleWriter = new BypassMergeSortShuffleWriter[String, String]( blockManager, diff --git a/core/src/test/scala/org/apache/spark/shuffle/sort/SortShuffleWriterBenchmark.scala b/core/src/test/scala/org/apache/spark/shuffle/sort/SortShuffleWriterBenchmark.scala new file mode 100644 index 0000000000000..43c378a85b869 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/shuffle/sort/SortShuffleWriterBenchmark.scala @@ -0,0 +1,305 @@ +/* + * 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, File, FileInputStream, FileOutputStream} +import java.util +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.Matchers.{any, anyInt} +import org.mockito.Mockito.{doAnswer, when} +import org.mockito.invocation.InvocationOnMock +import org.mockito.stubbing.Answer +import scala.collection.mutable +import scala.collection.mutable.ArrayBuffer +import scala.util.Random + +import org.apache.spark.{Aggregator, HashPartitioner, ShuffleDependency, SparkConf, SparkEnv, TaskContext} +import org.apache.spark.benchmark.{Benchmark, BenchmarkBase} +import org.apache.spark.executor.{ShuffleWriteMetrics, TaskMetrics} +import org.apache.spark.memory.{TaskMemoryManager, TestMemoryManager} +import org.apache.spark.serializer.{KryoSerializer, SerializerInstance, SerializerManager} +import org.apache.spark.shuffle.{BaseShuffleHandle, IndexShuffleBlockResolver} +import org.apache.spark.storage.{BlockId, BlockManager, DiskBlockManager, DiskBlockObjectWriter, TempShuffleBlockId} +import org.apache.spark.util.Utils + +/** + * Benchmark to measure performance for aggregate primitives. + * {{{ + * To run this benchmark: + * 1. without sbt: bin/spark-submit --class + * 2. build/sbt "sql/test:runMain " + * 3. generate result: SPARK_GENERATE_BENCHMARK_FILES=1 build/sbt "sql/test:runMain " + * Results will be written to "benchmarks/-results.txt". + * }}} + */ +object SortShuffleWriterBenchmark extends BenchmarkBase { + + @Mock(answer = RETURNS_SMART_NULLS) private var blockManager: BlockManager = _ + @Mock(answer = RETURNS_SMART_NULLS) private var blockResolver: IndexShuffleBlockResolver = _ + @Mock(answer = RETURNS_SMART_NULLS) private var diskBlockManager: DiskBlockManager = _ + @Mock(answer = RETURNS_SMART_NULLS) private var taskContext: TaskContext = _ + @Mock(answer = RETURNS_SMART_NULLS) private var dependency: + ShuffleDependency[String, String, String] = _ + + private var tempDir: File = _ + private var shuffleHandle: BaseShuffleHandle[String, String, String] = _ + private val spillFilesCreated: util.LinkedList[File] = new util.LinkedList[File] + private val partitioner: HashPartitioner = new HashPartitioner(10) + private val defaultConf: SparkConf = new SparkConf() + private val serializer = new KryoSerializer(defaultConf) + private val serializerManager = new SerializerManager(serializer, defaultConf) + private var memoryManager: TestMemoryManager = new TestMemoryManager(defaultConf) + private var taskMemoryManager: TaskMemoryManager = new TaskMemoryManager(memoryManager, 0) + + private val DEFAULT_DATA_STRING_SIZE = 5 + private val MIN_NUM_ITERS = 5 + + def setup(aggregator: Option[Aggregator[String, String, String]], + sorter: Option[Ordering[String]]): SortShuffleWriter[String, String, String] = { + MockitoAnnotations.initMocks(this) + // we need this since SortShuffleWriter uses SparkEnv to get lots of its private vars + val defaultSparkEnv = SparkEnv.get + SparkEnv.set(new SparkEnv( + "0", + null, + serializer, + null, + serializerManager, + null, + null, + null, + blockManager, + null, + null, + null, + null, + defaultConf + )) + shuffleHandle = new BaseShuffleHandle( + shuffleId = 0, + numMaps = 1, + dependency = dependency) + + when(dependency.partitioner).thenReturn(partitioner) + when(dependency.serializer).thenReturn(serializer) + when(dependency.shuffleId).thenReturn(0) + 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) + } + + tempDir = Utils.createTempDir() + val outputFile = File.createTempFile("shuffle", null, tempDir) + when(blockResolver.getDataFile(0, 0)).thenReturn(outputFile) + when(blockManager.diskBlockManager).thenReturn(diskBlockManager) + when(diskBlockManager.createTempShuffleBlock()).thenAnswer( + (invocation: InvocationOnMock) => { + val blockId = new TempShuffleBlockId(UUID.randomUUID) + val file = new File(tempDir, blockId.name) + // scalastyle:off println + println("created spill file") + spillFilesCreated.add(file) + // scalastyle:on println + (blockId, file) + }) + + when(blockManager.getDiskWriter( + any[BlockId], + any[File], + any[SerializerInstance], + anyInt(), + any[ShuffleWriteMetrics] + )).thenAnswer(new Answer[DiskBlockObjectWriter] { + override def answer(invocation: InvocationOnMock): DiskBlockObjectWriter = { + val args = invocation.getArguments + val manager = serializerManager + new DiskBlockObjectWriter( + args(1).asInstanceOf[File], + manager, + args(2).asInstanceOf[SerializerInstance], + args(3).asInstanceOf[Int], + syncWrites = false, + args(4).asInstanceOf[ShuffleWriteMetrics], + blockId = args(0).asInstanceOf[BlockId] + ) + } + }) + + doAnswer(new Answer[Void] { + def answer(invocationOnMock: InvocationOnMock): Void = { + val tmp: File = invocationOnMock.getArguments()(3).asInstanceOf[File] + if (tmp != null) { + outputFile.delete + tmp.renameTo(outputFile) + } + null + } + }).when(blockResolver) + .writeIndexFileAndCommit(anyInt, anyInt, any(classOf[Array[Long]]), any(classOf[File])) + + val taskMetrics = new TaskMetrics + when(taskContext.taskMetrics()).thenReturn(taskMetrics) + + memoryManager.limit(PackedRecordPointer.MAXIMUM_PAGE_SIZE_BYTES) + when(taskContext.taskMemoryManager()).thenReturn(taskMemoryManager) + + val shuffleWriter = new SortShuffleWriter[String, String, String]( + blockResolver, + shuffleHandle, + 0, + taskContext + ) + shuffleWriter + } + + def cleanupTempFiles(): Unit = { + FileUtils.deleteDirectory(tempDir) + } + + def createDataInMemory(size: Int): Array[(String, String)] = { + val random = new Random(123) + (1 to size).map { i => { + val x = random.alphanumeric.take(DEFAULT_DATA_STRING_SIZE).mkString + Tuple2(x, x) + } }.toArray + } + + def createDataOnDisk(size: Int): File = { + // scalastyle:off println + val tempDataFile: File = File.createTempFile("test-data", "") + println("Generating test data with num records: " + size) + val random = new Random(123) + val dataOutput = new FileOutputStream(tempDataFile) + try { + (1 to size).foreach { i => { + if (i % 1000000 == 0) { + println("Wrote " + i + " test data points") + } + val x = random.alphanumeric.take(DEFAULT_DATA_STRING_SIZE).mkString + dataOutput.write(x.getBytes) + }} + } + finally { + dataOutput.close() + } + tempDataFile + // scalastyle:off println + } + + private class DataIterator private ( + private val inputStream: BufferedInputStream, + private val buffer: Array[Byte]) extends Iterator[Product2[String, String]] { + override def hasNext: Boolean = { + inputStream.available() > 0 + } + + override def next(): Product2[String, String] = { + val read = inputStream.read(buffer) + assert(read == buffer.length) + val string = buffer.mkString + (string, string) + } + } + + private object DataIterator { + def apply(inputFile: File, bufferSize: Int): DataIterator = { + val inputStream = new BufferedInputStream( + new FileInputStream(inputFile), DEFAULT_DATA_STRING_SIZE) + val buffer = new Array[Byte](DEFAULT_DATA_STRING_SIZE) + new DataIterator(inputStream, buffer) + } + } + + def writeBenchmarkWithSmallDataset(): Unit = { + val size = 1000 + val benchmark = new Benchmark("SortShuffleWriter with spills", + size, + minNumIters = MIN_NUM_ITERS, + output = output) + benchmark.addTimerCase("small dataset without spills") { timer => + val writer = setup(Option.empty, Option.empty) + val array = createDataInMemory(1000) + timer.startTiming() + writer.write(array.iterator) + timer.stopTiming() + assert(spillFilesCreated.size() == 0) + cleanupTempFiles() + } + benchmark.run() + } + + def writeBenchmarkWithSpill(dataFile: File, size: Int): Unit = { + val benchmark = new Benchmark("SortShuffleWriter with spills", + size, + minNumIters = MIN_NUM_ITERS, + output = output, + outputPerIteration = true) + benchmark.addTimerCase("no map side combine") { timer => + val shuffleWriter = setup(Option.empty, Option.empty) + timer.startTiming() + shuffleWriter.write(DataIterator(inputFile = dataFile, DEFAULT_DATA_STRING_SIZE)) + timer.stopTiming() + assert(spillFilesCreated.size() > 0) + cleanupTempFiles() + } + + 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) + benchmark.addTimerCase("with map side aggregation") { timer => + val shuffleWriter = setup(Some(aggregator), Option.empty) + timer.startTiming() + shuffleWriter.write(DataIterator(inputFile = dataFile, DEFAULT_DATA_STRING_SIZE)) + timer.stopTiming() + assert(spillFilesCreated.size() > 0) + cleanupTempFiles() + } + + val sorter = Ordering.String + benchmark.addTimerCase("with map side sort") { timer => + val shuffleWriter = setup(Option.empty, Some(sorter)) + timer.startTiming() + shuffleWriter.write(DataIterator(inputFile = dataFile, DEFAULT_DATA_STRING_SIZE)) + timer.stopTiming() + assert(spillFilesCreated.size() > 0) + cleanupTempFiles() + } + benchmark.run() + } + + override def runBenchmarkSuite(mainArgs: Array[String]): Unit = { + + runBenchmark("SortShuffleWriter writer") { + writeBenchmarkWithSmallDataset() + val size = PackedRecordPointer.MAXIMUM_PAGE_SIZE_BYTES/4/DEFAULT_DATA_STRING_SIZE + val tempDataFile = createDataOnDisk(size) + writeBenchmarkWithSpill(tempDataFile, size) + tempDataFile.delete() + } + } +} From da0d91c2094a43ac450e0796c147f58094d76ae9 Mon Sep 17 00:00:00 2001 From: Yifei Huang Date: Wed, 27 Feb 2019 13:31:51 -0800 Subject: [PATCH 22/68] add stdev --- .../org/apache/spark/benchmark/Benchmark.scala | 14 ++++++++------ 1 file changed, 8 insertions(+), 6 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/benchmark/Benchmark.scala b/core/src/test/scala/org/apache/spark/benchmark/Benchmark.scala index bb389cdb39dfd..e041de316b9f0 100644 --- a/core/src/test/scala/org/apache/spark/benchmark/Benchmark.scala +++ b/core/src/test/scala/org/apache/spark/benchmark/Benchmark.scala @@ -111,16 +111,17 @@ 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)", - "Per Row(ns)", "Relative") + out.printf("%-40s %16s %12s %13s %10s %16s\n", name + ":", "Best/Avg Time(ms)", "Rate(M/s)", + "Per Row(ns)", "Relative", "Stdev (ms)") out.println("-" * 96) results.zip(benchmarks).foreach { case (result, benchmark) => - out.printf("%-40s %16s %12s %13s %10s\n", + out.printf("%-40s %16s %12s %13s %10s %16s\n", benchmark.name, "%5.0f / %4.0f" format (result.bestMs, result.avgMs), "%10.1f" format result.bestRate, "%6.1f" format (1000 / result.bestRate), - "%3.1fX" format (firstBest / result.bestMs)) + "%3.1fX" format (firstBest / result.bestMs), + "%5.0f" format result.stdevMs) } out.println // scalastyle:on @@ -158,7 +159,8 @@ private[spark] class Benchmark( // scalastyle:on val best = runTimes.min val avg = runTimes.sum / runTimes.size - Result(avg / 1000000.0, num / (best / 1000.0), best / 1000000.0) + val stdev = math.sqrt(runTimes.map(time => math.pow(time - avg, 2)).sum / runTimes.size) + Result(avg / 1000000.0, num / (best / 1000.0), best / 1000000.0, stdev / 1000000.0) } } @@ -191,7 +193,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. From e590917794bf8e37f39f8c0dfef55a7a07b96691 Mon Sep 17 00:00:00 2001 From: Yifei Huang Date: Wed, 27 Feb 2019 14:00:41 -0800 Subject: [PATCH 23/68] cleanup sort a bit --- .../spark/shuffle/sort/SortShuffleWriterBenchmark.scala | 9 +++------ 1 file changed, 3 insertions(+), 6 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/shuffle/sort/SortShuffleWriterBenchmark.scala b/core/src/test/scala/org/apache/spark/shuffle/sort/SortShuffleWriterBenchmark.scala index 43c378a85b869..f2401861a6db0 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/sort/SortShuffleWriterBenchmark.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/sort/SortShuffleWriterBenchmark.scala @@ -118,10 +118,7 @@ object SortShuffleWriterBenchmark extends BenchmarkBase { (invocation: InvocationOnMock) => { val blockId = new TempShuffleBlockId(UUID.randomUUID) val file = new File(tempDir, blockId.name) - // scalastyle:off println - println("created spill file") spillFilesCreated.add(file) - // scalastyle:on println (blockId, file) }) @@ -261,7 +258,7 @@ object SortShuffleWriterBenchmark extends BenchmarkBase { timer.startTiming() shuffleWriter.write(DataIterator(inputFile = dataFile, DEFAULT_DATA_STRING_SIZE)) timer.stopTiming() - assert(spillFilesCreated.size() > 0) + assert(spillFilesCreated.size() == 8) cleanupTempFiles() } @@ -276,7 +273,7 @@ object SortShuffleWriterBenchmark extends BenchmarkBase { timer.startTiming() shuffleWriter.write(DataIterator(inputFile = dataFile, DEFAULT_DATA_STRING_SIZE)) timer.stopTiming() - assert(spillFilesCreated.size() > 0) + assert(spillFilesCreated.size() == 8) cleanupTempFiles() } @@ -286,7 +283,7 @@ object SortShuffleWriterBenchmark extends BenchmarkBase { timer.startTiming() shuffleWriter.write(DataIterator(inputFile = dataFile, DEFAULT_DATA_STRING_SIZE)) timer.stopTiming() - assert(spillFilesCreated.size() > 0) + assert(spillFilesCreated.size() == 8) cleanupTempFiles() } benchmark.run() From cbfdb9934ef555f8b5a8a5e8f61b7da9bd35c42e Mon Sep 17 00:00:00 2001 From: Yifei Huang Date: Wed, 27 Feb 2019 14:28:15 -0800 Subject: [PATCH 24/68] fix stdev text --- .../test/scala/org/apache/spark/benchmark/Benchmark.scala | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/benchmark/Benchmark.scala b/core/src/test/scala/org/apache/spark/benchmark/Benchmark.scala index e041de316b9f0..cb7c1e2a691f8 100644 --- a/core/src/test/scala/org/apache/spark/benchmark/Benchmark.scala +++ b/core/src/test/scala/org/apache/spark/benchmark/Benchmark.scala @@ -111,11 +111,11 @@ 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 %16s\n", name + ":", "Best/Avg Time(ms)", "Rate(M/s)", + out.printf("%-40s %16s %12s %13s %10s %13s\n", name + ":", "Best/Avg Time(ms)", "Rate(M/s)", "Per Row(ns)", "Relative", "Stdev (ms)") - out.println("-" * 96) + out.println("-" * 110) results.zip(benchmarks).foreach { case (result, benchmark) => - out.printf("%-40s %16s %12s %13s %10s %16s\n", + out.printf("%-40s %16s %12s %13s %10s %13s\n", benchmark.name, "%5.0f / %4.0f" format (result.bestMs, result.avgMs), "%10.1f" format result.bestRate, From cbe38c6e2065fdfcca7fcc04e0d0b2289763662b Mon Sep 17 00:00:00 2001 From: Yifei Huang Date: Wed, 27 Feb 2019 14:28:34 -0800 Subject: [PATCH 25/68] fix sort shuffle --- .../apache/spark/shuffle/sort/SortShuffleWriterBenchmark.scala | 1 + 1 file changed, 1 insertion(+) diff --git a/core/src/test/scala/org/apache/spark/shuffle/sort/SortShuffleWriterBenchmark.scala b/core/src/test/scala/org/apache/spark/shuffle/sort/SortShuffleWriterBenchmark.scala index f2401861a6db0..c6d2d80465eaf 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/sort/SortShuffleWriterBenchmark.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/sort/SortShuffleWriterBenchmark.scala @@ -121,6 +121,7 @@ object SortShuffleWriterBenchmark extends BenchmarkBase { spillFilesCreated.add(file) (blockId, file) }) + spillFilesCreated.clear() when(blockManager.getDiskWriter( any[BlockId], From acdda71f5edba8eb878d2a2a4c70b6bcce72e4d4 Mon Sep 17 00:00:00 2001 From: Yifei Huang Date: Thu, 28 Feb 2019 12:33:30 -0800 Subject: [PATCH 26/68] initial code for read side --- .../BlockStoreShuffleReaderBenchmark.scala | 214 ++++++++++++++++++ .../sort/SortShuffleWriterBenchmark.scala | 6 +- 2 files changed, 216 insertions(+), 4 deletions(-) create mode 100644 core/src/test/scala/org/apache/spark/shuffle/BlockStoreShuffleReaderBenchmark.scala diff --git a/core/src/test/scala/org/apache/spark/shuffle/BlockStoreShuffleReaderBenchmark.scala b/core/src/test/scala/org/apache/spark/shuffle/BlockStoreShuffleReaderBenchmark.scala new file mode 100644 index 0000000000000..e4febb3116c97 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/shuffle/BlockStoreShuffleReaderBenchmark.scala @@ -0,0 +1,214 @@ +/* + * 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 + +import java.io.{File, FileOutputStream, OutputStream} +import java.util.concurrent.{Callable, Executors} + +import com.google.common.io.CountingOutputStream +import org.mockito.{Mock, MockitoAnnotations} +import org.mockito.Answers.RETURNS_SMART_NULLS +import org.mockito.Matchers.any +import org.mockito.Mockito.when +import org.mockito.invocation.InvocationOnMock +import scala.util.Random + +import org.apache.spark.{MapOutputTracker, ShuffleDependency, SparkConf, SparkEnv, TaskContext} +import org.apache.spark.benchmark.BenchmarkBase +import org.apache.spark.executor.TaskMetrics +import org.apache.spark.network.BlockTransferService +import org.apache.spark.network.buffer.FileSegmentManagedBuffer +import org.apache.spark.network.netty.SparkTransportConf +import org.apache.spark.network.shuffle.{BlockFetchingListener, DownloadFileManager} +import org.apache.spark.network.util.TransportConf +import org.apache.spark.serializer.{KryoSerializer, SerializerManager} +import org.apache.spark.storage.{BlockManager, BlockManagerId, ShuffleBlockId} +import org.apache.spark.util.Utils + +/** + * Benchmark to measure performance for aggregate primitives. + * {{{ + * To run this benchmark: + * 1. without sbt: bin/spark-submit --class + * 2. build/sbt "sql/test:runMain " + * 3. generate result: SPARK_GENERATE_BENCHMARK_FILES=1 build/sbt "sql/test:runMain " + * Results will be written to "benchmarks/-results.txt". + * }}} + */ +object BlockStoreShuffleReaderBenchmark extends BenchmarkBase { + + + @Mock(answer = RETURNS_SMART_NULLS) private var blockManager: BlockManager = _ + @Mock(answer = RETURNS_SMART_NULLS) private var transferService: BlockTransferService = _ + @Mock(answer = RETURNS_SMART_NULLS) private var dependency: + ShuffleDependency[String, String, String] = _ + @Mock(answer = RETURNS_SMART_NULLS) private var taskContext: TaskContext = _ + @Mock(answer = RETURNS_SMART_NULLS) private var mapOutputTracker: MapOutputTracker = _ + + private val defaultConf: SparkConf = new SparkConf() + .set("spark.shuffle.compress", "false") + .set("spark.shuffle.spill.compress", "false") + private val serializer: KryoSerializer = new KryoSerializer(defaultConf) + private val serializerManager: SerializerManager = new SerializerManager(serializer, defaultConf) + private val execBlockManagerId: BlockManagerId = BlockManagerId("execId", "host", 8000) + private val remoteBlockManagerId: BlockManagerId = BlockManagerId("remote", "remote", 8000) + private val transportConf: TransportConf = + SparkTransportConf.fromSparkConf(defaultConf, "shuffle") + + private var tempDir: File = _ + + private val SHUFFLE_ID: Int = 0 + private val REDUCE_ID: Int = 0 + private val NUM_MAPS: Int = 1 + + private val DEFAULT_DATA_STRING_SIZE = 5 + private val executorPool = Executors.newFixedThreadPool(10) + + + def setup(size: Int, fetchLocal: Boolean): BlockStoreShuffleReader[String, String] = { + MockitoAnnotations.initMocks(this) + SparkEnv.set(new SparkEnv( + "0", + null, + serializer, + null, + serializerManager, + null, + null, + null, + blockManager, + null, + null, + null, + null, + defaultConf + )) + + val shuffleHandle = new BaseShuffleHandle( + shuffleId = SHUFFLE_ID, + numMaps = NUM_MAPS, + dependency = dependency) + + val taskMetrics = new TaskMetrics + when(taskContext.taskMetrics()).thenReturn(taskMetrics) + + when(blockManager.shuffleClient).thenReturn(transferService) + when(dependency.serializer).thenReturn(serializer) + when(blockManager.blockManagerId).thenReturn(execBlockManagerId) + when(mapOutputTracker.getMapSizesByExecutorId(SHUFFLE_ID, REDUCE_ID, REDUCE_ID + 1)) + .thenReturn { + val shuffleBlockIdsAndSizes = (0 until NUM_MAPS).map { mapId => + val shuffleBlockId = ShuffleBlockId(SHUFFLE_ID, mapId, REDUCE_ID) + (shuffleBlockId, DEFAULT_DATA_STRING_SIZE * size.toLong) + } + // TODO: configurable + Seq((remoteBlockManagerId, shuffleBlockIdsAndSizes)).toIterator + } + + tempDir = Utils.createTempDir(null, "shuffle") + + if (fetchLocal) { + // to do + } else { + when(transferService.fetchBlocks( + any[String], + any[Int], + any[String], + any[Array[String]], + any[BlockFetchingListener], + any[DownloadFileManager] + )).thenAnswer((invocation: InvocationOnMock) => { + val blocks = invocation.getArguments()(3).asInstanceOf[Array[String]] + val listener = invocation.getArguments()(4).asInstanceOf[BlockFetchingListener] + + // TODO: do this in parallel? + for (blockId <- blocks) { + val generatedFile = generateDataOnDisk(size) + listener.onBlockFetchSuccess(blockId, new FileSegmentManagedBuffer( + transportConf, + generatedFile._1, + 0, + generatedFile._2 + )) +// executorPool.submit(new Callable[Unit] { +// override def call(): Unit = { +// val fileGenerated = generateDataOnDisk(size) +// listener.onBlockFetchSuccess(blockId, new FileSegmentManagedBuffer( +// transportConf, +// fileGenerated, +// 0, +// size.toLong * DEFAULT_DATA_STRING_SIZE +// )) +// } +// }) + } + }) + } + + // TODO: use aggregation + sort + when(dependency.aggregator).thenReturn(Option.empty) + when(dependency.keyOrdering).thenReturn(Option.empty) + + new BlockStoreShuffleReader[String, String]( + shuffleHandle, + 0, + 1, + taskContext, + taskMetrics.createTempShuffleReadMetrics(), + serializerManager, + blockManager, + mapOutputTracker + ) + } + + def generateDataOnDisk(size: Int): (File, Long) = { + // scalastyle:off println + val tempDataFile: File = File.createTempFile("test-data", "", tempDir) + println("Generating test data with num records: " + size) + val random = new Random(123) + val dataOutput = new FileOutputStream(tempDataFile) + val coutingOutput = new CountingOutputStream(dataOutput) + val serializedOutput = serializer.newInstance().serializeStream(coutingOutput) + try { + (1 to size).foreach { i => { + if (i % 1000000 == 0) { + println("Wrote " + i + " test data points") + } + val x = random.alphanumeric.take(DEFAULT_DATA_STRING_SIZE).mkString + serializedOutput.writeKey(x) + serializedOutput.writeValue(x) + }} + } + finally { + serializedOutput.close() + } + (tempDataFile, coutingOutput.getCount) + // scalastyle:off println + } + + override def runBenchmarkSuite(mainArgs: Array[String]): Unit = { + val reader = setup(10, false) + // scalastyle:off println + println(reader.read().length) + // scalastyle:on println +// assert(reader.read().length == 10 * NUM_MAPS) + + runBenchmark("SortShuffleWriter writer") { + // todo + } + } +} diff --git a/core/src/test/scala/org/apache/spark/shuffle/sort/SortShuffleWriterBenchmark.scala b/core/src/test/scala/org/apache/spark/shuffle/sort/SortShuffleWriterBenchmark.scala index c6d2d80465eaf..7a6f1bf2c7a2d 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/sort/SortShuffleWriterBenchmark.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/sort/SortShuffleWriterBenchmark.scala @@ -28,8 +28,6 @@ import org.mockito.Matchers.{any, anyInt} import org.mockito.Mockito.{doAnswer, when} import org.mockito.invocation.InvocationOnMock import org.mockito.stubbing.Answer -import scala.collection.mutable -import scala.collection.mutable.ArrayBuffer import scala.util.Random import org.apache.spark.{Aggregator, HashPartitioner, ShuffleDependency, SparkConf, SparkEnv, TaskContext} @@ -65,8 +63,8 @@ object SortShuffleWriterBenchmark extends BenchmarkBase { private val spillFilesCreated: util.LinkedList[File] = new util.LinkedList[File] private val partitioner: HashPartitioner = new HashPartitioner(10) private val defaultConf: SparkConf = new SparkConf() - private val serializer = new KryoSerializer(defaultConf) - private val serializerManager = new SerializerManager(serializer, defaultConf) + private val serializer: KryoSerializer = new KryoSerializer(defaultConf) + private val serializerManager: SerializerManager = new SerializerManager(serializer, defaultConf) private var memoryManager: TestMemoryManager = new TestMemoryManager(defaultConf) private var taskMemoryManager: TaskMemoryManager = new TaskMemoryManager(memoryManager, 0) From fd7a7c56e9c816687a8adc6b3fc3983d5c0e11a6 Mon Sep 17 00:00:00 2001 From: Yifei Huang Date: Thu, 28 Feb 2019 13:30:10 -0800 Subject: [PATCH 27/68] format --- .../org/apache/spark/benchmark/Benchmark.scala | 15 ++++++++------- 1 file changed, 8 insertions(+), 7 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/benchmark/Benchmark.scala b/core/src/test/scala/org/apache/spark/benchmark/Benchmark.scala index cb7c1e2a691f8..59ff50e8d70e4 100644 --- a/core/src/test/scala/org/apache/spark/benchmark/Benchmark.scala +++ b/core/src/test/scala/org/apache/spark/benchmark/Benchmark.scala @@ -111,17 +111,18 @@ 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 %13s\n", name + ":", "Best/Avg Time(ms)", "Rate(M/s)", - "Per Row(ns)", "Relative", "Stdev (ms)") - out.println("-" * 110) + 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("-" * 120) results.zip(benchmarks).foreach { case (result, benchmark) => - out.printf("%-40s %16s %12s %13s %10s %13s\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), - "%5.0f" format result.stdevMs) + "%3.1fX" format (firstBest / result.bestMs)) } out.println // scalastyle:on From d82618b833d22a568ddea3e6cc1800000b97fa83 Mon Sep 17 00:00:00 2001 From: Yifei Huang Date: Thu, 28 Feb 2019 13:33:34 -0800 Subject: [PATCH 28/68] use times and sample stdev --- .../src/test/scala/org/apache/spark/benchmark/Benchmark.scala | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/core/src/test/scala/org/apache/spark/benchmark/Benchmark.scala b/core/src/test/scala/org/apache/spark/benchmark/Benchmark.scala index 59ff50e8d70e4..206eba5a0534e 100644 --- a/core/src/test/scala/org/apache/spark/benchmark/Benchmark.scala +++ b/core/src/test/scala/org/apache/spark/benchmark/Benchmark.scala @@ -160,7 +160,9 @@ private[spark] class Benchmark( // scalastyle:on val best = runTimes.min val avg = runTimes.sum / runTimes.size - val stdev = math.sqrt(runTimes.map(time => math.pow(time - avg, 2)).sum / runTimes.size) + 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) } } From 610ea1daaf2e376ae416991ffcf433cb64681e9c Mon Sep 17 00:00:00 2001 From: Yifei Huang Date: Thu, 28 Feb 2019 13:41:06 -0800 Subject: [PATCH 29/68] add assert for at least one iteration --- core/src/test/scala/org/apache/spark/benchmark/Benchmark.scala | 1 + 1 file changed, 1 insertion(+) diff --git a/core/src/test/scala/org/apache/spark/benchmark/Benchmark.scala b/core/src/test/scala/org/apache/spark/benchmark/Benchmark.scala index 206eba5a0534e..5fdb0d289b6b1 100644 --- a/core/src/test/scala/org/apache/spark/benchmark/Benchmark.scala +++ b/core/src/test/scala/org/apache/spark/benchmark/Benchmark.scala @@ -158,6 +158,7 @@ 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 val stdev = if (runTimes.size > 1) { From 295d7f31c30e842de05341f534d74ff5493a3cfe Mon Sep 17 00:00:00 2001 From: Yifei Huang Date: Thu, 28 Feb 2019 22:25:58 -0800 Subject: [PATCH 30/68] cleanup shuffle write to use fewer mocks and single base interface --- ...ypassMergeSortShuffleWriterBenchmark.scala | 128 ++--------- .../sort/ShuffleWriterBenchmarkBase.scala | 203 +++++++++++++++++ .../sort/SortShuffleWriterBenchmark.scala | 209 +++--------------- .../sort/UnsafeShuffleWriterBenchmark.scala | 185 ++-------------- 4 files changed, 269 insertions(+), 456 deletions(-) create mode 100644 core/src/test/scala/org/apache/spark/shuffle/sort/ShuffleWriterBenchmarkBase.scala diff --git a/core/src/test/scala/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriterBenchmark.scala b/core/src/test/scala/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriterBenchmark.scala index c003dbca0429a..def740c1d5cda 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriterBenchmark.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriterBenchmark.scala @@ -17,26 +17,10 @@ package org.apache.spark.shuffle.sort -import java.io.File -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.Matchers.{any, anyInt} -import org.mockito.Mockito.{doAnswer, when} -import org.mockito.invocation.InvocationOnMock -import org.mockito.stubbing.Answer -import scala.collection.mutable import scala.util.Random -import org.apache.spark.{HashPartitioner, ShuffleDependency, SparkConf, TaskContext} -import org.apache.spark.benchmark.{Benchmark, BenchmarkBase} -import org.apache.spark.executor.{ShuffleWriteMetrics, TaskMetrics} -import org.apache.spark.serializer.{JavaSerializer, SerializerInstance, SerializerManager} -import org.apache.spark.shuffle.IndexShuffleBlockResolver -import org.apache.spark.storage.{BlockId, BlockManager, DiskBlockManager, DiskBlockObjectWriter, TempShuffleBlockId} -import org.apache.spark.util.Utils +import org.apache.spark.SparkConf +import org.apache.spark.benchmark.Benchmark /** * Benchmark to measure performance for aggregate primitives. @@ -48,109 +32,33 @@ import org.apache.spark.util.Utils * Results will be written to "benchmarks/-results.txt". * }}} */ -object BypassMergeSortShuffleWriterBenchmark extends BenchmarkBase { - - @Mock(answer = RETURNS_SMART_NULLS) private var blockManager: BlockManager = _ - @Mock(answer = RETURNS_SMART_NULLS) private var diskBlockManager: DiskBlockManager = _ - @Mock(answer = RETURNS_SMART_NULLS) private var taskContext: TaskContext = _ - @Mock(answer = RETURNS_SMART_NULLS) private var blockResolver: IndexShuffleBlockResolver = _ - @Mock(answer = RETURNS_SMART_NULLS) private var dependency: - ShuffleDependency[String, String, String] = _ +object BypassMergeSortShuffleWriterBenchmark extends ShuffleWriterBenchmarkBase(false) { - private var tempDir: File = _ - private var shuffleHandle: BypassMergeSortShuffleHandle[String, String] = _ - private val blockIdToFileMap: mutable.Map[BlockId, File] = new mutable.HashMap[BlockId, File] - private val partitioner: HashPartitioner = new HashPartitioner(10) - private val defaultConf: SparkConf = new SparkConf(loadDefaults = false) - private val javaSerializer: JavaSerializer = new JavaSerializer(defaultConf) + private var shuffleHandle: BypassMergeSortShuffleHandle[String, String] = + new BypassMergeSortShuffleHandle[String, String]( + shuffleId = 0, + numMaps = 1, + dependency) private val MIN_NUM_ITERS = 10 - def setup(transferTo: Boolean): BypassMergeSortShuffleWriter[String, String] = { - MockitoAnnotations.initMocks(this) + def constructWriter(transferTo: Boolean): BypassMergeSortShuffleWriter[String, String] = { val conf = new SparkConf(loadDefaults = false) conf.set("spark.file.transferTo", String.valueOf(transferTo)) conf.set("spark.shuffle.file.buffer", "32k") - if (shuffleHandle == null) { - shuffleHandle = new BypassMergeSortShuffleHandle[String, String]( - shuffleId = 0, - numMaps = 1, - dependency = dependency - ) - } - - val taskMetrics = new TaskMetrics - when(dependency.partitioner).thenReturn(partitioner) - when(dependency.serializer).thenReturn(javaSerializer) - when(dependency.shuffleId).thenReturn(0) - - // Create the temporary directory to write local shuffle and temp files - tempDir = Utils.createTempDir() - val outputFile = File.createTempFile("shuffle", null, tempDir) - // Final mapper data file output - when(blockResolver.getDataFile(0, 0)).thenReturn(outputFile) - - // Create the temporary writers (backed by files), one for each partition. - when(blockManager.diskBlockManager).thenReturn(diskBlockManager) - when(diskBlockManager.createTempShuffleBlock()).thenAnswer( - (invocation: InvocationOnMock) => { - val blockId = new TempShuffleBlockId(UUID.randomUUID) - val file = new File(tempDir, blockId.name) - blockIdToFileMap.put(blockId, file) - (blockId, file) - }) - when(blockManager.getDiskWriter( - any[BlockId], - any[File], - any[SerializerInstance], - anyInt(), - any[ShuffleWriteMetrics] - )).thenAnswer(new Answer[DiskBlockObjectWriter] { - override def answer(invocation: InvocationOnMock): DiskBlockObjectWriter = { - val args = invocation.getArguments - val manager = new SerializerManager(javaSerializer, conf) - new DiskBlockObjectWriter( - args(1).asInstanceOf[File], - manager, - args(2).asInstanceOf[SerializerInstance], - args(3).asInstanceOf[Int], - syncWrites = false, - args(4).asInstanceOf[ShuffleWriteMetrics], - blockId = args(0).asInstanceOf[BlockId] - ) - } - }) - - // writing the index file - doAnswer(new Answer[Void] { - def answer(invocationOnMock: InvocationOnMock): Void = { - val tmp: File = invocationOnMock.getArguments()(3).asInstanceOf[File] - if (tmp != null) { - outputFile.delete - tmp.renameTo(outputFile) - } - null - } - }).when(blockResolver) - .writeIndexFileAndCommit(anyInt, anyInt, any(classOf[Array[Long]]), any(classOf[File])) - val shuffleWriter = new BypassMergeSortShuffleWriter[String, String]( blockManager, blockResolver, shuffleHandle, 0, conf, - taskMetrics.shuffleWriteMetrics + taskContext.taskMetrics().shuffleWriteMetrics ) shuffleWriter } - def cleanupTempFiles(): Unit = { - FileUtils.deleteDirectory(tempDir) - } - def writeBenchmarkWithLargeDataset(): Unit = { val size = 10000000 val random = new Random(123) @@ -163,19 +71,18 @@ object BypassMergeSortShuffleWriterBenchmark extends BenchmarkBase { size, minNumIters = MIN_NUM_ITERS, output = output) - benchmark.addTimerCase("without transferTo") { timer => - val shuffleWriter = setup(false) + + addBenchmarkCase(benchmark, "without transferTo") { timer => + val shuffleWriter = constructWriter(false) timer.startTiming() shuffleWriter.write(data.iterator) timer.stopTiming() - cleanupTempFiles() } - benchmark.addTimerCase("with transferTo") { timer => - val shuffleWriter = setup(true) + addBenchmarkCase(benchmark, "with transferTo") { timer => + val shuffleWriter = constructWriter(false) timer.startTiming() shuffleWriter.write(data.iterator) timer.stopTiming() - cleanupTempFiles() } benchmark.run() } @@ -191,12 +98,11 @@ object BypassMergeSortShuffleWriterBenchmark extends BenchmarkBase { size, minNumIters = MIN_NUM_ITERS, output = output) - benchmark.addTimerCase("small dataset without spills on disk") { timer => - val shuffleWriter = setup(false) + addBenchmarkCase(benchmark, "small dataset without spills on disk") { timer => + val shuffleWriter = constructWriter(false) timer.startTiming() shuffleWriter.write(data.iterator) timer.stopTiming() - cleanupTempFiles() } benchmark.run() } diff --git a/core/src/test/scala/org/apache/spark/shuffle/sort/ShuffleWriterBenchmarkBase.scala b/core/src/test/scala/org/apache/spark/shuffle/sort/ShuffleWriterBenchmarkBase.scala new file mode 100644 index 0000000000000..d18c0896e77c7 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/shuffle/sort/ShuffleWriterBenchmarkBase.scala @@ -0,0 +1,203 @@ +/* + * 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, File, FileInputStream, FileOutputStream} +import java.nio.channels.ReadableByteChannel +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.Mockito.when +import scala.collection.mutable +import scala.collection.mutable.ArrayBuffer +import scala.concurrent.Future +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.{RpcAddress, RpcEndpoint, RpcEndpointRef, RpcEnv, RpcEnvFileServer} +import org.apache.spark.serializer.{JavaSerializer, 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(useKryoSerializer: Boolean) extends BenchmarkBase { + + private 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 = _ + + protected val defaultConf: SparkConf = new SparkConf(loadDefaults = false) + protected val serializer: Serializer = if (useKryoSerializer) { + new KryoSerializer(defaultConf) + } else { + new JavaSerializer(defaultConf) + } + protected val partitioner: HashPartitioner = new HashPartitioner(10) + protected val serializerManager: SerializerManager = + new SerializerManager(serializer, defaultConf) + + private var rpcEnv: RpcEnv = new RpcEnv(defaultConf) { + override def endpointRef(endpoint: RpcEndpoint): RpcEndpointRef = { null } + override def address: RpcAddress = null + override def setupEndpoint(name: String, endpoint: RpcEndpoint): RpcEndpointRef = { null } + override def asyncSetupEndpointRefByURI(uri: String): Future[RpcEndpointRef] = { null } + override def stop(endpoint: RpcEndpointRef): Unit = { } + override def shutdown(): Unit = { } + override def awaitTermination(): Unit = { } + override def deserialize[T](deserializationAction: () => T): T = { deserializationAction() } + override def fileServer: RpcEnvFileServer = { null } + override def openChannel(uri: String): ReadableByteChannel = { null } + } + + 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) + } + + protected var tempDir: File = _ + protected var shuffleMetrics: TaskMetrics = _ + + 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) + shuffleMetrics = new TaskMetrics + when(taskContext.taskMetrics()).thenReturn(shuffleMetrics) + + 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 private ( + private val inputStream: BufferedInputStream, + private val buffer: Array[Byte]) extends Iterator[Product2[String, String]] { + override def hasNext: Boolean = { + inputStream.available() > 0 + } + + override def next(): Product2[String, String] = { + val read = inputStream.read(buffer) + assert(read == buffer.length) + val string = buffer.mkString + (string, string) + } + } + + protected object DataIterator { + def apply(inputFile: File, bufferSize: Int): DataIterator = { + val inputStream = new BufferedInputStream( + new FileInputStream(inputFile), DEFAULT_DATA_STRING_SIZE) + val buffer = new Array[Byte](DEFAULT_DATA_STRING_SIZE) + new DataIterator(inputStream, buffer) + } + } + + def createDataInMemory(size: Int): Array[(String, String)] = { + val random = new Random(123) + (1 to size).map { i => { + val x = random.alphanumeric.take(DEFAULT_DATA_STRING_SIZE).mkString + Tuple2(x, x) + } }.toArray + } + + def createDataOnDisk(size: Int): File = { + // scalastyle:off println + println("Generating test data with num records: " + size) + val tempDataFile = File.createTempFile("test-data", "") + val random = new Random(123) + val dataOutput = new FileOutputStream(tempDataFile) + try { + (1 to size).foreach { i => { + if (i % 1000000 == 0) { + println("Wrote " + i + " test data points") + } + val x = random.alphanumeric.take(DEFAULT_DATA_STRING_SIZE).mkString + dataOutput.write(x.getBytes) + }} + } + finally { + dataOutput.close() + } + tempDataFile + // scalastyle:off println + } + +} diff --git a/core/src/test/scala/org/apache/spark/shuffle/sort/SortShuffleWriterBenchmark.scala b/core/src/test/scala/org/apache/spark/shuffle/sort/SortShuffleWriterBenchmark.scala index 7a6f1bf2c7a2d..e2428f846927f 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/sort/SortShuffleWriterBenchmark.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/sort/SortShuffleWriterBenchmark.scala @@ -17,27 +17,11 @@ package org.apache.spark.shuffle.sort -import java.io.{BufferedInputStream, File, FileInputStream, FileOutputStream} -import java.util -import java.util.UUID +import org.mockito.Mockito.when -import org.apache.commons.io.FileUtils -import org.mockito.{Mock, MockitoAnnotations} -import org.mockito.Answers.RETURNS_SMART_NULLS -import org.mockito.Matchers.{any, anyInt} -import org.mockito.Mockito.{doAnswer, when} -import org.mockito.invocation.InvocationOnMock -import org.mockito.stubbing.Answer -import scala.util.Random - -import org.apache.spark.{Aggregator, HashPartitioner, ShuffleDependency, SparkConf, SparkEnv, TaskContext} -import org.apache.spark.benchmark.{Benchmark, BenchmarkBase} -import org.apache.spark.executor.{ShuffleWriteMetrics, TaskMetrics} -import org.apache.spark.memory.{TaskMemoryManager, TestMemoryManager} -import org.apache.spark.serializer.{KryoSerializer, SerializerInstance, SerializerManager} -import org.apache.spark.shuffle.{BaseShuffleHandle, IndexShuffleBlockResolver} -import org.apache.spark.storage.{BlockId, BlockManager, DiskBlockManager, DiskBlockObjectWriter, TempShuffleBlockId} -import org.apache.spark.util.Utils +import org.apache.spark.{Aggregator, SparkEnv} +import org.apache.spark.benchmark.Benchmark +import org.apache.spark.shuffle.BaseShuffleHandle /** * Benchmark to measure performance for aggregate primitives. @@ -49,31 +33,20 @@ import org.apache.spark.util.Utils * Results will be written to "benchmarks/-results.txt". * }}} */ -object SortShuffleWriterBenchmark extends BenchmarkBase { - - @Mock(answer = RETURNS_SMART_NULLS) private var blockManager: BlockManager = _ - @Mock(answer = RETURNS_SMART_NULLS) private var blockResolver: IndexShuffleBlockResolver = _ - @Mock(answer = RETURNS_SMART_NULLS) private var diskBlockManager: DiskBlockManager = _ - @Mock(answer = RETURNS_SMART_NULLS) private var taskContext: TaskContext = _ - @Mock(answer = RETURNS_SMART_NULLS) private var dependency: - ShuffleDependency[String, String, String] = _ +object SortShuffleWriterBenchmark extends ShuffleWriterBenchmarkBase(true) { - private var tempDir: File = _ - private var shuffleHandle: BaseShuffleHandle[String, String, String] = _ - private val spillFilesCreated: util.LinkedList[File] = new util.LinkedList[File] - private val partitioner: HashPartitioner = new HashPartitioner(10) - private val defaultConf: SparkConf = new SparkConf() - private val serializer: KryoSerializer = new KryoSerializer(defaultConf) - private val serializerManager: SerializerManager = new SerializerManager(serializer, defaultConf) - private var memoryManager: TestMemoryManager = new TestMemoryManager(defaultConf) - private var taskMemoryManager: TaskMemoryManager = new TaskMemoryManager(memoryManager, 0) + private var shuffleHandle: BaseShuffleHandle[String, String, String] = + new BaseShuffleHandle( + shuffleId = 0, + numMaps = 1, + dependency = dependency) private val DEFAULT_DATA_STRING_SIZE = 5 private val MIN_NUM_ITERS = 5 - def setup(aggregator: Option[Aggregator[String, String, String]], - sorter: Option[Ordering[String]]): SortShuffleWriter[String, String, String] = { - MockitoAnnotations.initMocks(this) + def constructWriter(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 val defaultSparkEnv = SparkEnv.get SparkEnv.set(new SparkEnv( @@ -92,14 +65,7 @@ object SortShuffleWriterBenchmark extends BenchmarkBase { null, defaultConf )) - shuffleHandle = new BaseShuffleHandle( - shuffleId = 0, - numMaps = 1, - dependency = dependency) - when(dependency.partitioner).thenReturn(partitioner) - when(dependency.serializer).thenReturn(serializer) - when(dependency.shuffleId).thenReturn(0) if (aggregator.isEmpty && sorter.isEmpty) { when(dependency.mapSideCombine).thenReturn(false) } else { @@ -108,57 +74,6 @@ object SortShuffleWriterBenchmark extends BenchmarkBase { when(dependency.keyOrdering).thenReturn(sorter) } - tempDir = Utils.createTempDir() - val outputFile = File.createTempFile("shuffle", null, tempDir) - when(blockResolver.getDataFile(0, 0)).thenReturn(outputFile) - when(blockManager.diskBlockManager).thenReturn(diskBlockManager) - when(diskBlockManager.createTempShuffleBlock()).thenAnswer( - (invocation: InvocationOnMock) => { - val blockId = new TempShuffleBlockId(UUID.randomUUID) - val file = new File(tempDir, blockId.name) - spillFilesCreated.add(file) - (blockId, file) - }) - spillFilesCreated.clear() - - when(blockManager.getDiskWriter( - any[BlockId], - any[File], - any[SerializerInstance], - anyInt(), - any[ShuffleWriteMetrics] - )).thenAnswer(new Answer[DiskBlockObjectWriter] { - override def answer(invocation: InvocationOnMock): DiskBlockObjectWriter = { - val args = invocation.getArguments - val manager = serializerManager - new DiskBlockObjectWriter( - args(1).asInstanceOf[File], - manager, - args(2).asInstanceOf[SerializerInstance], - args(3).asInstanceOf[Int], - syncWrites = false, - args(4).asInstanceOf[ShuffleWriteMetrics], - blockId = args(0).asInstanceOf[BlockId] - ) - } - }) - - doAnswer(new Answer[Void] { - def answer(invocationOnMock: InvocationOnMock): Void = { - val tmp: File = invocationOnMock.getArguments()(3).asInstanceOf[File] - if (tmp != null) { - outputFile.delete - tmp.renameTo(outputFile) - } - null - } - }).when(blockResolver) - .writeIndexFileAndCommit(anyInt, anyInt, any(classOf[Array[Long]]), any(classOf[File])) - - val taskMetrics = new TaskMetrics - when(taskContext.taskMetrics()).thenReturn(taskMetrics) - - memoryManager.limit(PackedRecordPointer.MAXIMUM_PAGE_SIZE_BYTES) when(taskContext.taskMemoryManager()).thenReturn(taskMemoryManager) val shuffleWriter = new SortShuffleWriter[String, String, String]( @@ -170,95 +85,38 @@ object SortShuffleWriterBenchmark extends BenchmarkBase { shuffleWriter } - def cleanupTempFiles(): Unit = { - FileUtils.deleteDirectory(tempDir) - } - - def createDataInMemory(size: Int): Array[(String, String)] = { - val random = new Random(123) - (1 to size).map { i => { - val x = random.alphanumeric.take(DEFAULT_DATA_STRING_SIZE).mkString - Tuple2(x, x) - } }.toArray - } - - def createDataOnDisk(size: Int): File = { - // scalastyle:off println - val tempDataFile: File = File.createTempFile("test-data", "") - println("Generating test data with num records: " + size) - val random = new Random(123) - val dataOutput = new FileOutputStream(tempDataFile) - try { - (1 to size).foreach { i => { - if (i % 1000000 == 0) { - println("Wrote " + i + " test data points") - } - val x = random.alphanumeric.take(DEFAULT_DATA_STRING_SIZE).mkString - dataOutput.write(x.getBytes) - }} - } - finally { - dataOutput.close() - } - tempDataFile - // scalastyle:off println - } - - private class DataIterator private ( - private val inputStream: BufferedInputStream, - private val buffer: Array[Byte]) extends Iterator[Product2[String, String]] { - override def hasNext: Boolean = { - inputStream.available() > 0 - } - - override def next(): Product2[String, String] = { - val read = inputStream.read(buffer) - assert(read == buffer.length) - val string = buffer.mkString - (string, string) - } - } - - private object DataIterator { - def apply(inputFile: File, bufferSize: Int): DataIterator = { - val inputStream = new BufferedInputStream( - new FileInputStream(inputFile), DEFAULT_DATA_STRING_SIZE) - val buffer = new Array[Byte](DEFAULT_DATA_STRING_SIZE) - new DataIterator(inputStream, buffer) - } - } - def writeBenchmarkWithSmallDataset(): Unit = { val size = 1000 val benchmark = new Benchmark("SortShuffleWriter with spills", size, minNumIters = MIN_NUM_ITERS, output = output) - benchmark.addTimerCase("small dataset without spills") { timer => - val writer = setup(Option.empty, Option.empty) + addBenchmarkCase(benchmark, "small dataset without spills") { timer => + val writer = constructWriter(Option.empty, Option.empty) val array = createDataInMemory(1000) timer.startTiming() writer.write(array.iterator) timer.stopTiming() - assert(spillFilesCreated.size() == 0) - cleanupTempFiles() + assert(tempFilesCreated.length == 0) } benchmark.run() } - def writeBenchmarkWithSpill(dataFile: File, size: Int): Unit = { + def writeBenchmarkWithSpill(): Unit = { + val size = PackedRecordPointer.MAXIMUM_PAGE_SIZE_BYTES/4/DEFAULT_DATA_STRING_SIZE + val dataFile = createDataOnDisk(size) + val benchmark = new Benchmark("SortShuffleWriter with spills", size, minNumIters = MIN_NUM_ITERS, output = output, outputPerIteration = true) - benchmark.addTimerCase("no map side combine") { timer => - val shuffleWriter = setup(Option.empty, Option.empty) + addBenchmarkCase(benchmark, "no map side combine") { timer => + val shuffleWriter = constructWriter(Option.empty, Option.empty) timer.startTiming() shuffleWriter.write(DataIterator(inputFile = dataFile, DEFAULT_DATA_STRING_SIZE)) timer.stopTiming() - assert(spillFilesCreated.size() == 8) - cleanupTempFiles() + assert(tempFilesCreated.length == 8) } def createCombiner(i: String): String = i @@ -267,35 +125,30 @@ object SortShuffleWriterBenchmark extends BenchmarkBase { if (Ordering.String.compare(i, j) > 0) i else j val aggregator = new Aggregator[String, String, String](createCombiner, mergeValue, mergeCombiners) - benchmark.addTimerCase("with map side aggregation") { timer => - val shuffleWriter = setup(Some(aggregator), Option.empty) + addBenchmarkCase(benchmark, "with map side aggregation") { timer => + val shuffleWriter = constructWriter(Some(aggregator), Option.empty) timer.startTiming() shuffleWriter.write(DataIterator(inputFile = dataFile, DEFAULT_DATA_STRING_SIZE)) timer.stopTiming() - assert(spillFilesCreated.size() == 8) - cleanupTempFiles() + assert(tempFilesCreated.length == 8) } val sorter = Ordering.String - benchmark.addTimerCase("with map side sort") { timer => - val shuffleWriter = setup(Option.empty, Some(sorter)) + addBenchmarkCase(benchmark, "with map side sort") { timer => + val shuffleWriter = constructWriter(Option.empty, Some(sorter)) timer.startTiming() shuffleWriter.write(DataIterator(inputFile = dataFile, DEFAULT_DATA_STRING_SIZE)) timer.stopTiming() - assert(spillFilesCreated.size() == 8) - cleanupTempFiles() + assert(tempFilesCreated.length == 8) } benchmark.run() + dataFile.delete() } override def runBenchmarkSuite(mainArgs: Array[String]): Unit = { - runBenchmark("SortShuffleWriter writer") { writeBenchmarkWithSmallDataset() - val size = PackedRecordPointer.MAXIMUM_PAGE_SIZE_BYTES/4/DEFAULT_DATA_STRING_SIZE - val tempDataFile = createDataOnDisk(size) - writeBenchmarkWithSpill(tempDataFile, size) - tempDataFile.delete() + writeBenchmarkWithSpill() } } } diff --git a/core/src/test/scala/org/apache/spark/shuffle/sort/UnsafeShuffleWriterBenchmark.scala b/core/src/test/scala/org/apache/spark/shuffle/sort/UnsafeShuffleWriterBenchmark.scala index 7f41b44407474..f34dea0c82e51 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/sort/UnsafeShuffleWriterBenchmark.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/sort/UnsafeShuffleWriterBenchmark.scala @@ -17,25 +17,11 @@ package org.apache.spark.shuffle.sort import java.io.{BufferedInputStream, File, FileInputStream, FileOutputStream} -import java.util -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.Matchers.{any, anyInt} -import org.mockito.Mockito.{doAnswer, when} -import org.mockito.invocation.InvocationOnMock import scala.util.Random -import org.apache.spark.{HashPartitioner, ShuffleDependency, SparkConf, TaskContext} -import org.apache.spark.benchmark.{Benchmark, BenchmarkBase} -import org.apache.spark.executor.{ShuffleWriteMetrics, TaskMetrics} -import org.apache.spark.memory.{TaskMemoryManager, TestMemoryManager} -import org.apache.spark.serializer.{KryoSerializer, SerializerInstance, SerializerManager} -import org.apache.spark.shuffle.IndexShuffleBlockResolver -import org.apache.spark.storage.{BlockId, BlockManager, DiskBlockManager, DiskBlockObjectWriter, TempShuffleBlockId} -import org.apache.spark.util.Utils +import org.apache.spark.SparkConf +import org.apache.spark.benchmark.Benchmark /** * Benchmark to measure performance for aggregate primitives. @@ -47,176 +33,43 @@ import org.apache.spark.util.Utils * Results will be written to "benchmarks/-results.txt". * }}} */ -object UnsafeShuffleWriterBenchmark extends BenchmarkBase { +object UnsafeShuffleWriterBenchmark extends ShuffleWriterBenchmarkBase(true) { - @Mock(answer = RETURNS_SMART_NULLS) private var blockManager: BlockManager = _ - @Mock(answer = RETURNS_SMART_NULLS) private var shuffleBlockResolver: - IndexShuffleBlockResolver = _ - @Mock(answer = RETURNS_SMART_NULLS) private var dependency: - ShuffleDependency[String, String, String] = _ - @Mock(answer = RETURNS_SMART_NULLS) private[sort] val taskContext: TaskContext = null - @Mock(answer = RETURNS_SMART_NULLS) private[sort] val diskBlockManager: DiskBlockManager = null - - private[sort] val serializer = new KryoSerializer(new SparkConf) - private[sort] val hashPartitioner = new HashPartitioner(10) - private[sort] val spillFilesCreated: util.LinkedList[File] = new util.LinkedList[File] - private var tempDataFile: File = File.createTempFile("test-data", "") - private var tempDir: File = _ - private var shuffleHandle: SerializedShuffleHandle[String, String] = _ - private var memoryManager: TestMemoryManager = _ - private var taskMemoryManager: TaskMemoryManager = _ + private val shuffleHandle: SerializedShuffleHandle[String, String] = + new SerializedShuffleHandle[String, String](0, 0, this.dependency) private val DEFAULT_DATA_STRING_SIZE = 5 private val MIN_NUM_ITERS = 5 - def setup(transferTo: Boolean): UnsafeShuffleWriter[String, String] = { - MockitoAnnotations.initMocks(this) + def constructWriter(transferTo: Boolean): UnsafeShuffleWriter[String, String] = { val conf = new SparkConf(loadDefaults = false) conf.set("spark.file.transferTo", String.valueOf(transferTo)) - memoryManager = new TestMemoryManager(conf) - memoryManager.limit(PackedRecordPointer.MAXIMUM_PAGE_SIZE_BYTES) - taskMemoryManager = new TaskMemoryManager(memoryManager, 0) - if (shuffleHandle == null) { - shuffleHandle = new SerializedShuffleHandle[String, String](0, 0, dependency) - } - val taskMetrics = new TaskMetrics - - - tempDir = Utils.createTempDir(null, "test") - val mergedOutputFile = File.createTempFile("shuffle", "", tempDir) - - // copied from UnsafeShuffleWriterSuite - // Some tests will override this manager because they change the configuration. This is a - // default for tests that don't need a specific one. - val manager: SerializerManager = new SerializerManager(serializer, conf) - when(blockManager.serializerManager).thenReturn(manager) - - when(blockManager.diskBlockManager).thenReturn(diskBlockManager) - when(blockManager.getDiskWriter( - any[BlockId], - any[File], - any[SerializerInstance], - any[Int], - any[ShuffleWriteMetrics])) - .thenAnswer((invocationOnMock: InvocationOnMock) => { - val args = invocationOnMock.getArguments - new DiskBlockObjectWriter( - args(1).asInstanceOf[File], - blockManager.serializerManager, - args(2).asInstanceOf[SerializerInstance], - args(3).asInstanceOf[java.lang.Integer], - false, - args(4).asInstanceOf[ShuffleWriteMetrics], - args(0).asInstanceOf[BlockId]) - }) - - when(shuffleBlockResolver.getDataFile(anyInt, anyInt)).thenReturn(mergedOutputFile) - doAnswer((invocationOnMock: InvocationOnMock) => { - val tmp: File = invocationOnMock.getArguments()(3).asInstanceOf[File] - mergedOutputFile.delete - tmp.renameTo(mergedOutputFile) - }).when(shuffleBlockResolver).writeIndexFileAndCommit( - any[Int], - any[Int], - any[Array[Long]], - any[File]) - when(diskBlockManager.createTempShuffleBlock) - .thenAnswer((invocationOnMock: InvocationOnMock) => { - val blockId: TempShuffleBlockId = new TempShuffleBlockId(UUID.randomUUID) - val file: File = File.createTempFile("spillFile", ".spill", tempDir) - spillFilesCreated.add(file) - (blockId, file) - }) - - when(taskContext.taskMetrics()).thenReturn(taskMetrics) - when(dependency.serializer).thenReturn(serializer) - when(dependency.partitioner).thenReturn(hashPartitioner) - - spillFilesCreated.clear() new UnsafeShuffleWriter[String, String]( blockManager, - shuffleBlockResolver, + blockResolver, taskMemoryManager, shuffleHandle, 0, taskContext, conf, - taskMetrics.shuffleWriteMetrics + taskContext.taskMetrics().shuffleWriteMetrics ) } - def cleanupTempFiles(): Unit = { - FileUtils.deleteDirectory(tempDir) - } - - def createDataInMemory(size: Int): Array[(String, String)] = { - val random = new Random(123) - (1 to size).map { i => { - val x = random.alphanumeric.take(DEFAULT_DATA_STRING_SIZE).mkString - Tuple2(x, x) - } }.toArray - } - - def createDataOnDisk(size: Int): File = { - // scalastyle:off println - println("Generating test data with num records: " + size) - val random = new Random(123) - val dataOutput = new FileOutputStream(tempDataFile) - try { - (1 to size).foreach { i => { - if (i % 1000000 == 0) { - println("Wrote " + i + " test data points") - } - val x = random.alphanumeric.take(DEFAULT_DATA_STRING_SIZE).mkString - dataOutput.write(x.getBytes) - }} - } - finally { - dataOutput.close() - } - tempDataFile - // scalastyle:off println - } - - private class DataIterator private ( - private val inputStream: BufferedInputStream, - private val buffer: Array[Byte]) extends Iterator[Product2[String, String]] { - override def hasNext: Boolean = { - inputStream.available() > 0 - } - - override def next(): Product2[String, String] = { - val read = inputStream.read(buffer) - assert(read == buffer.length) - val string = buffer.mkString - (string, string) - } - } - - private object DataIterator { - def apply(inputFile: File, bufferSize: Int): DataIterator = { - val inputStream = new BufferedInputStream( - new FileInputStream(inputFile), DEFAULT_DATA_STRING_SIZE) - val buffer = new Array[Byte](DEFAULT_DATA_STRING_SIZE) - new DataIterator(inputStream, buffer) - } - } - def writeBenchmarkWithSmallDataset(): Unit = { val size = 1000 val benchmark = new Benchmark("UnsafeShuffleWriter with spills", size, minNumIters = MIN_NUM_ITERS, output = output) - benchmark.addTimerCase("small dataset without spills") { timer => - val writer = setup(false) + addBenchmarkCase(benchmark, "small dataset without spills") { timer => + val writer = constructWriter(false) val array = createDataInMemory(1000) timer.startTiming() writer.write(array.iterator) timer.stopTiming() - assert(spillFilesCreated.size() == 1) // The single temp file is for the temp index file - cleanupTempFiles() + assert(tempFilesCreated.length == 1) // The single temp file is for the temp index file } benchmark.run() } @@ -224,27 +77,25 @@ object UnsafeShuffleWriterBenchmark extends BenchmarkBase { def writeBenchmarkWithSpill(): Unit = { val size = PackedRecordPointer.MAXIMUM_PAGE_SIZE_BYTES/2/DEFAULT_DATA_STRING_SIZE val minNumIters = 5 - createDataOnDisk(size) + val tempDataFile = createDataOnDisk(size) val benchmark = new Benchmark("UnsafeShuffleWriter with spills", size, minNumIters = minNumIters, output = output, outputPerIteration = true) - benchmark.addTimerCase("without transferTo") { timer => - val shuffleWriter = setup(false) + addBenchmarkCase(benchmark, "without transferTo") { timer => + val shuffleWriter = constructWriter(false) timer.startTiming() shuffleWriter.write(DataIterator(inputFile = tempDataFile, DEFAULT_DATA_STRING_SIZE)) timer.stopTiming() - assert(spillFilesCreated.size() == 7) - cleanupTempFiles() + assert(tempFilesCreated.length == 7) } - benchmark.addTimerCase("with transferTo") { timer => - val shuffleWriter = setup(false) + addBenchmarkCase(benchmark, "with transferTo") { timer => + val shuffleWriter = constructWriter(false) timer.startTiming() shuffleWriter.write(DataIterator(inputFile = tempDataFile, DEFAULT_DATA_STRING_SIZE)) timer.stopTiming() - assert(spillFilesCreated.size() == 7) - cleanupTempFiles() + assert(tempFilesCreated.length == 7) } benchmark.run() tempDataFile.delete() From 0c696dc910d9257df8a1f2700b136073653832be Mon Sep 17 00:00:00 2001 From: Yifei Huang Date: Thu, 28 Feb 2019 22:27:13 -0800 Subject: [PATCH 31/68] shuffle read works with transport client... needs lots of cleaning --- .../BlockStoreShuffleReaderBenchmark.scala | 156 ++++++++++++------ 1 file changed, 104 insertions(+), 52 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/shuffle/BlockStoreShuffleReaderBenchmark.scala b/core/src/test/scala/org/apache/spark/shuffle/BlockStoreShuffleReaderBenchmark.scala index e4febb3116c97..119503ef880e1 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/BlockStoreShuffleReaderBenchmark.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/BlockStoreShuffleReaderBenchmark.scala @@ -16,28 +16,30 @@ */ package org.apache.spark.shuffle -import java.io.{File, FileOutputStream, OutputStream} -import java.util.concurrent.{Callable, Executors} +import java.io.{File, FileOutputStream} +import java.nio.channels.ReadableByteChannel +import java.util.concurrent.Executors import com.google.common.io.CountingOutputStream -import org.mockito.{Mock, MockitoAnnotations} import org.mockito.Answers.RETURNS_SMART_NULLS -import org.mockito.Matchers.any +import org.mockito.Matchers._ import org.mockito.Mockito.when -import org.mockito.invocation.InvocationOnMock +import org.mockito.{Mock, MockitoAnnotations} +import scala.concurrent.Future import scala.util.Random -import org.apache.spark.{MapOutputTracker, ShuffleDependency, SparkConf, SparkEnv, TaskContext} import org.apache.spark.benchmark.BenchmarkBase import org.apache.spark.executor.TaskMetrics +import org.apache.spark.memory.{MemoryManager, TaskMemoryManager, TestMemoryManager} import org.apache.spark.network.BlockTransferService -import org.apache.spark.network.buffer.FileSegmentManagedBuffer -import org.apache.spark.network.netty.SparkTransportConf -import org.apache.spark.network.shuffle.{BlockFetchingListener, DownloadFileManager} +import org.apache.spark.network.buffer.{FileSegmentManagedBuffer, ManagedBuffer} +import org.apache.spark.network.netty.{NettyBlockTransferService, SparkTransportConf} import org.apache.spark.network.util.TransportConf +import org.apache.spark.rpc.{RpcAddress, RpcEndpoint, RpcEndpointRef, RpcEnv, RpcEnvFileServer} import org.apache.spark.serializer.{KryoSerializer, SerializerManager} -import org.apache.spark.storage.{BlockManager, BlockManagerId, ShuffleBlockId} +import org.apache.spark.storage.{BlockId, BlockManager, BlockManagerId, BlockManagerMaster, ShuffleBlockId} import org.apache.spark.util.Utils +import org.apache.spark.{MapOutputTracker, ShuffleDependency, SparkConf, SparkEnv, TaskContext} /** * Benchmark to measure performance for aggregate primitives. @@ -51,21 +53,26 @@ import org.apache.spark.util.Utils */ object BlockStoreShuffleReaderBenchmark extends BenchmarkBase { - - @Mock(answer = RETURNS_SMART_NULLS) private var blockManager: BlockManager = _ - @Mock(answer = RETURNS_SMART_NULLS) private var transferService: BlockTransferService = _ + // this is only used to retrieve info about the aggregator/sorters/serializers, + // so it shouldn't affect the performance significantly @Mock(answer = RETURNS_SMART_NULLS) private var dependency: ShuffleDependency[String, String, String] = _ + // used only to get metrics, so does not affect perf significantly @Mock(answer = RETURNS_SMART_NULLS) private var taskContext: TaskContext = _ + // only used to retrieve info about the maps at the beginning, doesn't affect perf significantly @Mock(answer = RETURNS_SMART_NULLS) private var mapOutputTracker: MapOutputTracker = _ + // this is only used when initializing the block manager, so doesn't affect perf + @Mock(answer = RETURNS_SMART_NULLS) private var blockManagerMaster: BlockManagerMaster = _ private val defaultConf: SparkConf = new SparkConf() .set("spark.shuffle.compress", "false") .set("spark.shuffle.spill.compress", "false") + .set("spark.authenticate", "false") + .set("spark.app.id", "test-app") private val serializer: KryoSerializer = new KryoSerializer(defaultConf) private val serializerManager: SerializerManager = new SerializerManager(serializer, defaultConf) - private val execBlockManagerId: BlockManagerId = BlockManagerId("execId", "host", 8000) - private val remoteBlockManagerId: BlockManagerId = BlockManagerId("remote", "remote", 8000) + private val execBlockManagerId: BlockManagerId = BlockManagerId("localhost", "localhost", 7000) + private val remoteBlockManagerId: BlockManagerId = BlockManagerId("localhost", "localhost", 7002) private val transportConf: TransportConf = SparkTransportConf.fromSparkConf(defaultConf, "shuffle") @@ -76,18 +83,98 @@ object BlockStoreShuffleReaderBenchmark extends BenchmarkBase { private val NUM_MAPS: Int = 1 private val DEFAULT_DATA_STRING_SIZE = 5 - private val executorPool = Executors.newFixedThreadPool(10) + + private var rpcEnv: RpcEnv = new RpcEnv(defaultConf) { + override def endpointRef(endpoint: RpcEndpoint): RpcEndpointRef = { null } + override def address: RpcAddress = null + override def setupEndpoint(name: String, endpoint: RpcEndpoint): RpcEndpointRef = { null } + override def asyncSetupEndpointRefByURI(uri: String): Future[RpcEndpointRef] = { null } + override def stop(endpoint: RpcEndpointRef): Unit = { } + override def shutdown(): Unit = { } + override def awaitTermination(): Unit = { } + override def deserialize[T](deserializationAction: () => T): T = { deserializationAction() } + override def fileServer: RpcEnvFileServer = { null } + override def openChannel(uri: String): ReadableByteChannel = { null } + } + + protected var memoryManager: TestMemoryManager = _ + protected var taskMemoryManager: TaskMemoryManager = _ + + class TestBlockManager(memoryManager: MemoryManager, + transferService: BlockTransferService, + blockManagerMaster: BlockManagerMaster, + dataFile: File, + fileLength: Long) extends BlockManager("0", + rpcEnv, + blockManagerMaster, + serializerManager, + defaultConf, + memoryManager, + null, null, transferService, null, 1) { + blockManagerId = execBlockManagerId + + override def getBlockData(blockId: BlockId): ManagedBuffer = { + new FileSegmentManagedBuffer( + transportConf, + dataFile, + 0, + fileLength + ) + } + } + + private var blockManager : BlockManager = _ + private val securityManager = new org.apache.spark.SecurityManager(defaultConf) def setup(size: Int, fetchLocal: Boolean): BlockStoreShuffleReader[String, String] = { MockitoAnnotations.initMocks(this) + when(blockManagerMaster.registerBlockManager( + any[BlockManagerId], any[Long], any[Long], any[RpcEndpointRef])).thenReturn(null) + val dataFileAndLength = generateDataOnDisk(10) + + memoryManager = new TestMemoryManager(defaultConf) + taskMemoryManager = new TaskMemoryManager(memoryManager, 0) + val localShuffleClient = new NettyBlockTransferService( + defaultConf, + new org.apache.spark.SecurityManager(defaultConf), + "localhost", + "localhost", + 7000, + 1 + ) + val blockManager = + new TestBlockManager(memoryManager, + localShuffleClient, + blockManagerMaster, + dataFileAndLength._1, + dataFileAndLength._2) + blockManager.initialize(defaultConf.getAppId) + + val externalServer = new NettyBlockTransferService( + defaultConf, + new org.apache.spark.SecurityManager(defaultConf), + "localhost", + "localhost", + 7002, + 1 + ) + + val externalBlockManager = new TestBlockManager( + memoryManager, + externalServer, + blockManagerMaster, + dataFileAndLength._1, + dataFileAndLength._2) + externalBlockManager.initialize(defaultConf.getAppId) + SparkEnv.set(new SparkEnv( "0", null, serializer, null, serializerManager, - null, + mapOutputTracker, null, null, blockManager, @@ -106,9 +193,7 @@ object BlockStoreShuffleReaderBenchmark extends BenchmarkBase { val taskMetrics = new TaskMetrics when(taskContext.taskMetrics()).thenReturn(taskMetrics) - when(blockManager.shuffleClient).thenReturn(transferService) when(dependency.serializer).thenReturn(serializer) - when(blockManager.blockManagerId).thenReturn(execBlockManagerId) when(mapOutputTracker.getMapSizesByExecutorId(SHUFFLE_ID, REDUCE_ID, REDUCE_ID + 1)) .thenReturn { val shuffleBlockIdsAndSizes = (0 until NUM_MAPS).map { mapId => @@ -124,39 +209,6 @@ object BlockStoreShuffleReaderBenchmark extends BenchmarkBase { if (fetchLocal) { // to do } else { - when(transferService.fetchBlocks( - any[String], - any[Int], - any[String], - any[Array[String]], - any[BlockFetchingListener], - any[DownloadFileManager] - )).thenAnswer((invocation: InvocationOnMock) => { - val blocks = invocation.getArguments()(3).asInstanceOf[Array[String]] - val listener = invocation.getArguments()(4).asInstanceOf[BlockFetchingListener] - - // TODO: do this in parallel? - for (blockId <- blocks) { - val generatedFile = generateDataOnDisk(size) - listener.onBlockFetchSuccess(blockId, new FileSegmentManagedBuffer( - transportConf, - generatedFile._1, - 0, - generatedFile._2 - )) -// executorPool.submit(new Callable[Unit] { -// override def call(): Unit = { -// val fileGenerated = generateDataOnDisk(size) -// listener.onBlockFetchSuccess(blockId, new FileSegmentManagedBuffer( -// transportConf, -// fileGenerated, -// 0, -// size.toLong * DEFAULT_DATA_STRING_SIZE -// )) -// } -// }) - } - }) } // TODO: use aggregation + sort From 323a2963f8590958d8b1c568366c418ab4a7987b Mon Sep 17 00:00:00 2001 From: Yifei Huang Date: Fri, 1 Mar 2019 13:29:39 -0800 Subject: [PATCH 32/68] test running in cicle --- .circleci/config.yml | 1 + 1 file changed, 1 insertion(+) diff --git a/.circleci/config.yml b/.circleci/config.yml index 8fd23e9bad5e3..9dcfdc75f6e63 100644 --- a/.circleci/config.yml +++ b/.circleci/config.yml @@ -29,6 +29,7 @@ spark-25299-branch-only: &spark-25299-branch-only only: - spark-25299 - spark-25299-test-build + - yh/add-benchmarks-and-ci deployable-branches-and-tags: &deployable-branches-and-tags filters: From 85836c2a8474020709c00984c9a64919ee6f298b Mon Sep 17 00:00:00 2001 From: Yifei Huang Date: Fri, 1 Mar 2019 14:54:21 -0800 Subject: [PATCH 33/68] scalastyle --- .../BlockStoreShuffleReaderBenchmark.scala | 116 ++++++++++++++---- 1 file changed, 94 insertions(+), 22 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/shuffle/BlockStoreShuffleReaderBenchmark.scala b/core/src/test/scala/org/apache/spark/shuffle/BlockStoreShuffleReaderBenchmark.scala index 119503ef880e1..f7a782e808d1e 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/BlockStoreShuffleReaderBenchmark.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/BlockStoreShuffleReaderBenchmark.scala @@ -18,19 +18,19 @@ package org.apache.spark.shuffle import java.io.{File, FileOutputStream} import java.nio.channels.ReadableByteChannel -import java.util.concurrent.Executors import com.google.common.io.CountingOutputStream +import org.mockito.{Mock, MockitoAnnotations} import org.mockito.Answers.RETURNS_SMART_NULLS import org.mockito.Matchers._ import org.mockito.Mockito.when -import org.mockito.{Mock, MockitoAnnotations} import scala.concurrent.Future import scala.util.Random +import org.apache.spark.{MapOutputTracker, ShuffleDependency, SparkConf, SparkEnv, TaskContext} import org.apache.spark.benchmark.BenchmarkBase import org.apache.spark.executor.TaskMetrics -import org.apache.spark.memory.{MemoryManager, TaskMemoryManager, TestMemoryManager} +import org.apache.spark.memory.TestMemoryManager import org.apache.spark.network.BlockTransferService import org.apache.spark.network.buffer.{FileSegmentManagedBuffer, ManagedBuffer} import org.apache.spark.network.netty.{NettyBlockTransferService, SparkTransportConf} @@ -39,7 +39,6 @@ import org.apache.spark.rpc.{RpcAddress, RpcEndpoint, RpcEndpointRef, RpcEnv, Rp import org.apache.spark.serializer.{KryoSerializer, SerializerManager} import org.apache.spark.storage.{BlockId, BlockManager, BlockManagerId, BlockManagerMaster, ShuffleBlockId} import org.apache.spark.util.Utils -import org.apache.spark.{MapOutputTracker, ShuffleDependency, SparkConf, SparkEnv, TaskContext} /** * Benchmark to measure performance for aggregate primitives. @@ -53,13 +52,13 @@ import org.apache.spark.{MapOutputTracker, ShuffleDependency, SparkConf, SparkEn */ object BlockStoreShuffleReaderBenchmark extends BenchmarkBase { - // this is only used to retrieve info about the aggregator/sorters/serializers, + // this is only used to retrieve the aggregator/sorters/serializers, // so it shouldn't affect the performance significantly @Mock(answer = RETURNS_SMART_NULLS) private var dependency: ShuffleDependency[String, String, String] = _ // used only to get metrics, so does not affect perf significantly @Mock(answer = RETURNS_SMART_NULLS) private var taskContext: TaskContext = _ - // only used to retrieve info about the maps at the beginning, doesn't affect perf significantly + // only used to retrieve info about the maps at the beginning, doesn't affect perf @Mock(answer = RETURNS_SMART_NULLS) private var mapOutputTracker: MapOutputTracker = _ // this is only used when initializing the block manager, so doesn't affect perf @Mock(answer = RETURNS_SMART_NULLS) private var blockManagerMaster: BlockManagerMaster = _ @@ -75,6 +74,9 @@ object BlockStoreShuffleReaderBenchmark extends BenchmarkBase { private val remoteBlockManagerId: BlockManagerId = BlockManagerId("localhost", "localhost", 7002) private val transportConf: TransportConf = SparkTransportConf.fromSparkConf(defaultConf, "shuffle") + private val securityManager: org.apache.spark.SecurityManager = + new org.apache.spark.SecurityManager(defaultConf) + protected val memoryManager: TestMemoryManager = new TestMemoryManager(defaultConf) private var tempDir: File = _ @@ -84,7 +86,7 @@ object BlockStoreShuffleReaderBenchmark extends BenchmarkBase { private val DEFAULT_DATA_STRING_SIZE = 5 - private var rpcEnv: RpcEnv = new RpcEnv(defaultConf) { + private val rpcEnv: RpcEnv = new RpcEnv(defaultConf) { override def endpointRef(endpoint: RpcEndpoint): RpcEndpointRef = { null } override def address: RpcAddress = null override def setupEndpoint(name: String, endpoint: RpcEndpoint): RpcEndpointRef = { null } @@ -97,11 +99,7 @@ object BlockStoreShuffleReaderBenchmark extends BenchmarkBase { override def openChannel(uri: String): ReadableByteChannel = { null } } - protected var memoryManager: TestMemoryManager = _ - protected var taskMemoryManager: TaskMemoryManager = _ - - class TestBlockManager(memoryManager: MemoryManager, - transferService: BlockTransferService, + class TestBlockManager(transferService: BlockTransferService, blockManagerMaster: BlockManagerMaster, dataFile: File, fileLength: Long) extends BlockManager("0", @@ -124,8 +122,85 @@ object BlockStoreShuffleReaderBenchmark extends BenchmarkBase { } private var blockManager : BlockManager = _ - private val securityManager = new org.apache.spark.SecurityManager(defaultConf) + def setupTest(): Unit = { + MockitoAnnotations.initMocks(this) + when(blockManagerMaster.registerBlockManager( + any[BlockManagerId], any[Long], any[Long], any[RpcEndpointRef])).thenReturn(null) + val dataFileAndLength = generateDataOnDisk(10) + + val localShuffleClient = new NettyBlockTransferService( + defaultConf, + securityManager, + "localhost", + "localhost", + 7000, + 1 + ) + blockManager = + new TestBlockManager( + localShuffleClient, + blockManagerMaster, + dataFileAndLength._1, + dataFileAndLength._2) + blockManager.initialize(defaultConf.getAppId) + + val externalServer = new NettyBlockTransferService( + defaultConf, + securityManager, + "localhost", + "localhost", + 7002, + 1 + ) + + val externalBlockManager = new TestBlockManager( + externalServer, + blockManagerMaster, + dataFileAndLength._1, + dataFileAndLength._2) + externalBlockManager.initialize(defaultConf.getAppId) + + SparkEnv.set(new SparkEnv( + "0", + null, + serializer, + null, + serializerManager, + mapOutputTracker, + null, + null, + blockManager, + null, + null, + null, + null, + defaultConf + )) + + val shuffleHandle = new BaseShuffleHandle( + shuffleId = SHUFFLE_ID, + numMaps = NUM_MAPS, + dependency = dependency) + + val taskMetrics = new TaskMetrics + when(taskContext.taskMetrics()).thenReturn(taskMetrics) + + when(dependency.serializer).thenReturn(serializer) + when(mapOutputTracker.getMapSizesByExecutorId(SHUFFLE_ID, REDUCE_ID, REDUCE_ID + 1)) + .thenReturn { + val shuffleBlockIdsAndSizes = (0 until NUM_MAPS).map { mapId => + val shuffleBlockId = ShuffleBlockId(SHUFFLE_ID, mapId, REDUCE_ID) + (shuffleBlockId, 0L) + } + // TODO: configurable + Seq((remoteBlockManagerId, shuffleBlockIdsAndSizes)).toIterator + } + + // TODO: use aggregation + sort + when(dependency.aggregator).thenReturn(Option.empty) + when(dependency.keyOrdering).thenReturn(Option.empty) + } def setup(size: Int, fetchLocal: Boolean): BlockStoreShuffleReader[String, String] = { MockitoAnnotations.initMocks(this) @@ -133,18 +208,16 @@ object BlockStoreShuffleReaderBenchmark extends BenchmarkBase { any[BlockManagerId], any[Long], any[Long], any[RpcEndpointRef])).thenReturn(null) val dataFileAndLength = generateDataOnDisk(10) - memoryManager = new TestMemoryManager(defaultConf) - taskMemoryManager = new TaskMemoryManager(memoryManager, 0) val localShuffleClient = new NettyBlockTransferService( defaultConf, - new org.apache.spark.SecurityManager(defaultConf), + securityManager, "localhost", "localhost", 7000, 1 ) - val blockManager = - new TestBlockManager(memoryManager, + blockManager = + new TestBlockManager( localShuffleClient, blockManagerMaster, dataFileAndLength._1, @@ -153,7 +226,7 @@ object BlockStoreShuffleReaderBenchmark extends BenchmarkBase { val externalServer = new NettyBlockTransferService( defaultConf, - new org.apache.spark.SecurityManager(defaultConf), + securityManager, "localhost", "localhost", 7002, @@ -161,7 +234,6 @@ object BlockStoreShuffleReaderBenchmark extends BenchmarkBase { ) val externalBlockManager = new TestBlockManager( - memoryManager, externalServer, blockManagerMaster, dataFileAndLength._1, @@ -204,8 +276,6 @@ object BlockStoreShuffleReaderBenchmark extends BenchmarkBase { Seq((remoteBlockManagerId, shuffleBlockIdsAndSizes)).toIterator } - tempDir = Utils.createTempDir(null, "shuffle") - if (fetchLocal) { // to do } else { @@ -229,6 +299,7 @@ object BlockStoreShuffleReaderBenchmark extends BenchmarkBase { def generateDataOnDisk(size: Int): (File, Long) = { // scalastyle:off println + tempDir = Utils.createTempDir(null, "shuffle") val tempDataFile: File = File.createTempFile("test-data", "", tempDir) println("Generating test data with num records: " + size) val random = new Random(123) @@ -256,6 +327,7 @@ object BlockStoreShuffleReaderBenchmark extends BenchmarkBase { val reader = setup(10, false) // scalastyle:off println println(reader.read().length) + println(reader.read().length) // scalastyle:on println // assert(reader.read().length == 10 * NUM_MAPS) From b67d1f3c54b6c2a675d08d24928d060970c785be Mon Sep 17 00:00:00 2001 From: Yifei Huang Date: Fri, 1 Mar 2019 14:54:29 -0800 Subject: [PATCH 34/68] dont publish results yet --- .circleci/config.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.circleci/config.yml b/.circleci/config.yml index 9dcfdc75f6e63..29ff73969f4c2 100644 --- a/.circleci/config.yml +++ b/.circleci/config.yml @@ -474,7 +474,7 @@ jobs: - *restore-build-binaries-cache - *restore-home-sbt-cache - run: - command: ./dev/run-spark-25299-benchmarks.sh -u + command: ./dev/run-spark-25299-benchmarks.sh - store_artifacts: path: /tmp/artifacts/ From 252963d8f58bbe8b9e4eec115a2d1579c4733ffe Mon Sep 17 00:00:00 2001 From: Yifei Huang Date: Mon, 4 Mar 2019 11:01:13 -0800 Subject: [PATCH 35/68] cleanup writer code --- .../BlockStoreShuffleReaderBenchmark.scala | 12 ++-- ...ypassMergeSortShuffleWriterBenchmark.scala | 30 +++----- .../sort/ShuffleWriterBenchmarkBase.scala | 72 ++++++++----------- .../sort/SortShuffleWriterBenchmark.scala | 33 ++++++--- .../sort/UnsafeShuffleWriterBenchmark.scala | 25 ++++--- 5 files changed, 84 insertions(+), 88 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/shuffle/BlockStoreShuffleReaderBenchmark.scala b/core/src/test/scala/org/apache/spark/shuffle/BlockStoreShuffleReaderBenchmark.scala index f7a782e808d1e..d89cd9749e679 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/BlockStoreShuffleReaderBenchmark.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/BlockStoreShuffleReaderBenchmark.scala @@ -100,15 +100,19 @@ object BlockStoreShuffleReaderBenchmark extends BenchmarkBase { } class TestBlockManager(transferService: BlockTransferService, - blockManagerMaster: BlockManagerMaster, - dataFile: File, - fileLength: Long) extends BlockManager("0", + blockManagerMaster: BlockManagerMaster, + dataFile: File, + fileLength: Long) extends BlockManager("0", rpcEnv, blockManagerMaster, serializerManager, defaultConf, memoryManager, - null, null, transferService, null, 1) { + null, + null, + transferService, + null, + 1) { blockManagerId = execBlockManagerId override def getBlockData(blockId: BlockId): ManagedBuffer = { diff --git a/core/src/test/scala/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriterBenchmark.scala b/core/src/test/scala/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriterBenchmark.scala index def740c1d5cda..82b3e3da20d38 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriterBenchmark.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriterBenchmark.scala @@ -17,8 +17,6 @@ package org.apache.spark.shuffle.sort -import scala.util.Random - import org.apache.spark.SparkConf import org.apache.spark.benchmark.Benchmark @@ -32,7 +30,7 @@ import org.apache.spark.benchmark.Benchmark * Results will be written to "benchmarks/-results.txt". * }}} */ -object BypassMergeSortShuffleWriterBenchmark extends ShuffleWriterBenchmarkBase(false) { +object BypassMergeSortShuffleWriterBenchmark extends ShuffleWriterBenchmarkBase { private var shuffleHandle: BypassMergeSortShuffleHandle[String, String] = new BypassMergeSortShuffleHandle[String, String]( @@ -41,8 +39,10 @@ object BypassMergeSortShuffleWriterBenchmark extends ShuffleWriterBenchmarkBase( dependency) private val MIN_NUM_ITERS = 10 + private val DATA_SIZE_SMALL = 10000 + private val DATA_SIZE_LARGE = 10000000 - def constructWriter(transferTo: Boolean): BypassMergeSortShuffleWriter[String, String] = { + def getWriter(transferTo: Boolean): BypassMergeSortShuffleWriter[String, String] = { val conf = new SparkConf(loadDefaults = false) conf.set("spark.file.transferTo", String.valueOf(transferTo)) conf.set("spark.shuffle.file.buffer", "32k") @@ -60,12 +60,8 @@ object BypassMergeSortShuffleWriterBenchmark extends ShuffleWriterBenchmarkBase( } def writeBenchmarkWithLargeDataset(): Unit = { - val size = 10000000 - val random = new Random(123) - val data = (1 to size).map { i => { - val x = random.alphanumeric.take(5).mkString - Tuple2(x, x) - } }.toArray + val size = DATA_SIZE_LARGE + val data = createDataInMemory(size) val benchmark = new Benchmark( "BypassMergeSortShuffleWrite (with spill) " + size, size, @@ -73,13 +69,13 @@ object BypassMergeSortShuffleWriterBenchmark extends ShuffleWriterBenchmarkBase( output = output) addBenchmarkCase(benchmark, "without transferTo") { timer => - val shuffleWriter = constructWriter(false) + val shuffleWriter = getWriter(false) timer.startTiming() shuffleWriter.write(data.iterator) timer.stopTiming() } addBenchmarkCase(benchmark, "with transferTo") { timer => - val shuffleWriter = constructWriter(false) + val shuffleWriter = getWriter(false) timer.startTiming() shuffleWriter.write(data.iterator) timer.stopTiming() @@ -88,18 +84,14 @@ object BypassMergeSortShuffleWriterBenchmark extends ShuffleWriterBenchmarkBase( } def writeBenchmarkWithSmallDataset(): Unit = { - val size = 10000 - val random = new Random(123) - val data = (1 to size).map { i => { - val x = random.alphanumeric.take(5).mkString - Tuple2(x, x) - } }.toArray + val size = DATA_SIZE_SMALL + val data = createDataInMemory(size) val benchmark = new Benchmark("BypassMergeSortShuffleWrite (in memory buffer) " + size, size, minNumIters = MIN_NUM_ITERS, output = output) addBenchmarkCase(benchmark, "small dataset without spills on disk") { timer => - val shuffleWriter = constructWriter(false) + val shuffleWriter = getWriter(false) timer.startTiming() shuffleWriter.write(data.iterator) timer.stopTiming() diff --git a/core/src/test/scala/org/apache/spark/shuffle/sort/ShuffleWriterBenchmarkBase.scala b/core/src/test/scala/org/apache/spark/shuffle/sort/ShuffleWriterBenchmarkBase.scala index d18c0896e77c7..8946052dd5f37 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/sort/ShuffleWriterBenchmarkBase.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/sort/ShuffleWriterBenchmarkBase.scala @@ -17,31 +17,32 @@ package org.apache.spark.shuffle.sort -import java.io.{BufferedInputStream, File, FileInputStream, FileOutputStream} -import java.nio.channels.ReadableByteChannel +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.Matchers.any import org.mockito.Mockito.when +import org.slf4j.{Logger, LoggerFactory} import scala.collection.mutable import scala.collection.mutable.ArrayBuffer -import scala.concurrent.Future 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.{RpcAddress, RpcEndpoint, RpcEndpointRef, RpcEnv, RpcEnvFileServer} -import org.apache.spark.serializer.{JavaSerializer, KryoSerializer, Serializer, SerializerManager} +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(useKryoSerializer: Boolean) extends BenchmarkBase { +abstract class ShuffleWriterBenchmarkBase extends BenchmarkBase { + private def log: Logger = LoggerFactory.getLogger(this.getClass.getName.stripSuffix("$")) private val DEFAULT_DATA_STRING_SIZE = 5 // This is only used in the writer constructors, so it's ok to mock @@ -49,30 +50,15 @@ abstract class ShuffleWriterBenchmarkBase(useKryoSerializer: Boolean) extends Be 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 = if (useKryoSerializer) { - new KryoSerializer(defaultConf) - } else { - new JavaSerializer(defaultConf) - } + protected val serializer: Serializer = new KryoSerializer(defaultConf) protected val partitioner: HashPartitioner = new HashPartitioner(10) protected val serializerManager: SerializerManager = new SerializerManager(serializer, defaultConf) - private var rpcEnv: RpcEnv = new RpcEnv(defaultConf) { - override def endpointRef(endpoint: RpcEndpoint): RpcEndpointRef = { null } - override def address: RpcAddress = null - override def setupEndpoint(name: String, endpoint: RpcEndpoint): RpcEndpointRef = { null } - override def asyncSetupEndpointRefByURI(uri: String): Future[RpcEndpointRef] = { null } - override def stop(endpoint: RpcEndpointRef): Unit = { } - override def shutdown(): Unit = { } - override def awaitTermination(): Unit = { } - override def deserialize[T](deserializationAction: () => T): T = { deserializationAction() } - override def fileServer: RpcEnvFileServer = { null } - override def openChannel(uri: String): ReadableByteChannel = { null } - } - protected val tempFilesCreated: ArrayBuffer[File] = new ArrayBuffer[File] protected val filenameToFile: mutable.Map[String, File] = new mutable.HashMap[String, File] @@ -120,6 +106,7 @@ abstract class ShuffleWriterBenchmarkBase(useKryoSerializer: Boolean) extends Be when(dependency.shuffleId).thenReturn(0) shuffleMetrics = new TaskMetrics when(taskContext.taskMetrics()).thenReturn(shuffleMetrics) + when(rpcEnv.setupEndpoint(any[String], any[RpcEndpoint])).thenReturn(rpcEndpointRef) def setup(): Unit = { memoryManager = new TestMemoryManager(defaultConf) @@ -147,8 +134,9 @@ abstract class ShuffleWriterBenchmarkBase(useKryoSerializer: Boolean) extends Be } protected class DataIterator private ( - private val inputStream: BufferedInputStream, - private val buffer: Array[Byte]) extends Iterator[Product2[String, String]] { + inputStream: BufferedInputStream, + buffer: Array[Byte]) + extends Iterator[Product2[String, String]] with Closeable { override def hasNext: Boolean = { inputStream.available() > 0 } @@ -159,6 +147,8 @@ abstract class ShuffleWriterBenchmarkBase(useKryoSerializer: Boolean) extends Be val string = buffer.mkString (string, string) } + + override def close(): Unit = inputStream.close() } protected object DataIterator { @@ -170,8 +160,9 @@ abstract class ShuffleWriterBenchmarkBase(useKryoSerializer: Boolean) extends Be } } + private val random = new Random(123) + def createDataInMemory(size: Int): Array[(String, String)] = { - val random = new Random(123) (1 to size).map { i => { val x = random.alphanumeric.take(DEFAULT_DATA_STRING_SIZE).mkString Tuple2(x, x) @@ -179,25 +170,20 @@ abstract class ShuffleWriterBenchmarkBase(useKryoSerializer: Boolean) extends Be } def createDataOnDisk(size: Int): File = { - // scalastyle:off println - println("Generating test data with num records: " + size) + log.info("Generating test data with num records: " + size) val tempDataFile = File.createTempFile("test-data", "") - val random = new Random(123) - val dataOutput = new FileOutputStream(tempDataFile) - try { - (1 to size).foreach { i => { - if (i % 1000000 == 0) { - println("Wrote " + i + " test data points") - } - val x = random.alphanumeric.take(DEFAULT_DATA_STRING_SIZE).mkString - dataOutput.write(x.getBytes) - }} - } - finally { - dataOutput.close() + Utils.tryWithResource(new FileOutputStream(tempDataFile)) { + dataOutput => + (1 to size).foreach { i => { + if (i % 1000000 == 0) { + log.info("Wrote " + i + " test data points") + } + val x = random.alphanumeric.take(DEFAULT_DATA_STRING_SIZE).mkString + dataOutput.write(x.getBytes) + }} } + tempDataFile - // scalastyle:off println } } diff --git a/core/src/test/scala/org/apache/spark/shuffle/sort/SortShuffleWriterBenchmark.scala b/core/src/test/scala/org/apache/spark/shuffle/sort/SortShuffleWriterBenchmark.scala index e2428f846927f..9c53ce5d09f4c 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/sort/SortShuffleWriterBenchmark.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/sort/SortShuffleWriterBenchmark.scala @@ -22,6 +22,7 @@ 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. @@ -33,7 +34,7 @@ import org.apache.spark.shuffle.BaseShuffleHandle * Results will be written to "benchmarks/-results.txt". * }}} */ -object SortShuffleWriterBenchmark extends ShuffleWriterBenchmarkBase(true) { +object SortShuffleWriterBenchmark extends ShuffleWriterBenchmarkBase { private var shuffleHandle: BaseShuffleHandle[String, String, String] = new BaseShuffleHandle( @@ -113,9 +114,12 @@ object SortShuffleWriterBenchmark extends ShuffleWriterBenchmarkBase(true) { outputPerIteration = true) addBenchmarkCase(benchmark, "no map side combine") { timer => val shuffleWriter = constructWriter(Option.empty, Option.empty) - timer.startTiming() - shuffleWriter.write(DataIterator(inputFile = dataFile, DEFAULT_DATA_STRING_SIZE)) - timer.stopTiming() + Utils.tryWithResource(DataIterator(inputFile = dataFile, DEFAULT_DATA_STRING_SIZE)) { + iterator => + timer.startTiming() + shuffleWriter.write(iterator) + timer.stopTiming() + } assert(tempFilesCreated.length == 8) } @@ -127,19 +131,26 @@ object SortShuffleWriterBenchmark extends ShuffleWriterBenchmarkBase(true) { new Aggregator[String, String, String](createCombiner, mergeValue, mergeCombiners) addBenchmarkCase(benchmark, "with map side aggregation") { timer => val shuffleWriter = constructWriter(Some(aggregator), Option.empty) - timer.startTiming() - shuffleWriter.write(DataIterator(inputFile = dataFile, DEFAULT_DATA_STRING_SIZE)) - timer.stopTiming() + Utils.tryWithResource(DataIterator(inputFile = dataFile, DEFAULT_DATA_STRING_SIZE)) { + iterator => + timer.startTiming() + shuffleWriter.write(iterator) + timer.stopTiming() + } assert(tempFilesCreated.length == 8) } val sorter = Ordering.String addBenchmarkCase(benchmark, "with map side sort") { timer => val shuffleWriter = constructWriter(Option.empty, Some(sorter)) - timer.startTiming() - shuffleWriter.write(DataIterator(inputFile = dataFile, DEFAULT_DATA_STRING_SIZE)) - timer.stopTiming() - assert(tempFilesCreated.length == 8) + val iterator: DataIterator = DataIterator(inputFile = dataFile, DEFAULT_DATA_STRING_SIZE) + Utils.tryWithResource(DataIterator(inputFile = dataFile, DEFAULT_DATA_STRING_SIZE)) { + iterator => + timer.startTiming() + shuffleWriter.write(iterator) + timer.stopTiming() + } + assert(tempFilesCreated.length == 8) } benchmark.run() dataFile.delete() diff --git a/core/src/test/scala/org/apache/spark/shuffle/sort/UnsafeShuffleWriterBenchmark.scala b/core/src/test/scala/org/apache/spark/shuffle/sort/UnsafeShuffleWriterBenchmark.scala index f34dea0c82e51..3ba0fcd58d421 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/sort/UnsafeShuffleWriterBenchmark.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/sort/UnsafeShuffleWriterBenchmark.scala @@ -16,12 +16,9 @@ */ package org.apache.spark.shuffle.sort -import java.io.{BufferedInputStream, File, FileInputStream, FileOutputStream} - -import scala.util.Random - import org.apache.spark.SparkConf import org.apache.spark.benchmark.Benchmark +import org.apache.spark.util.Utils /** * Benchmark to measure performance for aggregate primitives. @@ -33,7 +30,7 @@ import org.apache.spark.benchmark.Benchmark * Results will be written to "benchmarks/-results.txt". * }}} */ -object UnsafeShuffleWriterBenchmark extends ShuffleWriterBenchmarkBase(true) { +object UnsafeShuffleWriterBenchmark extends ShuffleWriterBenchmarkBase { private val shuffleHandle: SerializedShuffleHandle[String, String] = new SerializedShuffleHandle[String, String](0, 0, this.dependency) @@ -85,16 +82,22 @@ object UnsafeShuffleWriterBenchmark extends ShuffleWriterBenchmarkBase(true) { outputPerIteration = true) addBenchmarkCase(benchmark, "without transferTo") { timer => val shuffleWriter = constructWriter(false) - timer.startTiming() - shuffleWriter.write(DataIterator(inputFile = tempDataFile, DEFAULT_DATA_STRING_SIZE)) - timer.stopTiming() + Utils.tryWithResource(DataIterator(inputFile = tempDataFile, DEFAULT_DATA_STRING_SIZE)) { + iterator => + timer.startTiming() + shuffleWriter.write(iterator) + timer.stopTiming() + } assert(tempFilesCreated.length == 7) } addBenchmarkCase(benchmark, "with transferTo") { timer => val shuffleWriter = constructWriter(false) - timer.startTiming() - shuffleWriter.write(DataIterator(inputFile = tempDataFile, DEFAULT_DATA_STRING_SIZE)) - timer.stopTiming() + Utils.tryWithResource(DataIterator(inputFile = tempDataFile, DEFAULT_DATA_STRING_SIZE)) { + iterator => + timer.startTiming() + shuffleWriter.write(iterator) + timer.stopTiming() + } assert(tempFilesCreated.length == 7) } benchmark.run() From f72afb2c00797b1880af74ae8f72d5981bb39f47 Mon Sep 17 00:00:00 2001 From: Yifei Huang Date: Mon, 4 Mar 2019 11:03:17 -0800 Subject: [PATCH 36/68] get only git message --- dev/run-spark-25299-benchmarks.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dev/run-spark-25299-benchmarks.sh b/dev/run-spark-25299-benchmarks.sh index 09a5d466ea43c..60fc8243da659 100755 --- a/dev/run-spark-25299-benchmarks.sh +++ b/dev/run-spark-25299-benchmarks.sh @@ -48,7 +48,7 @@ for benchmark_file in /tmp/artifacts/*.txt; do done echo $RESULTS -PULL_REQUEST_NUM=$(git log -1 | sed "5q;d" | awk '{print $NF}' | sed 's/(//g' | sed 's/)//g' | sed 's/#//g') +PULL_REQUEST_NUM=$(git log -1 --pretty=%B | awk '{print $NF}' | sed 's/(//g' | sed 's/)//g' | sed 's/#//g') USERNAME=svc-spark-25299 From 3bcd35e3c4bc9c1826eb6eb5d3928ad62df4926c Mon Sep 17 00:00:00 2001 From: Yifei Huang Date: Mon, 4 Mar 2019 11:13:41 -0800 Subject: [PATCH 37/68] fix command to get PR number --- dev/run-spark-25299-benchmarks.sh | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/dev/run-spark-25299-benchmarks.sh b/dev/run-spark-25299-benchmarks.sh index 60fc8243da659..7c05288310c45 100755 --- a/dev/run-spark-25299-benchmarks.sh +++ b/dev/run-spark-25299-benchmarks.sh @@ -1,5 +1,9 @@ #!/usr/bin/env bash +<<<<<<< Updated upstream +======= + +>>>>>>> Stashed changes set -ou pipefail @@ -48,7 +52,8 @@ for benchmark_file in /tmp/artifacts/*.txt; do done echo $RESULTS -PULL_REQUEST_NUM=$(git log -1 --pretty=%B | awk '{print $NF}' | sed 's/(//g' | sed 's/)//g' | sed 's/#//g') +# Get last git message, filter out empty lines, get the last number of the first line. This is the PR number +PULL_REQUEST_NUM=$(git log -1 --pretty=%B | awk NF | awk '{print $NF}' | head -1 | sed 's/(//g' | sed 's/)//g' | sed 's/#//g') USERNAME=svc-spark-25299 From d8b5d79121a2095175ccc4a108e2c1b06772e6dd Mon Sep 17 00:00:00 2001 From: Yifei Huang Date: Mon, 4 Mar 2019 11:17:04 -0800 Subject: [PATCH 38/68] add SortshuffleWriterBenchmark --- dev/run-spark-25299-benchmarks.sh | 1 + 1 file changed, 1 insertion(+) diff --git a/dev/run-spark-25299-benchmarks.sh b/dev/run-spark-25299-benchmarks.sh index 7c05288310c45..e8e0cd29b0479 100755 --- a/dev/run-spark-25299-benchmarks.sh +++ b/dev/run-spark-25299-benchmarks.sh @@ -33,6 +33,7 @@ echo "Running SPARK-25299 benchmarks" SPARK_GENERATE_BENCHMARK_FILES=1 ./build/sbt "sql/test:runMain org.apache.spark.shuffle.sort.BypassMergeSortShuffleWriterBenchmark" SPARK_GENERATE_BENCHMARK_FILES=1 ./build/sbt "sql/test:runMain org.apache.spark.shuffle.sort.UnsafeShuffleWriterBenchmark" +SPARK_GENERATE_BENCHMARK_FILES=1 ./build/sbt "sql/test:runMain org.apache.spark.shuffle.sort.SortShuffleWriterBenchmark" SPARK_DIR=`pwd` From d9fb78a1d6428ccbf86a464b3341b75f47049cc7 Mon Sep 17 00:00:00 2001 From: Yifei Huang Date: Mon, 4 Mar 2019 15:14:41 -0800 Subject: [PATCH 39/68] writer code --- .../BlockStoreShuffleReaderBenchmark.scala | 351 ++++++++++-------- .../sort/ShuffleWriterBenchmarkBase.scala | 6 +- .../sort/SortShuffleWriterBenchmark.scala | 3 +- 3 files changed, 198 insertions(+), 162 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/shuffle/BlockStoreShuffleReaderBenchmark.scala b/core/src/test/scala/org/apache/spark/shuffle/BlockStoreShuffleReaderBenchmark.scala index d89cd9749e679..bf5289e1504a8 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/BlockStoreShuffleReaderBenchmark.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/BlockStoreShuffleReaderBenchmark.scala @@ -17,28 +17,28 @@ package org.apache.spark.shuffle import java.io.{File, FileOutputStream} -import java.nio.channels.ReadableByteChannel import com.google.common.io.CountingOutputStream -import org.mockito.{Mock, MockitoAnnotations} +import org.apache.commons.io.FileUtils +import org.mockito.{Mock, Mockito, MockitoAnnotations} import org.mockito.Answers.RETURNS_SMART_NULLS import org.mockito.Matchers._ import org.mockito.Mockito.when -import scala.concurrent.Future import scala.util.Random -import org.apache.spark.{MapOutputTracker, ShuffleDependency, SparkConf, SparkEnv, TaskContext} -import org.apache.spark.benchmark.BenchmarkBase +import org.apache.spark.{Aggregator, MapOutputTracker, ShuffleDependency, SparkConf, SparkEnv, TaskContext} +import org.apache.spark.benchmark.{Benchmark, BenchmarkBase} import org.apache.spark.executor.TaskMetrics import org.apache.spark.memory.TestMemoryManager +import org.apache.spark.metrics.source.Source import org.apache.spark.network.BlockTransferService import org.apache.spark.network.buffer.{FileSegmentManagedBuffer, ManagedBuffer} import org.apache.spark.network.netty.{NettyBlockTransferService, SparkTransportConf} import org.apache.spark.network.util.TransportConf -import org.apache.spark.rpc.{RpcAddress, RpcEndpoint, RpcEndpointRef, RpcEnv, RpcEnvFileServer} +import org.apache.spark.rpc.{RpcEndpoint, RpcEndpointRef, RpcEnv} import org.apache.spark.serializer.{KryoSerializer, SerializerManager} import org.apache.spark.storage.{BlockId, BlockManager, BlockManagerId, BlockManagerMaster, ShuffleBlockId} -import org.apache.spark.util.Utils +import org.apache.spark.util.{AccumulatorV2, TaskCompletionListener, TaskFailureListener, Utils} /** * Benchmark to measure performance for aggregate primitives. @@ -56,12 +56,22 @@ object BlockStoreShuffleReaderBenchmark extends BenchmarkBase { // so it shouldn't affect the performance significantly @Mock(answer = RETURNS_SMART_NULLS) private var dependency: ShuffleDependency[String, String, String] = _ - // used only to get metrics, so does not affect perf significantly - @Mock(answer = RETURNS_SMART_NULLS) private var taskContext: TaskContext = _ // only used to retrieve info about the maps at the beginning, doesn't affect perf @Mock(answer = RETURNS_SMART_NULLS) private var mapOutputTracker: MapOutputTracker = _ - // this is only used when initializing the block manager, so doesn't affect perf + // this is only used when initializing the BlockManager, so doesn't affect perf @Mock(answer = RETURNS_SMART_NULLS) private var blockManagerMaster: BlockManagerMaster = _ + // this is only used when initiating the BlockManager, for comms between master and executor + @Mock(answer = RETURNS_SMART_NULLS) private var rpcEnv: RpcEnv = _ + @Mock(answer = RETURNS_SMART_NULLS) protected var rpcEndpointRef: RpcEndpointRef = _ + + private var tempDir: File = _ + + private val SHUFFLE_ID: Int = 0 + private val REDUCE_ID: Int = 0 + private val NUM_MAPS: Int = 5 + private val DEFAULT_DATA_STRING_SIZE = 5 + private val TEST_DATA_SIZE = 10000000 + private val MIN_NUM_ITERS: Int = 10 private val defaultConf: SparkConf = new SparkConf() .set("spark.shuffle.compress", "false") @@ -70,35 +80,14 @@ object BlockStoreShuffleReaderBenchmark extends BenchmarkBase { .set("spark.app.id", "test-app") private val serializer: KryoSerializer = new KryoSerializer(defaultConf) private val serializerManager: SerializerManager = new SerializerManager(serializer, defaultConf) - private val execBlockManagerId: BlockManagerId = BlockManagerId("localhost", "localhost", 7000) - private val remoteBlockManagerId: BlockManagerId = BlockManagerId("localhost", "localhost", 7002) + private val execBlockManagerId: BlockManagerId = BlockManagerId("0", "localhost", 17000) + private val remoteBlockManagerId: BlockManagerId = BlockManagerId("0", "localhost", 17002) private val transportConf: TransportConf = SparkTransportConf.fromSparkConf(defaultConf, "shuffle") private val securityManager: org.apache.spark.SecurityManager = new org.apache.spark.SecurityManager(defaultConf) protected val memoryManager: TestMemoryManager = new TestMemoryManager(defaultConf) - private var tempDir: File = _ - - private val SHUFFLE_ID: Int = 0 - private val REDUCE_ID: Int = 0 - private val NUM_MAPS: Int = 1 - - private val DEFAULT_DATA_STRING_SIZE = 5 - - private val rpcEnv: RpcEnv = new RpcEnv(defaultConf) { - override def endpointRef(endpoint: RpcEndpoint): RpcEndpointRef = { null } - override def address: RpcAddress = null - override def setupEndpoint(name: String, endpoint: RpcEndpoint): RpcEndpointRef = { null } - override def asyncSetupEndpointRefByURI(uri: String): Future[RpcEndpointRef] = { null } - override def stop(endpoint: RpcEndpointRef): Unit = { } - override def shutdown(): Unit = { } - override def awaitTermination(): Unit = { } - override def deserialize[T](deserializationAction: () => T): T = { deserializationAction() } - override def fileServer: RpcEnvFileServer = { null } - override def openChannel(uri: String): ReadableByteChannel = { null } - } - class TestBlockManager(transferService: BlockTransferService, blockManagerMaster: BlockManagerMaster, dataFile: File, @@ -126,124 +115,40 @@ object BlockStoreShuffleReaderBenchmark extends BenchmarkBase { } private var blockManager : BlockManager = _ + private var externalBlockManager: BlockManager = _ - def setupTest(): Unit = { - MockitoAnnotations.initMocks(this) - when(blockManagerMaster.registerBlockManager( - any[BlockManagerId], any[Long], any[Long], any[RpcEndpointRef])).thenReturn(null) - val dataFileAndLength = generateDataOnDisk(10) - - val localShuffleClient = new NettyBlockTransferService( - defaultConf, - securityManager, - "localhost", - "localhost", - 7000, - 1 - ) - blockManager = - new TestBlockManager( - localShuffleClient, - blockManagerMaster, - dataFileAndLength._1, - dataFileAndLength._2) - blockManager.initialize(defaultConf.getAppId) - - val externalServer = new NettyBlockTransferService( + def getTestBlockManager(port: Int, dataFile: File, dataFileLength: Long): TestBlockManager = { + val shuffleClient = new NettyBlockTransferService( defaultConf, securityManager, "localhost", "localhost", - 7002, + port, 1 ) - - val externalBlockManager = new TestBlockManager( - externalServer, + new TestBlockManager(shuffleClient, blockManagerMaster, - dataFileAndLength._1, - dataFileAndLength._2) - externalBlockManager.initialize(defaultConf.getAppId) - - SparkEnv.set(new SparkEnv( - "0", - null, - serializer, - null, - serializerManager, - mapOutputTracker, - null, - null, - blockManager, - null, - null, - null, - null, - defaultConf - )) - - val shuffleHandle = new BaseShuffleHandle( - shuffleId = SHUFFLE_ID, - numMaps = NUM_MAPS, - dependency = dependency) - - val taskMetrics = new TaskMetrics - when(taskContext.taskMetrics()).thenReturn(taskMetrics) - - when(dependency.serializer).thenReturn(serializer) - when(mapOutputTracker.getMapSizesByExecutorId(SHUFFLE_ID, REDUCE_ID, REDUCE_ID + 1)) - .thenReturn { - val shuffleBlockIdsAndSizes = (0 until NUM_MAPS).map { mapId => - val shuffleBlockId = ShuffleBlockId(SHUFFLE_ID, mapId, REDUCE_ID) - (shuffleBlockId, 0L) - } - // TODO: configurable - Seq((remoteBlockManagerId, shuffleBlockIdsAndSizes)).toIterator - } - - // TODO: use aggregation + sort - when(dependency.aggregator).thenReturn(Option.empty) - when(dependency.keyOrdering).thenReturn(Option.empty) + dataFile, + dataFileLength) } - def setup(size: Int, fetchLocal: Boolean): BlockStoreShuffleReader[String, String] = { + def initializeServers(dataFile: File, dataFileLength: Long): Unit = { MockitoAnnotations.initMocks(this) when(blockManagerMaster.registerBlockManager( any[BlockManagerId], any[Long], any[Long], any[RpcEndpointRef])).thenReturn(null) - val dataFileAndLength = generateDataOnDisk(10) - - val localShuffleClient = new NettyBlockTransferService( - defaultConf, - securityManager, - "localhost", - "localhost", - 7000, - 1 - ) - blockManager = - new TestBlockManager( - localShuffleClient, - blockManagerMaster, - dataFileAndLength._1, - dataFileAndLength._2) + when(rpcEnv.setupEndpoint(any[String], any[RpcEndpoint])).thenReturn(rpcEndpointRef) + blockManager = getTestBlockManager(17000, dataFile, dataFileLength) blockManager.initialize(defaultConf.getAppId) - - val externalServer = new NettyBlockTransferService( - defaultConf, - securityManager, - "localhost", - "localhost", - 7002, - 1 - ) - - val externalBlockManager = new TestBlockManager( - externalServer, - blockManagerMaster, - dataFileAndLength._1, - dataFileAndLength._2) + externalBlockManager = getTestBlockManager(17002, dataFile, dataFileLength) externalBlockManager.initialize(defaultConf.getAppId) + } + def setup(size: Int, + dataFile: File, + dataFileLength: Long, + fetchLocal: Boolean, + aggregator: Option[Aggregator[String, String, String]] = None, + sorter: Option[Ordering[String]] = None): BlockStoreShuffleReader[String, String] = { SparkEnv.set(new SparkEnv( "0", null, @@ -266,26 +171,52 @@ object BlockStoreShuffleReaderBenchmark extends BenchmarkBase { numMaps = NUM_MAPS, dependency = dependency) - val taskMetrics = new TaskMetrics - when(taskContext.taskMetrics()).thenReturn(taskMetrics) + // We cannot mock the TaskContext because it taskMetrics() gets called at every next() + // call on the reader, and Mockito will try to log all calls to taskMetrics(), thus OOM-ing + // the test + val taskContext = new TaskContext { + private val metrics: TaskMetrics = new TaskMetrics + override def isCompleted(): Boolean = false + override def isInterrupted(): Boolean = false + override def addTaskCompletionListener(listener: TaskCompletionListener): + TaskContext = { null } + override def addTaskFailureListener(listener: TaskFailureListener): TaskContext = { null } + override def stageId(): Int = 0 + override def stageAttemptNumber(): Int = 0 + override def partitionId(): Int = 0 + override def attemptNumber(): Int = 0 + override def taskAttemptId(): Long = 0 + override def getLocalProperty(key: String): String = "" + override def taskMetrics(): TaskMetrics = metrics + override def getMetricsSources(sourceName: String): Seq[Source] = Seq.empty + override private[spark] def killTaskIfInterrupted(): Unit = {} + override private[spark] def getKillReason() = None + override private[spark] def taskMemoryManager() = { null } + override private[spark] def registerAccumulator(a: AccumulatorV2[_, _]): Unit = {} + override private[spark] def setFetchFailed(fetchFailed: FetchFailedException): Unit = {} + override private[spark] def markInterrupted(reason: String): Unit = {} + override private[spark] def markTaskFailed(error: Throwable): Unit = {} + override private[spark] def markTaskCompleted(error: Option[Throwable]): Unit = {} + override private[spark] def fetchFailed = None + override private[spark] def getLocalProperties = { null } + } + + var dataBlockId: BlockManagerId = execBlockManagerId + if (!fetchLocal) { + dataBlockId = remoteBlockManagerId + } - when(dependency.serializer).thenReturn(serializer) when(mapOutputTracker.getMapSizesByExecutorId(SHUFFLE_ID, REDUCE_ID, REDUCE_ID + 1)) .thenReturn { val shuffleBlockIdsAndSizes = (0 until NUM_MAPS).map { mapId => val shuffleBlockId = ShuffleBlockId(SHUFFLE_ID, mapId, REDUCE_ID) - (shuffleBlockId, DEFAULT_DATA_STRING_SIZE * size.toLong) + (shuffleBlockId, dataFileLength) } - // TODO: configurable - Seq((remoteBlockManagerId, shuffleBlockIdsAndSizes)).toIterator + Seq((dataBlockId, shuffleBlockIdsAndSizes)).toIterator } - if (fetchLocal) { - // to do - } else { - } - // TODO: use aggregation + sort + when(dependency.serializer).thenReturn(serializer) when(dependency.aggregator).thenReturn(Option.empty) when(dependency.keyOrdering).thenReturn(Option.empty) @@ -294,20 +225,18 @@ object BlockStoreShuffleReaderBenchmark extends BenchmarkBase { 0, 1, taskContext, - taskMetrics.createTempShuffleReadMetrics(), + taskContext.taskMetrics().createTempShuffleReadMetrics(), serializerManager, blockManager, mapOutputTracker ) } - def generateDataOnDisk(size: Int): (File, Long) = { + def generateDataOnDisk(size: Int, file: File): Long = { // scalastyle:off println - tempDir = Utils.createTempDir(null, "shuffle") - val tempDataFile: File = File.createTempFile("test-data", "", tempDir) println("Generating test data with num records: " + size) val random = new Random(123) - val dataOutput = new FileOutputStream(tempDataFile) + val dataOutput = new FileOutputStream(file) val coutingOutput = new CountingOutputStream(dataOutput) val serializedOutput = serializer.newInstance().serializeStream(coutingOutput) try { @@ -323,20 +252,124 @@ object BlockStoreShuffleReaderBenchmark extends BenchmarkBase { finally { serializedOutput.close() } - (tempDataFile, coutingOutput.getCount) + coutingOutput.getCount // scalastyle:off println } + + def countElements(reader: BlockStoreShuffleReader[String, String]): Long = { + val iterator = reader.read() + var count = 0 + while (iterator.hasNext) { + iterator.next() + count += 1 + } + count +// val count: Long = reader.read().toStream.foldLeft(0) { (acc, value) => acc + 1 } +// count + } + override def runBenchmarkSuite(mainArgs: Array[String]): Unit = { - val reader = setup(10, false) - // scalastyle:off println - println(reader.read().length) - println(reader.read().length) - // scalastyle:on println -// assert(reader.read().length == 10 * NUM_MAPS) + tempDir = Utils.createTempDir(null, "shuffle") + val tempDataFile: File = File.createTempFile("test-data", "", tempDir) + val dataFileLength = generateDataOnDisk(TEST_DATA_SIZE, tempDataFile) + initializeServers(tempDataFile, dataFileLength) runBenchmark("SortShuffleWriter writer") { - // todo + val baseBenchmark = + new Benchmark("no aggregation or sorting", + TEST_DATA_SIZE, + minNumIters = MIN_NUM_ITERS, + output = output, + outputPerIteration = true) + baseBenchmark.addTimerCase("local fetch") { timer => + val reader = setup(TEST_DATA_SIZE, tempDataFile, dataFileLength, fetchLocal = true) + timer.startTiming() + val numRead = countElements(reader) + timer.stopTiming() + assert(numRead == TEST_DATA_SIZE * NUM_MAPS) + } + baseBenchmark.addTimerCase("remote rpc fetch") { timer => + val reader = setup(TEST_DATA_SIZE, tempDataFile, dataFileLength, fetchLocal = false) + timer.startTiming() + val numRead = reader.read().length + timer.stopTiming() + assert(numRead == TEST_DATA_SIZE * NUM_MAPS) + } + baseBenchmark.run() + + + 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) + + val aggregationBenchmark = + new Benchmark("with aggregation", + TEST_DATA_SIZE, + minNumIters = MIN_NUM_ITERS, + output = output, + outputPerIteration = true) + aggregationBenchmark.addTimerCase("local fetch") { timer => + val reader = setup(TEST_DATA_SIZE, + tempDataFile, + dataFileLength, + fetchLocal = true, + aggregator = Some(aggregator)) + timer.startTiming() + val numRead = reader.read().length + timer.stopTiming() + assert(numRead == TEST_DATA_SIZE * NUM_MAPS) + } + aggregationBenchmark.addTimerCase("remote rpc fetch") { timer => + val reader = setup(TEST_DATA_SIZE, + tempDataFile, + dataFileLength, + fetchLocal = false, + aggregator = Some(aggregator)) + timer.startTiming() + val numRead = reader.read().length + timer.stopTiming() + assert(numRead == TEST_DATA_SIZE * NUM_MAPS) + } + aggregationBenchmark.run() + + + val sorter = Ordering.String + val sortingBenchmark = + new Benchmark("with sorting", + TEST_DATA_SIZE, + minNumIters = MIN_NUM_ITERS, + output = output, + outputPerIteration = true) + sortingBenchmark.addTimerCase("local fetch") { timer => + val reader = setup(TEST_DATA_SIZE, + tempDataFile, + dataFileLength, + fetchLocal = true, + sorter = Some(sorter)) + timer.startTiming() + val numRead = reader.read().length + timer.stopTiming() + assert(numRead == TEST_DATA_SIZE * NUM_MAPS) + } + sortingBenchmark.addTimerCase("remote rpc fetch") { timer => + val reader = setup(TEST_DATA_SIZE, + tempDataFile, + dataFileLength, + fetchLocal = false, + sorter = Some(sorter)) + timer.startTiming() + val numRead = reader.read().length + timer.stopTiming() + assert(numRead == TEST_DATA_SIZE * NUM_MAPS) + } + sortingBenchmark.run() + } + + FileUtils.deleteDirectory(tempDir) } } diff --git a/core/src/test/scala/org/apache/spark/shuffle/sort/ShuffleWriterBenchmarkBase.scala b/core/src/test/scala/org/apache/spark/shuffle/sort/ShuffleWriterBenchmarkBase.scala index 8946052dd5f37..b2efdf0761e7b 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/sort/ShuffleWriterBenchmarkBase.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/sort/ShuffleWriterBenchmarkBase.scala @@ -87,7 +87,11 @@ abstract class ShuffleWriterBenchmarkBase extends BenchmarkBase { serializerManager, defaultConf, memoryManager, - null, null, null, null, 1) { + null, + null, + null, + null, + 1) { override val diskBlockManager = new TestDiskBlockManager(tempDir) } diff --git a/core/src/test/scala/org/apache/spark/shuffle/sort/SortShuffleWriterBenchmark.scala b/core/src/test/scala/org/apache/spark/shuffle/sort/SortShuffleWriterBenchmark.scala index 9c53ce5d09f4c..5ed971bff0a8d 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/sort/SortShuffleWriterBenchmark.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/sort/SortShuffleWriterBenchmark.scala @@ -46,8 +46,7 @@ object SortShuffleWriterBenchmark extends ShuffleWriterBenchmarkBase { private val MIN_NUM_ITERS = 5 def constructWriter(aggregator: Option[Aggregator[String, String, String]], - sorter: Option[Ordering[String]]): - SortShuffleWriter[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 val defaultSparkEnv = SparkEnv.get SparkEnv.set(new SparkEnv( From b142951fdf5bf986216bc4c02e54694c117db925 Mon Sep 17 00:00:00 2001 From: Yifei Huang Date: Mon, 4 Mar 2019 17:32:00 -0800 Subject: [PATCH 40/68] cleanup --- .../BlockStoreShuffleReaderBenchmark.scala | 97 ++++++++++--------- .../sort/ShuffleWriterBenchmarkBase.scala | 8 +- .../sort/SortShuffleWriterBenchmark.scala | 12 +-- .../sort/UnsafeShuffleWriterBenchmark.scala | 8 +- dev/run-spark-25299-benchmarks.sh | 4 +- 5 files changed, 68 insertions(+), 61 deletions(-) rename core/src/test/scala/org/apache/spark/shuffle/{ => sort}/BlockStoreShuffleReaderBenchmark.scala (84%) diff --git a/core/src/test/scala/org/apache/spark/shuffle/BlockStoreShuffleReaderBenchmark.scala b/core/src/test/scala/org/apache/spark/shuffle/sort/BlockStoreShuffleReaderBenchmark.scala similarity index 84% rename from core/src/test/scala/org/apache/spark/shuffle/BlockStoreShuffleReaderBenchmark.scala rename to core/src/test/scala/org/apache/spark/shuffle/sort/BlockStoreShuffleReaderBenchmark.scala index bf5289e1504a8..946848bb2650a 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/BlockStoreShuffleReaderBenchmark.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/sort/BlockStoreShuffleReaderBenchmark.scala @@ -14,13 +14,13 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.shuffle +package org.apache.spark.shuffle.sort import java.io.{File, FileOutputStream} import com.google.common.io.CountingOutputStream import org.apache.commons.io.FileUtils -import org.mockito.{Mock, Mockito, MockitoAnnotations} +import org.mockito.{Mock, MockitoAnnotations} import org.mockito.Answers.RETURNS_SMART_NULLS import org.mockito.Matchers._ import org.mockito.Mockito.when @@ -29,7 +29,7 @@ import scala.util.Random import org.apache.spark.{Aggregator, MapOutputTracker, ShuffleDependency, SparkConf, SparkEnv, TaskContext} import org.apache.spark.benchmark.{Benchmark, BenchmarkBase} import org.apache.spark.executor.TaskMetrics -import org.apache.spark.memory.TestMemoryManager +import org.apache.spark.memory.{TaskMemoryManager, TestMemoryManager} import org.apache.spark.metrics.source.Source import org.apache.spark.network.BlockTransferService import org.apache.spark.network.buffer.{FileSegmentManagedBuffer, ManagedBuffer} @@ -37,6 +37,7 @@ import org.apache.spark.network.netty.{NettyBlockTransferService, SparkTransport import org.apache.spark.network.util.TransportConf import org.apache.spark.rpc.{RpcEndpoint, RpcEndpointRef, RpcEnv} import org.apache.spark.serializer.{KryoSerializer, SerializerManager} +import org.apache.spark.shuffle.{BaseShuffleHandle, BlockStoreShuffleReader, FetchFailedException} import org.apache.spark.storage.{BlockId, BlockManager, BlockManagerId, BlockManagerMaster, ShuffleBlockId} import org.apache.spark.util.{AccumulatorV2, TaskCompletionListener, TaskFailureListener, Utils} @@ -70,9 +71,14 @@ object BlockStoreShuffleReaderBenchmark extends BenchmarkBase { private val REDUCE_ID: Int = 0 private val NUM_MAPS: Int = 5 private val DEFAULT_DATA_STRING_SIZE = 5 - private val TEST_DATA_SIZE = 10000000 + private val TEST_DATA_SIZE: Int = 10000000 + private val SORT_DATA_SIZE: Int = 1000000 private val MIN_NUM_ITERS: Int = 10 + private val executorId: String = "0" + private val localPort: Int = 17000 + private val remotePort: Int = 17002 + private val defaultConf: SparkConf = new SparkConf() .set("spark.shuffle.compress", "false") .set("spark.shuffle.spill.compress", "false") @@ -80,8 +86,10 @@ object BlockStoreShuffleReaderBenchmark extends BenchmarkBase { .set("spark.app.id", "test-app") private val serializer: KryoSerializer = new KryoSerializer(defaultConf) private val serializerManager: SerializerManager = new SerializerManager(serializer, defaultConf) - private val execBlockManagerId: BlockManagerId = BlockManagerId("0", "localhost", 17000) - private val remoteBlockManagerId: BlockManagerId = BlockManagerId("0", "localhost", 17002) + private val execBlockManagerId: BlockManagerId = + BlockManagerId(executorId, "localhost", localPort) + private val remoteBlockManagerId: BlockManagerId = + BlockManagerId(executorId, "localhost", remotePort) private val transportConf: TransportConf = SparkTransportConf.fromSparkConf(defaultConf, "shuffle") private val securityManager: org.apache.spark.SecurityManager = @@ -91,7 +99,8 @@ object BlockStoreShuffleReaderBenchmark extends BenchmarkBase { class TestBlockManager(transferService: BlockTransferService, blockManagerMaster: BlockManagerMaster, dataFile: File, - fileLength: Long) extends BlockManager("0", + fileLength: Long) extends BlockManager( + executorId, rpcEnv, blockManagerMaster, serializerManager, @@ -137,14 +146,18 @@ object BlockStoreShuffleReaderBenchmark extends BenchmarkBase { when(blockManagerMaster.registerBlockManager( any[BlockManagerId], any[Long], any[Long], any[RpcEndpointRef])).thenReturn(null) when(rpcEnv.setupEndpoint(any[String], any[RpcEndpoint])).thenReturn(rpcEndpointRef) - blockManager = getTestBlockManager(17000, dataFile, dataFileLength) + blockManager = getTestBlockManager(localPort, dataFile, dataFileLength) blockManager.initialize(defaultConf.getAppId) - externalBlockManager = getTestBlockManager(17002, dataFile, dataFileLength) + externalBlockManager = getTestBlockManager(remotePort, dataFile, dataFileLength) externalBlockManager.initialize(defaultConf.getAppId) } - def setup(size: Int, - dataFile: File, + def stopServers(): Unit = { + blockManager.stop() + externalBlockManager.stop() + } + + def setup(dataFile: File, dataFileLength: Long, fetchLocal: Boolean, aggregator: Option[Aggregator[String, String, String]] = None, @@ -176,6 +189,9 @@ object BlockStoreShuffleReaderBenchmark extends BenchmarkBase { // the test val taskContext = new TaskContext { private val metrics: TaskMetrics = new TaskMetrics + private val testMemManager = new TestMemoryManager(defaultConf) + private val taskMemManager = new TaskMemoryManager(testMemManager, 0) + testMemManager.limit(PackedRecordPointer.MAXIMUM_PAGE_SIZE_BYTES) override def isCompleted(): Boolean = false override def isInterrupted(): Boolean = false override def addTaskCompletionListener(listener: TaskCompletionListener): @@ -191,7 +207,7 @@ object BlockStoreShuffleReaderBenchmark extends BenchmarkBase { override def getMetricsSources(sourceName: String): Seq[Source] = Seq.empty override private[spark] def killTaskIfInterrupted(): Unit = {} override private[spark] def getKillReason() = None - override private[spark] def taskMemoryManager() = { null } + override private[spark] def taskMemoryManager() = taskMemManager override private[spark] def registerAccumulator(a: AccumulatorV2[_, _]): Unit = {} override private[spark] def setFetchFailed(fetchFailed: FetchFailedException): Unit = {} override private[spark] def markInterrupted(reason: String): Unit = {} @@ -200,6 +216,7 @@ object BlockStoreShuffleReaderBenchmark extends BenchmarkBase { override private[spark] def fetchFailed = None override private[spark] def getLocalProperties = { null } } + TaskContext.setTaskContext(taskContext) var dataBlockId: BlockManagerId = execBlockManagerId if (!fetchLocal) { @@ -215,10 +232,9 @@ object BlockStoreShuffleReaderBenchmark extends BenchmarkBase { Seq((dataBlockId, shuffleBlockIdsAndSizes)).toIterator } - // TODO: use aggregation + sort when(dependency.serializer).thenReturn(serializer) - when(dependency.aggregator).thenReturn(Option.empty) - when(dependency.keyOrdering).thenReturn(Option.empty) + when(dependency.aggregator).thenReturn(aggregator) + when(dependency.keyOrdering).thenReturn(sorter) new BlockStoreShuffleReader[String, String]( shuffleHandle, @@ -256,19 +272,6 @@ object BlockStoreShuffleReaderBenchmark extends BenchmarkBase { // scalastyle:off println } - - def countElements(reader: BlockStoreShuffleReader[String, String]): Long = { - val iterator = reader.read() - var count = 0 - while (iterator.hasNext) { - iterator.next() - count += 1 - } - count -// val count: Long = reader.read().toStream.foldLeft(0) { (acc, value) => acc + 1 } -// count - } - override def runBenchmarkSuite(mainArgs: Array[String]): Unit = { tempDir = Utils.createTempDir(null, "shuffle") val tempDataFile: File = File.createTempFile("test-data", "", tempDir) @@ -283,14 +286,14 @@ object BlockStoreShuffleReaderBenchmark extends BenchmarkBase { output = output, outputPerIteration = true) baseBenchmark.addTimerCase("local fetch") { timer => - val reader = setup(TEST_DATA_SIZE, tempDataFile, dataFileLength, fetchLocal = true) + val reader = setup(tempDataFile, dataFileLength, fetchLocal = true) timer.startTiming() - val numRead = countElements(reader) + val numRead = reader.read().length timer.stopTiming() assert(numRead == TEST_DATA_SIZE * NUM_MAPS) } baseBenchmark.addTimerCase("remote rpc fetch") { timer => - val reader = setup(TEST_DATA_SIZE, tempDataFile, dataFileLength, fetchLocal = false) + val reader = setup(tempDataFile, dataFileLength, fetchLocal = false) timer.startTiming() val numRead = reader.read().length timer.stopTiming() @@ -313,7 +316,7 @@ object BlockStoreShuffleReaderBenchmark extends BenchmarkBase { output = output, outputPerIteration = true) aggregationBenchmark.addTimerCase("local fetch") { timer => - val reader = setup(TEST_DATA_SIZE, + val reader = setup( tempDataFile, dataFileLength, fetchLocal = true, @@ -321,10 +324,10 @@ object BlockStoreShuffleReaderBenchmark extends BenchmarkBase { timer.startTiming() val numRead = reader.read().length timer.stopTiming() - assert(numRead == TEST_DATA_SIZE * NUM_MAPS) + assert(numRead > 0) } aggregationBenchmark.addTimerCase("remote rpc fetch") { timer => - val reader = setup(TEST_DATA_SIZE, + val reader = setup( tempDataFile, dataFileLength, fetchLocal = false, @@ -332,44 +335,48 @@ object BlockStoreShuffleReaderBenchmark extends BenchmarkBase { timer.startTiming() val numRead = reader.read().length timer.stopTiming() - assert(numRead == TEST_DATA_SIZE * NUM_MAPS) + assert(numRead > 0) } aggregationBenchmark.run() + stopServers() + val sortDataFile: File = File.createTempFile("test-data", "", tempDir) + val sortFileLength = generateDataOnDisk(SORT_DATA_SIZE, sortDataFile) + initializeServers(sortDataFile, sortFileLength) val sorter = Ordering.String val sortingBenchmark = new Benchmark("with sorting", - TEST_DATA_SIZE, + SORT_DATA_SIZE, minNumIters = MIN_NUM_ITERS, output = output, outputPerIteration = true) sortingBenchmark.addTimerCase("local fetch") { timer => - val reader = setup(TEST_DATA_SIZE, - tempDataFile, - dataFileLength, + val reader = setup( + sortDataFile, + sortFileLength, fetchLocal = true, sorter = Some(sorter)) timer.startTiming() val numRead = reader.read().length timer.stopTiming() - assert(numRead == TEST_DATA_SIZE * NUM_MAPS) + assert(numRead == SORT_DATA_SIZE * NUM_MAPS) } sortingBenchmark.addTimerCase("remote rpc fetch") { timer => - val reader = setup(TEST_DATA_SIZE, - tempDataFile, - dataFileLength, + val reader = setup( + sortDataFile, + sortFileLength, fetchLocal = false, sorter = Some(sorter)) timer.startTiming() val numRead = reader.read().length timer.stopTiming() - assert(numRead == TEST_DATA_SIZE * NUM_MAPS) + assert(numRead == SORT_DATA_SIZE * NUM_MAPS) } sortingBenchmark.run() - } + stopServers() FileUtils.deleteDirectory(tempDir) } } diff --git a/core/src/test/scala/org/apache/spark/shuffle/sort/ShuffleWriterBenchmarkBase.scala b/core/src/test/scala/org/apache/spark/shuffle/sort/ShuffleWriterBenchmarkBase.scala index b2efdf0761e7b..2754383cbedd8 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/sort/ShuffleWriterBenchmarkBase.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/sort/ShuffleWriterBenchmarkBase.scala @@ -25,7 +25,6 @@ import org.mockito.{Mock, MockitoAnnotations} import org.mockito.Answers.RETURNS_SMART_NULLS import org.mockito.Matchers.any import org.mockito.Mockito.when -import org.slf4j.{Logger, LoggerFactory} import scala.collection.mutable import scala.collection.mutable.ArrayBuffer import scala.util.Random @@ -42,7 +41,6 @@ import org.apache.spark.util.Utils abstract class ShuffleWriterBenchmarkBase extends BenchmarkBase { - private def log: Logger = LoggerFactory.getLogger(this.getClass.getName.stripSuffix("$")) private val DEFAULT_DATA_STRING_SIZE = 5 // This is only used in the writer constructors, so it's ok to mock @@ -174,18 +172,20 @@ abstract class ShuffleWriterBenchmarkBase extends BenchmarkBase { } def createDataOnDisk(size: Int): File = { - log.info("Generating test data with num records: " + size) + // scalastyle:off println + println("Generating test data with num records: " + size) val tempDataFile = File.createTempFile("test-data", "") Utils.tryWithResource(new FileOutputStream(tempDataFile)) { dataOutput => (1 to size).foreach { i => { if (i % 1000000 == 0) { - log.info("Wrote " + i + " test data points") + println("Wrote " + i + " test data points") } val x = random.alphanumeric.take(DEFAULT_DATA_STRING_SIZE).mkString dataOutput.write(x.getBytes) }} } + // scalastyle:on println tempDataFile } diff --git a/core/src/test/scala/org/apache/spark/shuffle/sort/SortShuffleWriterBenchmark.scala b/core/src/test/scala/org/apache/spark/shuffle/sort/SortShuffleWriterBenchmark.scala index 5ed971bff0a8d..56cf65b202df1 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/sort/SortShuffleWriterBenchmark.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/sort/SortShuffleWriterBenchmark.scala @@ -45,8 +45,8 @@ object SortShuffleWriterBenchmark extends ShuffleWriterBenchmarkBase { private val DEFAULT_DATA_STRING_SIZE = 5 private val MIN_NUM_ITERS = 5 - def constructWriter(aggregator: Option[Aggregator[String, String, String]], - sorter: Option[Ordering[String]]): SortShuffleWriter[String, String, String] = { + 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 val defaultSparkEnv = SparkEnv.get SparkEnv.set(new SparkEnv( @@ -92,7 +92,7 @@ object SortShuffleWriterBenchmark extends ShuffleWriterBenchmarkBase { minNumIters = MIN_NUM_ITERS, output = output) addBenchmarkCase(benchmark, "small dataset without spills") { timer => - val writer = constructWriter(Option.empty, Option.empty) + val writer = getWriter(Option.empty, Option.empty) val array = createDataInMemory(1000) timer.startTiming() writer.write(array.iterator) @@ -112,7 +112,7 @@ object SortShuffleWriterBenchmark extends ShuffleWriterBenchmarkBase { output = output, outputPerIteration = true) addBenchmarkCase(benchmark, "no map side combine") { timer => - val shuffleWriter = constructWriter(Option.empty, Option.empty) + val shuffleWriter = getWriter(Option.empty, Option.empty) Utils.tryWithResource(DataIterator(inputFile = dataFile, DEFAULT_DATA_STRING_SIZE)) { iterator => timer.startTiming() @@ -129,7 +129,7 @@ object SortShuffleWriterBenchmark extends ShuffleWriterBenchmarkBase { val aggregator = new Aggregator[String, String, String](createCombiner, mergeValue, mergeCombiners) addBenchmarkCase(benchmark, "with map side aggregation") { timer => - val shuffleWriter = constructWriter(Some(aggregator), Option.empty) + val shuffleWriter = getWriter(Some(aggregator), Option.empty) Utils.tryWithResource(DataIterator(inputFile = dataFile, DEFAULT_DATA_STRING_SIZE)) { iterator => timer.startTiming() @@ -141,7 +141,7 @@ object SortShuffleWriterBenchmark extends ShuffleWriterBenchmarkBase { val sorter = Ordering.String addBenchmarkCase(benchmark, "with map side sort") { timer => - val shuffleWriter = constructWriter(Option.empty, Some(sorter)) + val shuffleWriter = getWriter(Option.empty, Some(sorter)) val iterator: DataIterator = DataIterator(inputFile = dataFile, DEFAULT_DATA_STRING_SIZE) Utils.tryWithResource(DataIterator(inputFile = dataFile, DEFAULT_DATA_STRING_SIZE)) { iterator => diff --git a/core/src/test/scala/org/apache/spark/shuffle/sort/UnsafeShuffleWriterBenchmark.scala b/core/src/test/scala/org/apache/spark/shuffle/sort/UnsafeShuffleWriterBenchmark.scala index 3ba0fcd58d421..76d6cd258eec5 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/sort/UnsafeShuffleWriterBenchmark.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/sort/UnsafeShuffleWriterBenchmark.scala @@ -38,7 +38,7 @@ object UnsafeShuffleWriterBenchmark extends ShuffleWriterBenchmarkBase { private val DEFAULT_DATA_STRING_SIZE = 5 private val MIN_NUM_ITERS = 5 - def constructWriter(transferTo: Boolean): UnsafeShuffleWriter[String, String] = { + def getWriter(transferTo: Boolean): UnsafeShuffleWriter[String, String] = { val conf = new SparkConf(loadDefaults = false) conf.set("spark.file.transferTo", String.valueOf(transferTo)) @@ -61,7 +61,7 @@ object UnsafeShuffleWriterBenchmark extends ShuffleWriterBenchmarkBase { minNumIters = MIN_NUM_ITERS, output = output) addBenchmarkCase(benchmark, "small dataset without spills") { timer => - val writer = constructWriter(false) + val writer = getWriter(false) val array = createDataInMemory(1000) timer.startTiming() writer.write(array.iterator) @@ -81,7 +81,7 @@ object UnsafeShuffleWriterBenchmark extends ShuffleWriterBenchmarkBase { output = output, outputPerIteration = true) addBenchmarkCase(benchmark, "without transferTo") { timer => - val shuffleWriter = constructWriter(false) + val shuffleWriter = getWriter(false) Utils.tryWithResource(DataIterator(inputFile = tempDataFile, DEFAULT_DATA_STRING_SIZE)) { iterator => timer.startTiming() @@ -91,7 +91,7 @@ object UnsafeShuffleWriterBenchmark extends ShuffleWriterBenchmarkBase { assert(tempFilesCreated.length == 7) } addBenchmarkCase(benchmark, "with transferTo") { timer => - val shuffleWriter = constructWriter(false) + val shuffleWriter = getWriter(false) Utils.tryWithResource(DataIterator(inputFile = tempDataFile, DEFAULT_DATA_STRING_SIZE)) { iterator => timer.startTiming() diff --git a/dev/run-spark-25299-benchmarks.sh b/dev/run-spark-25299-benchmarks.sh index e8e0cd29b0479..f6d66aa47b38e 100755 --- a/dev/run-spark-25299-benchmarks.sh +++ b/dev/run-spark-25299-benchmarks.sh @@ -34,12 +34,12 @@ echo "Running SPARK-25299 benchmarks" SPARK_GENERATE_BENCHMARK_FILES=1 ./build/sbt "sql/test:runMain org.apache.spark.shuffle.sort.BypassMergeSortShuffleWriterBenchmark" SPARK_GENERATE_BENCHMARK_FILES=1 ./build/sbt "sql/test:runMain org.apache.spark.shuffle.sort.UnsafeShuffleWriterBenchmark" SPARK_GENERATE_BENCHMARK_FILES=1 ./build/sbt "sql/test:runMain org.apache.spark.shuffle.sort.SortShuffleWriterBenchmark" +SPARK_GENERATE_BENCHMARK_FILES=1 ./build/sbt "sql/test:runMain org.apache.spark.shuffle.sort.BlockStoreShuffleReaderBenchmark" SPARK_DIR=`pwd` mkdir -p /tmp/artifacts -cp $SPARK_DIR/sql/core/benchmarks/BypassMergeSortShuffleWriterBenchmark-results.txt /tmp/artifacts/ -cp $SPARK_DIR/sql/core/benchmarks/UnsafeShuffleWriterBenchmark-results.txt /tmp/artifacts/ +cp $SPARK_DIR/sql/core/benchmarks/* /tmp/artifacts/ if [ "$UPLOAD" = false ]; then exit 0 From f91dfad205911508e13bb58ee9d04d8642c0590c Mon Sep 17 00:00:00 2001 From: Yifei Huang Date: Mon, 4 Mar 2019 17:34:40 -0800 Subject: [PATCH 41/68] fix benchmark script --- dev/run-spark-25299-benchmarks.sh | 4 ---- 1 file changed, 4 deletions(-) diff --git a/dev/run-spark-25299-benchmarks.sh b/dev/run-spark-25299-benchmarks.sh index f6d66aa47b38e..e6604aad53d4d 100755 --- a/dev/run-spark-25299-benchmarks.sh +++ b/dev/run-spark-25299-benchmarks.sh @@ -1,9 +1,5 @@ #!/usr/bin/env bash -<<<<<<< Updated upstream -======= - ->>>>>>> Stashed changes set -ou pipefail From 5839b1d451257470cb71b4a60f01ad6d16c7c54b Mon Sep 17 00:00:00 2001 From: Yifei Huang Date: Mon, 4 Mar 2019 17:41:06 -0800 Subject: [PATCH 42/68] use ArgumentMatchers --- .../spark/shuffle/sort/BlockStoreShuffleReaderBenchmark.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/test/scala/org/apache/spark/shuffle/sort/BlockStoreShuffleReaderBenchmark.scala b/core/src/test/scala/org/apache/spark/shuffle/sort/BlockStoreShuffleReaderBenchmark.scala index 946848bb2650a..f1f437e910781 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/sort/BlockStoreShuffleReaderBenchmark.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/sort/BlockStoreShuffleReaderBenchmark.scala @@ -21,8 +21,8 @@ import java.io.{File, FileOutputStream} import com.google.common.io.CountingOutputStream import org.apache.commons.io.FileUtils import org.mockito.{Mock, MockitoAnnotations} +import org.mockito.ArgumentMatchers.any import org.mockito.Answers.RETURNS_SMART_NULLS -import org.mockito.Matchers._ import org.mockito.Mockito.when import scala.util.Random From 0b8c7ed641be205214cad80246d5cc0a559a8105 Mon Sep 17 00:00:00 2001 From: Yifei Huang Date: Mon, 4 Mar 2019 17:50:36 -0800 Subject: [PATCH 43/68] also in shufflewriterbenchmarkbase --- .../apache/spark/shuffle/sort/ShuffleWriterBenchmarkBase.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/test/scala/org/apache/spark/shuffle/sort/ShuffleWriterBenchmarkBase.scala b/core/src/test/scala/org/apache/spark/shuffle/sort/ShuffleWriterBenchmarkBase.scala index 2754383cbedd8..6aa679deb9bdf 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/sort/ShuffleWriterBenchmarkBase.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/sort/ShuffleWriterBenchmarkBase.scala @@ -23,7 +23,7 @@ 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.Matchers.any +import org.mockito.ArgumentMatchers.any import org.mockito.Mockito.when import scala.collection.mutable import scala.collection.mutable.ArrayBuffer From d11f87f64a650936417e6b32bac10db789498027 Mon Sep 17 00:00:00 2001 From: Yifei Huang Date: Mon, 4 Mar 2019 17:58:08 -0800 Subject: [PATCH 44/68] scalastyle --- .../spark/shuffle/sort/BlockStoreShuffleReaderBenchmark.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/test/scala/org/apache/spark/shuffle/sort/BlockStoreShuffleReaderBenchmark.scala b/core/src/test/scala/org/apache/spark/shuffle/sort/BlockStoreShuffleReaderBenchmark.scala index f1f437e910781..895f465a13440 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/sort/BlockStoreShuffleReaderBenchmark.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/sort/BlockStoreShuffleReaderBenchmark.scala @@ -21,8 +21,8 @@ import java.io.{File, FileOutputStream} import com.google.common.io.CountingOutputStream import org.apache.commons.io.FileUtils import org.mockito.{Mock, MockitoAnnotations} -import org.mockito.ArgumentMatchers.any import org.mockito.Answers.RETURNS_SMART_NULLS +import org.mockito.ArgumentMatchers.any import org.mockito.Mockito.when import scala.util.Random From 6f2779f12f2690d119792c7de49a1de818233162 Mon Sep 17 00:00:00 2001 From: Yifei Huang Date: Mon, 4 Mar 2019 21:40:42 -0800 Subject: [PATCH 45/68] add apache license --- dev/run-spark-25299-benchmarks.sh | 23 +++++++++++++++++++++++ 1 file changed, 23 insertions(+) diff --git a/dev/run-spark-25299-benchmarks.sh b/dev/run-spark-25299-benchmarks.sh index e6604aad53d4d..b63934ec19656 100755 --- a/dev/run-spark-25299-benchmarks.sh +++ b/dev/run-spark-25299-benchmarks.sh @@ -1,4 +1,27 @@ #!/usr/bin/env bash +# +# 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. +# + +# +# Script to create a binary distribution for easy deploys of Spark. +# The distribution directory defaults to dist/ but can be overridden below. +# The distribution contains fat (assembly) jars that include the Scala library, +# so it is completely self contained. +# It does not contain source or *.class files. set -ou pipefail From bbe9edcc53f7146a9a5d4694a53b99366ebe0dfc Mon Sep 17 00:00:00 2001 From: Yifei Huang Date: Tue, 5 Mar 2019 11:20:06 -0800 Subject: [PATCH 46/68] fix some scale stuff --- .../BlockStoreShuffleReaderBenchmark.scala | 36 +++++++++---------- .../sort/SortShuffleWriterBenchmark.scala | 1 - 2 files changed, 18 insertions(+), 19 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/shuffle/sort/BlockStoreShuffleReaderBenchmark.scala b/core/src/test/scala/org/apache/spark/shuffle/sort/BlockStoreShuffleReaderBenchmark.scala index 895f465a13440..9b0e6085e87ab 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/sort/BlockStoreShuffleReaderBenchmark.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/sort/BlockStoreShuffleReaderBenchmark.scala @@ -72,7 +72,7 @@ object BlockStoreShuffleReaderBenchmark extends BenchmarkBase { private val NUM_MAPS: Int = 5 private val DEFAULT_DATA_STRING_SIZE = 5 private val TEST_DATA_SIZE: Int = 10000000 - private val SORT_DATA_SIZE: Int = 1000000 + private val SMALLER_DATA_SIZE: Int = 1000000 private val MIN_NUM_ITERS: Int = 10 private val executorId: String = "0" @@ -302,23 +302,27 @@ object BlockStoreShuffleReaderBenchmark extends BenchmarkBase { baseBenchmark.run() + stopServers() + val smallerDataFile: File = File.createTempFile("test-data", "", tempDir) + val smallerFileLength = generateDataOnDisk(SMALLER_DATA_SIZE, smallerDataFile) + initializeServers(smallerDataFile, smallerFileLength) + 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) - val aggregationBenchmark = new Benchmark("with aggregation", - TEST_DATA_SIZE, + SMALLER_DATA_SIZE, minNumIters = MIN_NUM_ITERS, output = output, outputPerIteration = true) aggregationBenchmark.addTimerCase("local fetch") { timer => val reader = setup( - tempDataFile, - dataFileLength, + smallerDataFile, + smallerFileLength, fetchLocal = true, aggregator = Some(aggregator)) timer.startTiming() @@ -328,8 +332,8 @@ object BlockStoreShuffleReaderBenchmark extends BenchmarkBase { } aggregationBenchmark.addTimerCase("remote rpc fetch") { timer => val reader = setup( - tempDataFile, - dataFileLength, + smallerDataFile, + smallerFileLength, fetchLocal = false, aggregator = Some(aggregator)) timer.startTiming() @@ -340,38 +344,34 @@ object BlockStoreShuffleReaderBenchmark extends BenchmarkBase { aggregationBenchmark.run() - stopServers() - val sortDataFile: File = File.createTempFile("test-data", "", tempDir) - val sortFileLength = generateDataOnDisk(SORT_DATA_SIZE, sortDataFile) - initializeServers(sortDataFile, sortFileLength) val sorter = Ordering.String val sortingBenchmark = new Benchmark("with sorting", - SORT_DATA_SIZE, + SMALLER_DATA_SIZE, minNumIters = MIN_NUM_ITERS, output = output, outputPerIteration = true) sortingBenchmark.addTimerCase("local fetch") { timer => val reader = setup( - sortDataFile, - sortFileLength, + smallerDataFile, + smallerFileLength, fetchLocal = true, sorter = Some(sorter)) timer.startTiming() val numRead = reader.read().length timer.stopTiming() - assert(numRead == SORT_DATA_SIZE * NUM_MAPS) + assert(numRead == SMALLER_DATA_SIZE * NUM_MAPS) } sortingBenchmark.addTimerCase("remote rpc fetch") { timer => val reader = setup( - sortDataFile, - sortFileLength, + smallerDataFile, + smallerFileLength, fetchLocal = false, sorter = Some(sorter)) timer.startTiming() val numRead = reader.read().length timer.stopTiming() - assert(numRead == SORT_DATA_SIZE * NUM_MAPS) + assert(numRead == SMALLER_DATA_SIZE * NUM_MAPS) } sortingBenchmark.run() } diff --git a/core/src/test/scala/org/apache/spark/shuffle/sort/SortShuffleWriterBenchmark.scala b/core/src/test/scala/org/apache/spark/shuffle/sort/SortShuffleWriterBenchmark.scala index 56cf65b202df1..69ebed4077647 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/sort/SortShuffleWriterBenchmark.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/sort/SortShuffleWriterBenchmark.scala @@ -142,7 +142,6 @@ object SortShuffleWriterBenchmark extends ShuffleWriterBenchmarkBase { val sorter = Ordering.String addBenchmarkCase(benchmark, "with map side sort") { timer => val shuffleWriter = getWriter(Option.empty, Some(sorter)) - val iterator: DataIterator = DataIterator(inputFile = dataFile, DEFAULT_DATA_STRING_SIZE) Utils.tryWithResource(DataIterator(inputFile = dataFile, DEFAULT_DATA_STRING_SIZE)) { iterator => timer.startTiming() From 567d372dc038a699fba8954572d5ddc89a356a77 Mon Sep 17 00:00:00 2001 From: Yifei Huang Date: Tue, 5 Mar 2019 12:11:33 -0800 Subject: [PATCH 47/68] fix up tests --- .../BlockStoreShuffleReaderBenchmark.scala | 212 +++++++++--------- ...ypassMergeSortShuffleWriterBenchmark.scala | 35 +-- .../sort/ShuffleWriterBenchmarkBase.scala | 5 +- .../sort/SortShuffleWriterBenchmark.scala | 19 +- .../sort/UnsafeShuffleWriterBenchmark.scala | 19 +- 5 files changed, 155 insertions(+), 135 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/shuffle/sort/BlockStoreShuffleReaderBenchmark.scala b/core/src/test/scala/org/apache/spark/shuffle/sort/BlockStoreShuffleReaderBenchmark.scala index 9b0e6085e87ab..85a209286c058 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/sort/BlockStoreShuffleReaderBenchmark.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/sort/BlockStoreShuffleReaderBenchmark.scala @@ -157,11 +157,12 @@ object BlockStoreShuffleReaderBenchmark extends BenchmarkBase { externalBlockManager.stop() } - def setup(dataFile: File, - dataFileLength: Long, - fetchLocal: Boolean, - aggregator: Option[Aggregator[String, String, String]] = None, - sorter: Option[Ordering[String]] = None): BlockStoreShuffleReader[String, String] = { + def setupReader( + dataFile: File, + dataFileLength: Long, + fetchLocal: Boolean, + aggregator: Option[Aggregator[String, String, String]] = None, + sorter: Option[Ordering[String]] = None): BlockStoreShuffleReader[String, String] = { SparkEnv.set(new SparkEnv( "0", null, @@ -272,111 +273,120 @@ object BlockStoreShuffleReaderBenchmark extends BenchmarkBase { // scalastyle:off println } - override def runBenchmarkSuite(mainArgs: Array[String]): Unit = { - tempDir = Utils.createTempDir(null, "shuffle") + def runWithLargeDataset(): Unit = { + val size = TEST_DATA_SIZE val tempDataFile: File = File.createTempFile("test-data", "", tempDir) - val dataFileLength = generateDataOnDisk(TEST_DATA_SIZE, tempDataFile) - initializeServers(tempDataFile, dataFileLength) + val dataFileLength = generateDataOnDisk(size, tempDataFile) + val baseBenchmark = + new Benchmark("no aggregation or sorting", + size, + minNumIters = MIN_NUM_ITERS, + output = output, + outputPerIteration = true) + baseBenchmark.addTimerCase("local fetch") { timer => + val reader = setupReader(tempDataFile, dataFileLength, fetchLocal = true) + timer.startTiming() + val numRead = reader.read().length + timer.stopTiming() + assert(numRead == size * NUM_MAPS) + } + baseBenchmark.addTimerCase("remote rpc fetch") { timer => + val reader = setupReader(tempDataFile, dataFileLength, fetchLocal = false) + timer.startTiming() + val numRead = reader.read().length + timer.stopTiming() + assert(numRead == size * NUM_MAPS) + } + baseBenchmark.run() + tempDataFile.delete() + stopServers() + } - runBenchmark("SortShuffleWriter writer") { - val baseBenchmark = - new Benchmark("no aggregation or sorting", - TEST_DATA_SIZE, - minNumIters = MIN_NUM_ITERS, - output = output, - outputPerIteration = true) - baseBenchmark.addTimerCase("local fetch") { timer => - val reader = setup(tempDataFile, dataFileLength, fetchLocal = true) - timer.startTiming() - val numRead = reader.read().length - timer.stopTiming() - assert(numRead == TEST_DATA_SIZE * NUM_MAPS) - } - baseBenchmark.addTimerCase("remote rpc fetch") { timer => - val reader = setup(tempDataFile, dataFileLength, fetchLocal = false) - timer.startTiming() - val numRead = reader.read().length - timer.stopTiming() - assert(numRead == TEST_DATA_SIZE * NUM_MAPS) - } - baseBenchmark.run() + def runWithSmallerDataset(): Unit = { + val size = SMALLER_DATA_SIZE + val smallerDataFile: File = File.createTempFile("test-data", "", tempDir) + val smallerFileLength = generateDataOnDisk(size, smallerDataFile) + initializeServers(smallerDataFile, smallerFileLength) + 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) + val aggregationBenchmark = + new Benchmark("with aggregation", + size, + minNumIters = MIN_NUM_ITERS, + output = output, + outputPerIteration = true) + aggregationBenchmark.addTimerCase("local fetch") { timer => + val reader = setupReader( + smallerDataFile, + smallerFileLength, + fetchLocal = true, + aggregator = Some(aggregator)) + timer.startTiming() + val numRead = reader.read().length + timer.stopTiming() + assert(numRead > 0) + } + aggregationBenchmark.addTimerCase("remote rpc fetch") { timer => + val reader = setupReader( + smallerDataFile, + smallerFileLength, + fetchLocal = false, + aggregator = Some(aggregator)) + timer.startTiming() + val numRead = reader.read().length + timer.stopTiming() + assert(numRead > 0) + } + aggregationBenchmark.run() - stopServers() - val smallerDataFile: File = File.createTempFile("test-data", "", tempDir) - val smallerFileLength = generateDataOnDisk(SMALLER_DATA_SIZE, smallerDataFile) - initializeServers(smallerDataFile, smallerFileLength) - 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) - val aggregationBenchmark = - new Benchmark("with aggregation", - SMALLER_DATA_SIZE, - minNumIters = MIN_NUM_ITERS, - output = output, - outputPerIteration = true) - aggregationBenchmark.addTimerCase("local fetch") { timer => - val reader = setup( - smallerDataFile, - smallerFileLength, - fetchLocal = true, - aggregator = Some(aggregator)) - timer.startTiming() - val numRead = reader.read().length - timer.stopTiming() - assert(numRead > 0) - } - aggregationBenchmark.addTimerCase("remote rpc fetch") { timer => - val reader = setup( - smallerDataFile, - smallerFileLength, - fetchLocal = false, - aggregator = Some(aggregator)) - timer.startTiming() - val numRead = reader.read().length - timer.stopTiming() - assert(numRead > 0) - } - aggregationBenchmark.run() + val sorter = Ordering.String + val sortingBenchmark = + new Benchmark("with sorting", + size, + minNumIters = MIN_NUM_ITERS, + output = output, + outputPerIteration = true) + sortingBenchmark.addTimerCase("local fetch") { timer => + val reader = setupReader( + smallerDataFile, + smallerFileLength, + fetchLocal = true, + sorter = Some(sorter)) + timer.startTiming() + val numRead = reader.read().length + timer.stopTiming() + assert(numRead == size * NUM_MAPS) + } + sortingBenchmark.addTimerCase("remote rpc fetch") { timer => + val reader = setupReader( + smallerDataFile, + smallerFileLength, + fetchLocal = false, + sorter = Some(sorter)) + timer.startTiming() + val numRead = reader.read().length + timer.stopTiming() + assert(numRead == size * NUM_MAPS) + } + sortingBenchmark.run() + stopServers() + smallerDataFile.delete() + } + override def runBenchmarkSuite(mainArgs: Array[String]): Unit = { + tempDir = Utils.createTempDir(null, "shuffle") - val sorter = Ordering.String - val sortingBenchmark = - new Benchmark("with sorting", - SMALLER_DATA_SIZE, - minNumIters = MIN_NUM_ITERS, - output = output, - outputPerIteration = true) - sortingBenchmark.addTimerCase("local fetch") { timer => - val reader = setup( - smallerDataFile, - smallerFileLength, - fetchLocal = true, - sorter = Some(sorter)) - timer.startTiming() - val numRead = reader.read().length - timer.stopTiming() - assert(numRead == SMALLER_DATA_SIZE * NUM_MAPS) - } - sortingBenchmark.addTimerCase("remote rpc fetch") { timer => - val reader = setup( - smallerDataFile, - smallerFileLength, - fetchLocal = false, - sorter = Some(sorter)) - timer.startTiming() - val numRead = reader.read().length - timer.stopTiming() - assert(numRead == SMALLER_DATA_SIZE * NUM_MAPS) - } - sortingBenchmark.run() + runBenchmark("BlockStoreShuffleReader reader") { + runWithLargeDataset() + runWithSmallerDataset() } - stopServers() FileUtils.deleteDirectory(tempDir) } } diff --git a/core/src/test/scala/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriterBenchmark.scala b/core/src/test/scala/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriterBenchmark.scala index 82b3e3da20d38..081312b0f1d90 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriterBenchmark.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriterBenchmark.scala @@ -19,6 +19,7 @@ package org.apache.spark.shuffle.sort import org.apache.spark.SparkConf import org.apache.spark.benchmark.Benchmark +import org.apache.spark.util.Utils /** * Benchmark to measure performance for aggregate primitives. @@ -32,15 +33,16 @@ import org.apache.spark.benchmark.Benchmark */ object BypassMergeSortShuffleWriterBenchmark extends ShuffleWriterBenchmarkBase { - private var shuffleHandle: BypassMergeSortShuffleHandle[String, String] = + private val shuffleHandle: BypassMergeSortShuffleHandle[String, String] = new BypassMergeSortShuffleHandle[String, String]( shuffleId = 0, numMaps = 1, dependency) private val MIN_NUM_ITERS = 10 - private val DATA_SIZE_SMALL = 10000 - private val DATA_SIZE_LARGE = 10000000 + private val DATA_SIZE_SMALL = 1000 + private val DATA_SIZE_LARGE = + PackedRecordPointer.MAXIMUM_PAGE_SIZE_BYTES/4/DEFAULT_DATA_STRING_SIZE def getWriter(transferTo: Boolean): BypassMergeSortShuffleWriter[String, String] = { val conf = new SparkConf(loadDefaults = false) @@ -61,9 +63,9 @@ object BypassMergeSortShuffleWriterBenchmark extends ShuffleWriterBenchmarkBase def writeBenchmarkWithLargeDataset(): Unit = { val size = DATA_SIZE_LARGE - val data = createDataInMemory(size) + val dataFile = createDataOnDisk(size) val benchmark = new Benchmark( - "BypassMergeSortShuffleWrite (with spill) " + size, + "BypassMergeSortShuffleWrite with spill", size, minNumIters = MIN_NUM_ITERS, output = output) @@ -71,14 +73,21 @@ object BypassMergeSortShuffleWriterBenchmark extends ShuffleWriterBenchmarkBase addBenchmarkCase(benchmark, "without transferTo") { timer => val shuffleWriter = getWriter(false) timer.startTiming() - shuffleWriter.write(data.iterator) - timer.stopTiming() + Utils.tryWithResource(DataIterator(inputFile = dataFile, DEFAULT_DATA_STRING_SIZE)) { + iterator => + timer.startTiming() + shuffleWriter.write(iterator) + timer.stopTiming() + } } addBenchmarkCase(benchmark, "with transferTo") { timer => - val shuffleWriter = getWriter(false) - timer.startTiming() - shuffleWriter.write(data.iterator) - timer.stopTiming() + val shuffleWriter = getWriter(true) + Utils.tryWithResource(DataIterator(inputFile = dataFile, DEFAULT_DATA_STRING_SIZE)) { + iterator => + timer.startTiming() + shuffleWriter.write(iterator) + timer.stopTiming() + } } benchmark.run() } @@ -86,11 +95,11 @@ object BypassMergeSortShuffleWriterBenchmark extends ShuffleWriterBenchmarkBase def writeBenchmarkWithSmallDataset(): Unit = { val size = DATA_SIZE_SMALL val data = createDataInMemory(size) - val benchmark = new Benchmark("BypassMergeSortShuffleWrite (in memory buffer) " + size, + val benchmark = new Benchmark("BypassMergeSortShuffleWrite without spill", size, minNumIters = MIN_NUM_ITERS, output = output) - addBenchmarkCase(benchmark, "small dataset without spills on disk") { timer => + addBenchmarkCase(benchmark, "small dataset without disk spill") { timer => val shuffleWriter = getWriter(false) timer.startTiming() shuffleWriter.write(data.iterator) diff --git a/core/src/test/scala/org/apache/spark/shuffle/sort/ShuffleWriterBenchmarkBase.scala b/core/src/test/scala/org/apache/spark/shuffle/sort/ShuffleWriterBenchmarkBase.scala index 6aa679deb9bdf..a9d7e0458c5c5 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/sort/ShuffleWriterBenchmarkBase.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/sort/ShuffleWriterBenchmarkBase.scala @@ -41,7 +41,7 @@ import org.apache.spark.util.Utils abstract class ShuffleWriterBenchmarkBase extends BenchmarkBase { - private val DEFAULT_DATA_STRING_SIZE = 5 + 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: @@ -56,6 +56,7 @@ abstract class ShuffleWriterBenchmarkBase extends BenchmarkBase { 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] @@ -94,7 +95,6 @@ abstract class ShuffleWriterBenchmarkBase extends BenchmarkBase { } protected var tempDir: File = _ - protected var shuffleMetrics: TaskMetrics = _ protected var blockManager: BlockManager = _ protected var blockResolver: IndexShuffleBlockResolver = _ @@ -106,7 +106,6 @@ abstract class ShuffleWriterBenchmarkBase extends BenchmarkBase { when(dependency.partitioner).thenReturn(partitioner) when(dependency.serializer).thenReturn(serializer) when(dependency.shuffleId).thenReturn(0) - shuffleMetrics = new TaskMetrics when(taskContext.taskMetrics()).thenReturn(shuffleMetrics) when(rpcEnv.setupEndpoint(any[String], any[RpcEndpoint])).thenReturn(rpcEndpointRef) diff --git a/core/src/test/scala/org/apache/spark/shuffle/sort/SortShuffleWriterBenchmark.scala b/core/src/test/scala/org/apache/spark/shuffle/sort/SortShuffleWriterBenchmark.scala index 69ebed4077647..4862f39425c90 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/sort/SortShuffleWriterBenchmark.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/sort/SortShuffleWriterBenchmark.scala @@ -36,19 +36,20 @@ import org.apache.spark.util.Utils */ object SortShuffleWriterBenchmark extends ShuffleWriterBenchmarkBase { - private var shuffleHandle: BaseShuffleHandle[String, String, String] = + private val shuffleHandle: BaseShuffleHandle[String, String, String] = new BaseShuffleHandle( shuffleId = 0, numMaps = 1, dependency = dependency) - private val DEFAULT_DATA_STRING_SIZE = 5 - private val MIN_NUM_ITERS = 5 + 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 - val defaultSparkEnv = SparkEnv.get SparkEnv.set(new SparkEnv( "0", null, @@ -86,24 +87,24 @@ object SortShuffleWriterBenchmark extends ShuffleWriterBenchmarkBase { } def writeBenchmarkWithSmallDataset(): Unit = { - val size = 1000 - val benchmark = new Benchmark("SortShuffleWriter with spills", + 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 writer = getWriter(Option.empty, Option.empty) - val array = createDataInMemory(1000) + val array = createDataInMemory(size) timer.startTiming() writer.write(array.iterator) timer.stopTiming() - assert(tempFilesCreated.length == 0) + assert(tempFilesCreated.isEmpty) } benchmark.run() } def writeBenchmarkWithSpill(): Unit = { - val size = PackedRecordPointer.MAXIMUM_PAGE_SIZE_BYTES/4/DEFAULT_DATA_STRING_SIZE + val size = DATA_SIZE_LARGE val dataFile = createDataOnDisk(size) val benchmark = new Benchmark("SortShuffleWriter with spills", diff --git a/core/src/test/scala/org/apache/spark/shuffle/sort/UnsafeShuffleWriterBenchmark.scala b/core/src/test/scala/org/apache/spark/shuffle/sort/UnsafeShuffleWriterBenchmark.scala index 76d6cd258eec5..22b18e0e01fbd 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/sort/UnsafeShuffleWriterBenchmark.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/sort/UnsafeShuffleWriterBenchmark.scala @@ -35,8 +35,10 @@ object UnsafeShuffleWriterBenchmark extends ShuffleWriterBenchmarkBase { private val shuffleHandle: SerializedShuffleHandle[String, String] = new SerializedShuffleHandle[String, String](0, 0, this.dependency) - private val DEFAULT_DATA_STRING_SIZE = 5 - private val MIN_NUM_ITERS = 5 + private val MIN_NUM_ITERS = 10 + private val DATA_SIZE_SMALL = 1000 + private val DATA_SIZE_LARGE = + PackedRecordPointer.MAXIMUM_PAGE_SIZE_BYTES/2/DEFAULT_DATA_STRING_SIZE def getWriter(transferTo: Boolean): UnsafeShuffleWriter[String, String] = { val conf = new SparkConf(loadDefaults = false) @@ -55,14 +57,14 @@ object UnsafeShuffleWriterBenchmark extends ShuffleWriterBenchmarkBase { } def writeBenchmarkWithSmallDataset(): Unit = { - val size = 1000 - val benchmark = new Benchmark("UnsafeShuffleWriter with spills", + val size = DATA_SIZE_SMALL + val array = createDataInMemory(size) + val benchmark = new Benchmark("UnsafeShuffleWriter without spills", size, minNumIters = MIN_NUM_ITERS, output = output) addBenchmarkCase(benchmark, "small dataset without spills") { timer => val writer = getWriter(false) - val array = createDataInMemory(1000) timer.startTiming() writer.write(array.iterator) timer.stopTiming() @@ -72,12 +74,11 @@ object UnsafeShuffleWriterBenchmark extends ShuffleWriterBenchmarkBase { } def writeBenchmarkWithSpill(): Unit = { - val size = PackedRecordPointer.MAXIMUM_PAGE_SIZE_BYTES/2/DEFAULT_DATA_STRING_SIZE - val minNumIters = 5 + val size = DATA_SIZE_LARGE val tempDataFile = createDataOnDisk(size) val benchmark = new Benchmark("UnsafeShuffleWriter with spills", size, - minNumIters = minNumIters, + minNumIters = MIN_NUM_ITERS, output = output, outputPerIteration = true) addBenchmarkCase(benchmark, "without transferTo") { timer => @@ -91,7 +92,7 @@ object UnsafeShuffleWriterBenchmark extends ShuffleWriterBenchmarkBase { assert(tempFilesCreated.length == 7) } addBenchmarkCase(benchmark, "with transferTo") { timer => - val shuffleWriter = getWriter(false) + val shuffleWriter = getWriter(true) Utils.tryWithResource(DataIterator(inputFile = tempDataFile, DEFAULT_DATA_STRING_SIZE)) { iterator => timer.startTiming() From 47c1938b1b65458aa55545fc8a6c95df75c18055 Mon Sep 17 00:00:00 2001 From: Yifei Huang Date: Tue, 5 Mar 2019 12:23:00 -0800 Subject: [PATCH 48/68] only copy benchmarks we care about --- dev/run-spark-25299-benchmarks.sh | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/dev/run-spark-25299-benchmarks.sh b/dev/run-spark-25299-benchmarks.sh index b63934ec19656..a131dd1257aad 100755 --- a/dev/run-spark-25299-benchmarks.sh +++ b/dev/run-spark-25299-benchmarks.sh @@ -58,7 +58,10 @@ SPARK_GENERATE_BENCHMARK_FILES=1 ./build/sbt "sql/test:runMain org.apache.spark. SPARK_DIR=`pwd` mkdir -p /tmp/artifacts -cp $SPARK_DIR/sql/core/benchmarks/* /tmp/artifacts/ +cp $SPARK_DIR/sql/core/benchmarks/BlockStoreShuffleReaderBenchmark-results.txt /tmp/artifacts/ +cp $SPARK_DIR/sql/core/benchmarks/BypassMergeSortShuffleWriterBenchmark-results.txt /tmp/artifacts/ +cp $SPARK_DIR/sql/core/benchmarks/SortShuffleWriterBenchmark-results.txt /tmp/artifacts/ +cp $SPARK_DIR/sql/core/benchmarks/UnsafeShuffleWriterBenchmark-results.txt /tmp/artifacts/ if [ "$UPLOAD" = false ]; then exit 0 From e79ac2872d70c0f70a25551c84e1290bb25052c1 Mon Sep 17 00:00:00 2001 From: Yifei Huang Date: Tue, 5 Mar 2019 12:25:26 -0800 Subject: [PATCH 49/68] increase size for reader again --- .../spark/shuffle/sort/BlockStoreShuffleReaderBenchmark.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/test/scala/org/apache/spark/shuffle/sort/BlockStoreShuffleReaderBenchmark.scala b/core/src/test/scala/org/apache/spark/shuffle/sort/BlockStoreShuffleReaderBenchmark.scala index 85a209286c058..70548af632495 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/sort/BlockStoreShuffleReaderBenchmark.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/sort/BlockStoreShuffleReaderBenchmark.scala @@ -72,7 +72,7 @@ object BlockStoreShuffleReaderBenchmark extends BenchmarkBase { private val NUM_MAPS: Int = 5 private val DEFAULT_DATA_STRING_SIZE = 5 private val TEST_DATA_SIZE: Int = 10000000 - private val SMALLER_DATA_SIZE: Int = 1000000 + private val SMALLER_DATA_SIZE: Int = 2000000 private val MIN_NUM_ITERS: Int = 10 private val executorId: String = "0" From c3858df80cf556f5d4e5a3200676ac840eed4787 Mon Sep 17 00:00:00 2001 From: Yifei Huang Date: Tue, 5 Mar 2019 12:31:10 -0800 Subject: [PATCH 50/68] delete two writers and reader for PR --- .../BlockStoreShuffleReaderBenchmark.scala | 392 ------------------ ...ypassMergeSortShuffleWriterBenchmark.scala | 117 ------ .../sort/UnsafeShuffleWriterBenchmark.scala | 114 ----- dev/run-spark-25299-benchmarks.sh | 6 - 4 files changed, 629 deletions(-) delete mode 100644 core/src/test/scala/org/apache/spark/shuffle/sort/BlockStoreShuffleReaderBenchmark.scala delete mode 100644 core/src/test/scala/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriterBenchmark.scala delete mode 100644 core/src/test/scala/org/apache/spark/shuffle/sort/UnsafeShuffleWriterBenchmark.scala diff --git a/core/src/test/scala/org/apache/spark/shuffle/sort/BlockStoreShuffleReaderBenchmark.scala b/core/src/test/scala/org/apache/spark/shuffle/sort/BlockStoreShuffleReaderBenchmark.scala deleted file mode 100644 index 70548af632495..0000000000000 --- a/core/src/test/scala/org/apache/spark/shuffle/sort/BlockStoreShuffleReaderBenchmark.scala +++ /dev/null @@ -1,392 +0,0 @@ -/* - * 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.{File, FileOutputStream} - -import com.google.common.io.CountingOutputStream -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.util.Random - -import org.apache.spark.{Aggregator, MapOutputTracker, ShuffleDependency, SparkConf, SparkEnv, TaskContext} -import org.apache.spark.benchmark.{Benchmark, BenchmarkBase} -import org.apache.spark.executor.TaskMetrics -import org.apache.spark.memory.{TaskMemoryManager, TestMemoryManager} -import org.apache.spark.metrics.source.Source -import org.apache.spark.network.BlockTransferService -import org.apache.spark.network.buffer.{FileSegmentManagedBuffer, ManagedBuffer} -import org.apache.spark.network.netty.{NettyBlockTransferService, SparkTransportConf} -import org.apache.spark.network.util.TransportConf -import org.apache.spark.rpc.{RpcEndpoint, RpcEndpointRef, RpcEnv} -import org.apache.spark.serializer.{KryoSerializer, SerializerManager} -import org.apache.spark.shuffle.{BaseShuffleHandle, BlockStoreShuffleReader, FetchFailedException} -import org.apache.spark.storage.{BlockId, BlockManager, BlockManagerId, BlockManagerMaster, ShuffleBlockId} -import org.apache.spark.util.{AccumulatorV2, TaskCompletionListener, TaskFailureListener, Utils} - -/** - * Benchmark to measure performance for aggregate primitives. - * {{{ - * To run this benchmark: - * 1. without sbt: bin/spark-submit --class - * 2. build/sbt "sql/test:runMain " - * 3. generate result: SPARK_GENERATE_BENCHMARK_FILES=1 build/sbt "sql/test:runMain " - * Results will be written to "benchmarks/-results.txt". - * }}} - */ -object BlockStoreShuffleReaderBenchmark extends BenchmarkBase { - - // this is only used to retrieve the aggregator/sorters/serializers, - // so it shouldn't affect the performance significantly - @Mock(answer = RETURNS_SMART_NULLS) private var dependency: - ShuffleDependency[String, String, String] = _ - // only used to retrieve info about the maps at the beginning, doesn't affect perf - @Mock(answer = RETURNS_SMART_NULLS) private var mapOutputTracker: MapOutputTracker = _ - // this is only used when initializing the BlockManager, so doesn't affect perf - @Mock(answer = RETURNS_SMART_NULLS) private var blockManagerMaster: BlockManagerMaster = _ - // this is only used when initiating the BlockManager, for comms between master and executor - @Mock(answer = RETURNS_SMART_NULLS) private var rpcEnv: RpcEnv = _ - @Mock(answer = RETURNS_SMART_NULLS) protected var rpcEndpointRef: RpcEndpointRef = _ - - private var tempDir: File = _ - - private val SHUFFLE_ID: Int = 0 - private val REDUCE_ID: Int = 0 - private val NUM_MAPS: Int = 5 - private val DEFAULT_DATA_STRING_SIZE = 5 - private val TEST_DATA_SIZE: Int = 10000000 - private val SMALLER_DATA_SIZE: Int = 2000000 - private val MIN_NUM_ITERS: Int = 10 - - private val executorId: String = "0" - private val localPort: Int = 17000 - private val remotePort: Int = 17002 - - private val defaultConf: SparkConf = new SparkConf() - .set("spark.shuffle.compress", "false") - .set("spark.shuffle.spill.compress", "false") - .set("spark.authenticate", "false") - .set("spark.app.id", "test-app") - private val serializer: KryoSerializer = new KryoSerializer(defaultConf) - private val serializerManager: SerializerManager = new SerializerManager(serializer, defaultConf) - private val execBlockManagerId: BlockManagerId = - BlockManagerId(executorId, "localhost", localPort) - private val remoteBlockManagerId: BlockManagerId = - BlockManagerId(executorId, "localhost", remotePort) - private val transportConf: TransportConf = - SparkTransportConf.fromSparkConf(defaultConf, "shuffle") - private val securityManager: org.apache.spark.SecurityManager = - new org.apache.spark.SecurityManager(defaultConf) - protected val memoryManager: TestMemoryManager = new TestMemoryManager(defaultConf) - - class TestBlockManager(transferService: BlockTransferService, - blockManagerMaster: BlockManagerMaster, - dataFile: File, - fileLength: Long) extends BlockManager( - executorId, - rpcEnv, - blockManagerMaster, - serializerManager, - defaultConf, - memoryManager, - null, - null, - transferService, - null, - 1) { - blockManagerId = execBlockManagerId - - override def getBlockData(blockId: BlockId): ManagedBuffer = { - new FileSegmentManagedBuffer( - transportConf, - dataFile, - 0, - fileLength - ) - } - } - - private var blockManager : BlockManager = _ - private var externalBlockManager: BlockManager = _ - - def getTestBlockManager(port: Int, dataFile: File, dataFileLength: Long): TestBlockManager = { - val shuffleClient = new NettyBlockTransferService( - defaultConf, - securityManager, - "localhost", - "localhost", - port, - 1 - ) - new TestBlockManager(shuffleClient, - blockManagerMaster, - dataFile, - dataFileLength) - } - - def initializeServers(dataFile: File, dataFileLength: Long): Unit = { - MockitoAnnotations.initMocks(this) - when(blockManagerMaster.registerBlockManager( - any[BlockManagerId], any[Long], any[Long], any[RpcEndpointRef])).thenReturn(null) - when(rpcEnv.setupEndpoint(any[String], any[RpcEndpoint])).thenReturn(rpcEndpointRef) - blockManager = getTestBlockManager(localPort, dataFile, dataFileLength) - blockManager.initialize(defaultConf.getAppId) - externalBlockManager = getTestBlockManager(remotePort, dataFile, dataFileLength) - externalBlockManager.initialize(defaultConf.getAppId) - } - - def stopServers(): Unit = { - blockManager.stop() - externalBlockManager.stop() - } - - def setupReader( - dataFile: File, - dataFileLength: Long, - fetchLocal: Boolean, - aggregator: Option[Aggregator[String, String, String]] = None, - sorter: Option[Ordering[String]] = None): BlockStoreShuffleReader[String, String] = { - SparkEnv.set(new SparkEnv( - "0", - null, - serializer, - null, - serializerManager, - mapOutputTracker, - null, - null, - blockManager, - null, - null, - null, - null, - defaultConf - )) - - val shuffleHandle = new BaseShuffleHandle( - shuffleId = SHUFFLE_ID, - numMaps = NUM_MAPS, - dependency = dependency) - - // We cannot mock the TaskContext because it taskMetrics() gets called at every next() - // call on the reader, and Mockito will try to log all calls to taskMetrics(), thus OOM-ing - // the test - val taskContext = new TaskContext { - private val metrics: TaskMetrics = new TaskMetrics - private val testMemManager = new TestMemoryManager(defaultConf) - private val taskMemManager = new TaskMemoryManager(testMemManager, 0) - testMemManager.limit(PackedRecordPointer.MAXIMUM_PAGE_SIZE_BYTES) - override def isCompleted(): Boolean = false - override def isInterrupted(): Boolean = false - override def addTaskCompletionListener(listener: TaskCompletionListener): - TaskContext = { null } - override def addTaskFailureListener(listener: TaskFailureListener): TaskContext = { null } - override def stageId(): Int = 0 - override def stageAttemptNumber(): Int = 0 - override def partitionId(): Int = 0 - override def attemptNumber(): Int = 0 - override def taskAttemptId(): Long = 0 - override def getLocalProperty(key: String): String = "" - override def taskMetrics(): TaskMetrics = metrics - override def getMetricsSources(sourceName: String): Seq[Source] = Seq.empty - override private[spark] def killTaskIfInterrupted(): Unit = {} - override private[spark] def getKillReason() = None - override private[spark] def taskMemoryManager() = taskMemManager - override private[spark] def registerAccumulator(a: AccumulatorV2[_, _]): Unit = {} - override private[spark] def setFetchFailed(fetchFailed: FetchFailedException): Unit = {} - override private[spark] def markInterrupted(reason: String): Unit = {} - override private[spark] def markTaskFailed(error: Throwable): Unit = {} - override private[spark] def markTaskCompleted(error: Option[Throwable]): Unit = {} - override private[spark] def fetchFailed = None - override private[spark] def getLocalProperties = { null } - } - TaskContext.setTaskContext(taskContext) - - var dataBlockId: BlockManagerId = execBlockManagerId - if (!fetchLocal) { - dataBlockId = remoteBlockManagerId - } - - when(mapOutputTracker.getMapSizesByExecutorId(SHUFFLE_ID, REDUCE_ID, REDUCE_ID + 1)) - .thenReturn { - val shuffleBlockIdsAndSizes = (0 until NUM_MAPS).map { mapId => - val shuffleBlockId = ShuffleBlockId(SHUFFLE_ID, mapId, REDUCE_ID) - (shuffleBlockId, dataFileLength) - } - Seq((dataBlockId, shuffleBlockIdsAndSizes)).toIterator - } - - when(dependency.serializer).thenReturn(serializer) - when(dependency.aggregator).thenReturn(aggregator) - when(dependency.keyOrdering).thenReturn(sorter) - - new BlockStoreShuffleReader[String, String]( - shuffleHandle, - 0, - 1, - taskContext, - taskContext.taskMetrics().createTempShuffleReadMetrics(), - serializerManager, - blockManager, - mapOutputTracker - ) - } - - def generateDataOnDisk(size: Int, file: File): Long = { - // scalastyle:off println - println("Generating test data with num records: " + size) - val random = new Random(123) - val dataOutput = new FileOutputStream(file) - val coutingOutput = new CountingOutputStream(dataOutput) - val serializedOutput = serializer.newInstance().serializeStream(coutingOutput) - try { - (1 to size).foreach { i => { - if (i % 1000000 == 0) { - println("Wrote " + i + " test data points") - } - val x = random.alphanumeric.take(DEFAULT_DATA_STRING_SIZE).mkString - serializedOutput.writeKey(x) - serializedOutput.writeValue(x) - }} - } - finally { - serializedOutput.close() - } - coutingOutput.getCount - // scalastyle:off println - } - - def runWithLargeDataset(): Unit = { - val size = TEST_DATA_SIZE - val tempDataFile: File = File.createTempFile("test-data", "", tempDir) - val dataFileLength = generateDataOnDisk(size, tempDataFile) - val baseBenchmark = - new Benchmark("no aggregation or sorting", - size, - minNumIters = MIN_NUM_ITERS, - output = output, - outputPerIteration = true) - baseBenchmark.addTimerCase("local fetch") { timer => - val reader = setupReader(tempDataFile, dataFileLength, fetchLocal = true) - timer.startTiming() - val numRead = reader.read().length - timer.stopTiming() - assert(numRead == size * NUM_MAPS) - } - baseBenchmark.addTimerCase("remote rpc fetch") { timer => - val reader = setupReader(tempDataFile, dataFileLength, fetchLocal = false) - timer.startTiming() - val numRead = reader.read().length - timer.stopTiming() - assert(numRead == size * NUM_MAPS) - } - baseBenchmark.run() - tempDataFile.delete() - stopServers() - } - - def runWithSmallerDataset(): Unit = { - val size = SMALLER_DATA_SIZE - val smallerDataFile: File = File.createTempFile("test-data", "", tempDir) - val smallerFileLength = generateDataOnDisk(size, smallerDataFile) - initializeServers(smallerDataFile, smallerFileLength) - - 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) - val aggregationBenchmark = - new Benchmark("with aggregation", - size, - minNumIters = MIN_NUM_ITERS, - output = output, - outputPerIteration = true) - aggregationBenchmark.addTimerCase("local fetch") { timer => - val reader = setupReader( - smallerDataFile, - smallerFileLength, - fetchLocal = true, - aggregator = Some(aggregator)) - timer.startTiming() - val numRead = reader.read().length - timer.stopTiming() - assert(numRead > 0) - } - aggregationBenchmark.addTimerCase("remote rpc fetch") { timer => - val reader = setupReader( - smallerDataFile, - smallerFileLength, - fetchLocal = false, - aggregator = Some(aggregator)) - timer.startTiming() - val numRead = reader.read().length - timer.stopTiming() - assert(numRead > 0) - } - aggregationBenchmark.run() - - - val sorter = Ordering.String - val sortingBenchmark = - new Benchmark("with sorting", - size, - minNumIters = MIN_NUM_ITERS, - output = output, - outputPerIteration = true) - sortingBenchmark.addTimerCase("local fetch") { timer => - val reader = setupReader( - smallerDataFile, - smallerFileLength, - fetchLocal = true, - sorter = Some(sorter)) - timer.startTiming() - val numRead = reader.read().length - timer.stopTiming() - assert(numRead == size * NUM_MAPS) - } - sortingBenchmark.addTimerCase("remote rpc fetch") { timer => - val reader = setupReader( - smallerDataFile, - smallerFileLength, - fetchLocal = false, - sorter = Some(sorter)) - timer.startTiming() - val numRead = reader.read().length - timer.stopTiming() - assert(numRead == size * NUM_MAPS) - } - sortingBenchmark.run() - stopServers() - smallerDataFile.delete() - } - - override def runBenchmarkSuite(mainArgs: Array[String]): Unit = { - tempDir = Utils.createTempDir(null, "shuffle") - - runBenchmark("BlockStoreShuffleReader reader") { - runWithLargeDataset() - runWithSmallerDataset() - } - - FileUtils.deleteDirectory(tempDir) - } -} diff --git a/core/src/test/scala/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriterBenchmark.scala b/core/src/test/scala/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriterBenchmark.scala deleted file mode 100644 index 081312b0f1d90..0000000000000 --- a/core/src/test/scala/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriterBenchmark.scala +++ /dev/null @@ -1,117 +0,0 @@ -/* - * 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.apache.spark.SparkConf -import org.apache.spark.benchmark.Benchmark -import org.apache.spark.util.Utils - -/** - * Benchmark to measure performance for aggregate primitives. - * {{{ - * To run this benchmark: - * 1. without sbt: bin/spark-submit --class - * 2. build/sbt "sql/test:runMain " - * 3. generate result: SPARK_GENERATE_BENCHMARK_FILES=1 build/sbt "sql/test:runMain " - * Results will be written to "benchmarks/-results.txt". - * }}} - */ -object BypassMergeSortShuffleWriterBenchmark extends ShuffleWriterBenchmarkBase { - - private val shuffleHandle: BypassMergeSortShuffleHandle[String, String] = - new BypassMergeSortShuffleHandle[String, String]( - shuffleId = 0, - numMaps = 1, - 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(transferTo: Boolean): BypassMergeSortShuffleWriter[String, String] = { - val conf = new SparkConf(loadDefaults = false) - conf.set("spark.file.transferTo", String.valueOf(transferTo)) - conf.set("spark.shuffle.file.buffer", "32k") - - val shuffleWriter = new BypassMergeSortShuffleWriter[String, String]( - blockManager, - blockResolver, - shuffleHandle, - 0, - conf, - taskContext.taskMetrics().shuffleWriteMetrics - ) - - shuffleWriter - } - - def writeBenchmarkWithLargeDataset(): Unit = { - val size = DATA_SIZE_LARGE - val dataFile = createDataOnDisk(size) - val benchmark = new Benchmark( - "BypassMergeSortShuffleWrite with spill", - size, - minNumIters = MIN_NUM_ITERS, - output = output) - - addBenchmarkCase(benchmark, "without transferTo") { timer => - val shuffleWriter = getWriter(false) - timer.startTiming() - Utils.tryWithResource(DataIterator(inputFile = dataFile, DEFAULT_DATA_STRING_SIZE)) { - iterator => - timer.startTiming() - shuffleWriter.write(iterator) - timer.stopTiming() - } - } - addBenchmarkCase(benchmark, "with transferTo") { timer => - val shuffleWriter = getWriter(true) - Utils.tryWithResource(DataIterator(inputFile = dataFile, DEFAULT_DATA_STRING_SIZE)) { - iterator => - timer.startTiming() - shuffleWriter.write(iterator) - timer.stopTiming() - } - } - benchmark.run() - } - - def writeBenchmarkWithSmallDataset(): Unit = { - val size = DATA_SIZE_SMALL - val data = createDataInMemory(size) - val benchmark = new Benchmark("BypassMergeSortShuffleWrite without spill", - size, - minNumIters = MIN_NUM_ITERS, - output = output) - addBenchmarkCase(benchmark, "small dataset without disk spill") { timer => - val shuffleWriter = getWriter(false) - timer.startTiming() - shuffleWriter.write(data.iterator) - timer.stopTiming() - } - benchmark.run() - } - - override def runBenchmarkSuite(mainArgs: Array[String]): Unit = { - runBenchmark("BypassMergeSortShuffleWriter write") { - writeBenchmarkWithSmallDataset() - writeBenchmarkWithLargeDataset() - } - } -} diff --git a/core/src/test/scala/org/apache/spark/shuffle/sort/UnsafeShuffleWriterBenchmark.scala b/core/src/test/scala/org/apache/spark/shuffle/sort/UnsafeShuffleWriterBenchmark.scala deleted file mode 100644 index 22b18e0e01fbd..0000000000000 --- a/core/src/test/scala/org/apache/spark/shuffle/sort/UnsafeShuffleWriterBenchmark.scala +++ /dev/null @@ -1,114 +0,0 @@ -/* - * 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.apache.spark.SparkConf -import org.apache.spark.benchmark.Benchmark -import org.apache.spark.util.Utils - -/** - * Benchmark to measure performance for aggregate primitives. - * {{{ - * To run this benchmark: - * 1. without sbt: bin/spark-submit --class - * 2. build/sbt "sql/test:runMain " - * 3. generate result: SPARK_GENERATE_BENCHMARK_FILES=1 build/sbt "sql/test:runMain " - * Results will be written to "benchmarks/-results.txt". - * }}} - */ -object UnsafeShuffleWriterBenchmark extends ShuffleWriterBenchmarkBase { - - private val shuffleHandle: SerializedShuffleHandle[String, String] = - new SerializedShuffleHandle[String, String](0, 0, this.dependency) - - private val MIN_NUM_ITERS = 10 - private val DATA_SIZE_SMALL = 1000 - private val DATA_SIZE_LARGE = - PackedRecordPointer.MAXIMUM_PAGE_SIZE_BYTES/2/DEFAULT_DATA_STRING_SIZE - - def getWriter(transferTo: Boolean): UnsafeShuffleWriter[String, String] = { - val conf = new SparkConf(loadDefaults = false) - conf.set("spark.file.transferTo", String.valueOf(transferTo)) - - new UnsafeShuffleWriter[String, String]( - blockManager, - blockResolver, - taskMemoryManager, - shuffleHandle, - 0, - taskContext, - conf, - taskContext.taskMetrics().shuffleWriteMetrics - ) - } - - def writeBenchmarkWithSmallDataset(): Unit = { - val size = DATA_SIZE_SMALL - val array = createDataInMemory(size) - val benchmark = new Benchmark("UnsafeShuffleWriter without spills", - size, - minNumIters = MIN_NUM_ITERS, - output = output) - addBenchmarkCase(benchmark, "small dataset without spills") { timer => - val writer = getWriter(false) - timer.startTiming() - writer.write(array.iterator) - timer.stopTiming() - assert(tempFilesCreated.length == 1) // The single temp file is for the temp index file - } - benchmark.run() - } - - def writeBenchmarkWithSpill(): Unit = { - val size = DATA_SIZE_LARGE - val tempDataFile = createDataOnDisk(size) - val benchmark = new Benchmark("UnsafeShuffleWriter with spills", - size, - minNumIters = MIN_NUM_ITERS, - output = output, - outputPerIteration = true) - addBenchmarkCase(benchmark, "without transferTo") { timer => - val shuffleWriter = getWriter(false) - Utils.tryWithResource(DataIterator(inputFile = tempDataFile, DEFAULT_DATA_STRING_SIZE)) { - iterator => - timer.startTiming() - shuffleWriter.write(iterator) - timer.stopTiming() - } - assert(tempFilesCreated.length == 7) - } - addBenchmarkCase(benchmark, "with transferTo") { timer => - val shuffleWriter = getWriter(true) - Utils.tryWithResource(DataIterator(inputFile = tempDataFile, DEFAULT_DATA_STRING_SIZE)) { - iterator => - timer.startTiming() - shuffleWriter.write(iterator) - timer.stopTiming() - } - assert(tempFilesCreated.length == 7) - } - benchmark.run() - tempDataFile.delete() - } - - override def runBenchmarkSuite(mainArgs: Array[String]): Unit = { - runBenchmark("UnsafeShuffleWriter write") { - writeBenchmarkWithSmallDataset() - writeBenchmarkWithSpill() - } - } -} diff --git a/dev/run-spark-25299-benchmarks.sh b/dev/run-spark-25299-benchmarks.sh index a131dd1257aad..4f055f1f06271 100755 --- a/dev/run-spark-25299-benchmarks.sh +++ b/dev/run-spark-25299-benchmarks.sh @@ -50,18 +50,12 @@ done echo "Running SPARK-25299 benchmarks" -SPARK_GENERATE_BENCHMARK_FILES=1 ./build/sbt "sql/test:runMain org.apache.spark.shuffle.sort.BypassMergeSortShuffleWriterBenchmark" -SPARK_GENERATE_BENCHMARK_FILES=1 ./build/sbt "sql/test:runMain org.apache.spark.shuffle.sort.UnsafeShuffleWriterBenchmark" SPARK_GENERATE_BENCHMARK_FILES=1 ./build/sbt "sql/test:runMain org.apache.spark.shuffle.sort.SortShuffleWriterBenchmark" -SPARK_GENERATE_BENCHMARK_FILES=1 ./build/sbt "sql/test:runMain org.apache.spark.shuffle.sort.BlockStoreShuffleReaderBenchmark" SPARK_DIR=`pwd` mkdir -p /tmp/artifacts -cp $SPARK_DIR/sql/core/benchmarks/BlockStoreShuffleReaderBenchmark-results.txt /tmp/artifacts/ -cp $SPARK_DIR/sql/core/benchmarks/BypassMergeSortShuffleWriterBenchmark-results.txt /tmp/artifacts/ cp $SPARK_DIR/sql/core/benchmarks/SortShuffleWriterBenchmark-results.txt /tmp/artifacts/ -cp $SPARK_DIR/sql/core/benchmarks/UnsafeShuffleWriterBenchmark-results.txt /tmp/artifacts/ if [ "$UPLOAD" = false ]; then exit 0 From 9d46fae9a36c6229a888bb647b1b63f51d83b407 Mon Sep 17 00:00:00 2001 From: Yifei Huang Date: Tue, 5 Mar 2019 12:42:28 -0800 Subject: [PATCH 51/68] SPARK-25299: Add shuffle reader benchmarks (#506) --- .../BlockStoreShuffleReaderBenchmark.scala | 392 ++++++++++++++++++ dev/run-spark-25299-benchmarks.sh | 2 + 2 files changed, 394 insertions(+) create mode 100644 core/src/test/scala/org/apache/spark/shuffle/sort/BlockStoreShuffleReaderBenchmark.scala diff --git a/core/src/test/scala/org/apache/spark/shuffle/sort/BlockStoreShuffleReaderBenchmark.scala b/core/src/test/scala/org/apache/spark/shuffle/sort/BlockStoreShuffleReaderBenchmark.scala new file mode 100644 index 0000000000000..70548af632495 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/shuffle/sort/BlockStoreShuffleReaderBenchmark.scala @@ -0,0 +1,392 @@ +/* + * 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.{File, FileOutputStream} + +import com.google.common.io.CountingOutputStream +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.util.Random + +import org.apache.spark.{Aggregator, MapOutputTracker, ShuffleDependency, SparkConf, SparkEnv, TaskContext} +import org.apache.spark.benchmark.{Benchmark, BenchmarkBase} +import org.apache.spark.executor.TaskMetrics +import org.apache.spark.memory.{TaskMemoryManager, TestMemoryManager} +import org.apache.spark.metrics.source.Source +import org.apache.spark.network.BlockTransferService +import org.apache.spark.network.buffer.{FileSegmentManagedBuffer, ManagedBuffer} +import org.apache.spark.network.netty.{NettyBlockTransferService, SparkTransportConf} +import org.apache.spark.network.util.TransportConf +import org.apache.spark.rpc.{RpcEndpoint, RpcEndpointRef, RpcEnv} +import org.apache.spark.serializer.{KryoSerializer, SerializerManager} +import org.apache.spark.shuffle.{BaseShuffleHandle, BlockStoreShuffleReader, FetchFailedException} +import org.apache.spark.storage.{BlockId, BlockManager, BlockManagerId, BlockManagerMaster, ShuffleBlockId} +import org.apache.spark.util.{AccumulatorV2, TaskCompletionListener, TaskFailureListener, Utils} + +/** + * Benchmark to measure performance for aggregate primitives. + * {{{ + * To run this benchmark: + * 1. without sbt: bin/spark-submit --class + * 2. build/sbt "sql/test:runMain " + * 3. generate result: SPARK_GENERATE_BENCHMARK_FILES=1 build/sbt "sql/test:runMain " + * Results will be written to "benchmarks/-results.txt". + * }}} + */ +object BlockStoreShuffleReaderBenchmark extends BenchmarkBase { + + // this is only used to retrieve the aggregator/sorters/serializers, + // so it shouldn't affect the performance significantly + @Mock(answer = RETURNS_SMART_NULLS) private var dependency: + ShuffleDependency[String, String, String] = _ + // only used to retrieve info about the maps at the beginning, doesn't affect perf + @Mock(answer = RETURNS_SMART_NULLS) private var mapOutputTracker: MapOutputTracker = _ + // this is only used when initializing the BlockManager, so doesn't affect perf + @Mock(answer = RETURNS_SMART_NULLS) private var blockManagerMaster: BlockManagerMaster = _ + // this is only used when initiating the BlockManager, for comms between master and executor + @Mock(answer = RETURNS_SMART_NULLS) private var rpcEnv: RpcEnv = _ + @Mock(answer = RETURNS_SMART_NULLS) protected var rpcEndpointRef: RpcEndpointRef = _ + + private var tempDir: File = _ + + private val SHUFFLE_ID: Int = 0 + private val REDUCE_ID: Int = 0 + private val NUM_MAPS: Int = 5 + private val DEFAULT_DATA_STRING_SIZE = 5 + private val TEST_DATA_SIZE: Int = 10000000 + private val SMALLER_DATA_SIZE: Int = 2000000 + private val MIN_NUM_ITERS: Int = 10 + + private val executorId: String = "0" + private val localPort: Int = 17000 + private val remotePort: Int = 17002 + + private val defaultConf: SparkConf = new SparkConf() + .set("spark.shuffle.compress", "false") + .set("spark.shuffle.spill.compress", "false") + .set("spark.authenticate", "false") + .set("spark.app.id", "test-app") + private val serializer: KryoSerializer = new KryoSerializer(defaultConf) + private val serializerManager: SerializerManager = new SerializerManager(serializer, defaultConf) + private val execBlockManagerId: BlockManagerId = + BlockManagerId(executorId, "localhost", localPort) + private val remoteBlockManagerId: BlockManagerId = + BlockManagerId(executorId, "localhost", remotePort) + private val transportConf: TransportConf = + SparkTransportConf.fromSparkConf(defaultConf, "shuffle") + private val securityManager: org.apache.spark.SecurityManager = + new org.apache.spark.SecurityManager(defaultConf) + protected val memoryManager: TestMemoryManager = new TestMemoryManager(defaultConf) + + class TestBlockManager(transferService: BlockTransferService, + blockManagerMaster: BlockManagerMaster, + dataFile: File, + fileLength: Long) extends BlockManager( + executorId, + rpcEnv, + blockManagerMaster, + serializerManager, + defaultConf, + memoryManager, + null, + null, + transferService, + null, + 1) { + blockManagerId = execBlockManagerId + + override def getBlockData(blockId: BlockId): ManagedBuffer = { + new FileSegmentManagedBuffer( + transportConf, + dataFile, + 0, + fileLength + ) + } + } + + private var blockManager : BlockManager = _ + private var externalBlockManager: BlockManager = _ + + def getTestBlockManager(port: Int, dataFile: File, dataFileLength: Long): TestBlockManager = { + val shuffleClient = new NettyBlockTransferService( + defaultConf, + securityManager, + "localhost", + "localhost", + port, + 1 + ) + new TestBlockManager(shuffleClient, + blockManagerMaster, + dataFile, + dataFileLength) + } + + def initializeServers(dataFile: File, dataFileLength: Long): Unit = { + MockitoAnnotations.initMocks(this) + when(blockManagerMaster.registerBlockManager( + any[BlockManagerId], any[Long], any[Long], any[RpcEndpointRef])).thenReturn(null) + when(rpcEnv.setupEndpoint(any[String], any[RpcEndpoint])).thenReturn(rpcEndpointRef) + blockManager = getTestBlockManager(localPort, dataFile, dataFileLength) + blockManager.initialize(defaultConf.getAppId) + externalBlockManager = getTestBlockManager(remotePort, dataFile, dataFileLength) + externalBlockManager.initialize(defaultConf.getAppId) + } + + def stopServers(): Unit = { + blockManager.stop() + externalBlockManager.stop() + } + + def setupReader( + dataFile: File, + dataFileLength: Long, + fetchLocal: Boolean, + aggregator: Option[Aggregator[String, String, String]] = None, + sorter: Option[Ordering[String]] = None): BlockStoreShuffleReader[String, String] = { + SparkEnv.set(new SparkEnv( + "0", + null, + serializer, + null, + serializerManager, + mapOutputTracker, + null, + null, + blockManager, + null, + null, + null, + null, + defaultConf + )) + + val shuffleHandle = new BaseShuffleHandle( + shuffleId = SHUFFLE_ID, + numMaps = NUM_MAPS, + dependency = dependency) + + // We cannot mock the TaskContext because it taskMetrics() gets called at every next() + // call on the reader, and Mockito will try to log all calls to taskMetrics(), thus OOM-ing + // the test + val taskContext = new TaskContext { + private val metrics: TaskMetrics = new TaskMetrics + private val testMemManager = new TestMemoryManager(defaultConf) + private val taskMemManager = new TaskMemoryManager(testMemManager, 0) + testMemManager.limit(PackedRecordPointer.MAXIMUM_PAGE_SIZE_BYTES) + override def isCompleted(): Boolean = false + override def isInterrupted(): Boolean = false + override def addTaskCompletionListener(listener: TaskCompletionListener): + TaskContext = { null } + override def addTaskFailureListener(listener: TaskFailureListener): TaskContext = { null } + override def stageId(): Int = 0 + override def stageAttemptNumber(): Int = 0 + override def partitionId(): Int = 0 + override def attemptNumber(): Int = 0 + override def taskAttemptId(): Long = 0 + override def getLocalProperty(key: String): String = "" + override def taskMetrics(): TaskMetrics = metrics + override def getMetricsSources(sourceName: String): Seq[Source] = Seq.empty + override private[spark] def killTaskIfInterrupted(): Unit = {} + override private[spark] def getKillReason() = None + override private[spark] def taskMemoryManager() = taskMemManager + override private[spark] def registerAccumulator(a: AccumulatorV2[_, _]): Unit = {} + override private[spark] def setFetchFailed(fetchFailed: FetchFailedException): Unit = {} + override private[spark] def markInterrupted(reason: String): Unit = {} + override private[spark] def markTaskFailed(error: Throwable): Unit = {} + override private[spark] def markTaskCompleted(error: Option[Throwable]): Unit = {} + override private[spark] def fetchFailed = None + override private[spark] def getLocalProperties = { null } + } + TaskContext.setTaskContext(taskContext) + + var dataBlockId: BlockManagerId = execBlockManagerId + if (!fetchLocal) { + dataBlockId = remoteBlockManagerId + } + + when(mapOutputTracker.getMapSizesByExecutorId(SHUFFLE_ID, REDUCE_ID, REDUCE_ID + 1)) + .thenReturn { + val shuffleBlockIdsAndSizes = (0 until NUM_MAPS).map { mapId => + val shuffleBlockId = ShuffleBlockId(SHUFFLE_ID, mapId, REDUCE_ID) + (shuffleBlockId, dataFileLength) + } + Seq((dataBlockId, shuffleBlockIdsAndSizes)).toIterator + } + + when(dependency.serializer).thenReturn(serializer) + when(dependency.aggregator).thenReturn(aggregator) + when(dependency.keyOrdering).thenReturn(sorter) + + new BlockStoreShuffleReader[String, String]( + shuffleHandle, + 0, + 1, + taskContext, + taskContext.taskMetrics().createTempShuffleReadMetrics(), + serializerManager, + blockManager, + mapOutputTracker + ) + } + + def generateDataOnDisk(size: Int, file: File): Long = { + // scalastyle:off println + println("Generating test data with num records: " + size) + val random = new Random(123) + val dataOutput = new FileOutputStream(file) + val coutingOutput = new CountingOutputStream(dataOutput) + val serializedOutput = serializer.newInstance().serializeStream(coutingOutput) + try { + (1 to size).foreach { i => { + if (i % 1000000 == 0) { + println("Wrote " + i + " test data points") + } + val x = random.alphanumeric.take(DEFAULT_DATA_STRING_SIZE).mkString + serializedOutput.writeKey(x) + serializedOutput.writeValue(x) + }} + } + finally { + serializedOutput.close() + } + coutingOutput.getCount + // scalastyle:off println + } + + def runWithLargeDataset(): Unit = { + val size = TEST_DATA_SIZE + val tempDataFile: File = File.createTempFile("test-data", "", tempDir) + val dataFileLength = generateDataOnDisk(size, tempDataFile) + val baseBenchmark = + new Benchmark("no aggregation or sorting", + size, + minNumIters = MIN_NUM_ITERS, + output = output, + outputPerIteration = true) + baseBenchmark.addTimerCase("local fetch") { timer => + val reader = setupReader(tempDataFile, dataFileLength, fetchLocal = true) + timer.startTiming() + val numRead = reader.read().length + timer.stopTiming() + assert(numRead == size * NUM_MAPS) + } + baseBenchmark.addTimerCase("remote rpc fetch") { timer => + val reader = setupReader(tempDataFile, dataFileLength, fetchLocal = false) + timer.startTiming() + val numRead = reader.read().length + timer.stopTiming() + assert(numRead == size * NUM_MAPS) + } + baseBenchmark.run() + tempDataFile.delete() + stopServers() + } + + def runWithSmallerDataset(): Unit = { + val size = SMALLER_DATA_SIZE + val smallerDataFile: File = File.createTempFile("test-data", "", tempDir) + val smallerFileLength = generateDataOnDisk(size, smallerDataFile) + initializeServers(smallerDataFile, smallerFileLength) + + 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) + val aggregationBenchmark = + new Benchmark("with aggregation", + size, + minNumIters = MIN_NUM_ITERS, + output = output, + outputPerIteration = true) + aggregationBenchmark.addTimerCase("local fetch") { timer => + val reader = setupReader( + smallerDataFile, + smallerFileLength, + fetchLocal = true, + aggregator = Some(aggregator)) + timer.startTiming() + val numRead = reader.read().length + timer.stopTiming() + assert(numRead > 0) + } + aggregationBenchmark.addTimerCase("remote rpc fetch") { timer => + val reader = setupReader( + smallerDataFile, + smallerFileLength, + fetchLocal = false, + aggregator = Some(aggregator)) + timer.startTiming() + val numRead = reader.read().length + timer.stopTiming() + assert(numRead > 0) + } + aggregationBenchmark.run() + + + val sorter = Ordering.String + val sortingBenchmark = + new Benchmark("with sorting", + size, + minNumIters = MIN_NUM_ITERS, + output = output, + outputPerIteration = true) + sortingBenchmark.addTimerCase("local fetch") { timer => + val reader = setupReader( + smallerDataFile, + smallerFileLength, + fetchLocal = true, + sorter = Some(sorter)) + timer.startTiming() + val numRead = reader.read().length + timer.stopTiming() + assert(numRead == size * NUM_MAPS) + } + sortingBenchmark.addTimerCase("remote rpc fetch") { timer => + val reader = setupReader( + smallerDataFile, + smallerFileLength, + fetchLocal = false, + sorter = Some(sorter)) + timer.startTiming() + val numRead = reader.read().length + timer.stopTiming() + assert(numRead == size * NUM_MAPS) + } + sortingBenchmark.run() + stopServers() + smallerDataFile.delete() + } + + override def runBenchmarkSuite(mainArgs: Array[String]): Unit = { + tempDir = Utils.createTempDir(null, "shuffle") + + runBenchmark("BlockStoreShuffleReader reader") { + runWithLargeDataset() + runWithSmallerDataset() + } + + FileUtils.deleteDirectory(tempDir) + } +} diff --git a/dev/run-spark-25299-benchmarks.sh b/dev/run-spark-25299-benchmarks.sh index 4f055f1f06271..82810899bbd3b 100755 --- a/dev/run-spark-25299-benchmarks.sh +++ b/dev/run-spark-25299-benchmarks.sh @@ -51,10 +51,12 @@ done echo "Running SPARK-25299 benchmarks" SPARK_GENERATE_BENCHMARK_FILES=1 ./build/sbt "sql/test:runMain org.apache.spark.shuffle.sort.SortShuffleWriterBenchmark" +SPARK_GENERATE_BENCHMARK_FILES=1 ./build/sbt "sql/test:runMain org.apache.spark.shuffle.sort.BlockStoreShuffleReaderBenchmark" SPARK_DIR=`pwd` mkdir -p /tmp/artifacts +cp $SPARK_DIR/sql/core/benchmarks/BlockStoreShuffleReaderBenchmark-results.txt /tmp/artifacts/ cp $SPARK_DIR/sql/core/benchmarks/SortShuffleWriterBenchmark-results.txt /tmp/artifacts/ if [ "$UPLOAD" = false ]; then From 9f517581140e2889cbf76a3c1b7aeec991d9fd7a Mon Sep 17 00:00:00 2001 From: Yifei Huang Date: Tue, 5 Mar 2019 12:44:44 -0800 Subject: [PATCH 52/68] Revert "SPARK-25299: Add shuffle reader benchmarks (#506)" This reverts commit 9d46fae9a36c6229a888bb647b1b63f51d83b407. --- .../BlockStoreShuffleReaderBenchmark.scala | 392 ------------------ dev/run-spark-25299-benchmarks.sh | 2 - 2 files changed, 394 deletions(-) delete mode 100644 core/src/test/scala/org/apache/spark/shuffle/sort/BlockStoreShuffleReaderBenchmark.scala diff --git a/core/src/test/scala/org/apache/spark/shuffle/sort/BlockStoreShuffleReaderBenchmark.scala b/core/src/test/scala/org/apache/spark/shuffle/sort/BlockStoreShuffleReaderBenchmark.scala deleted file mode 100644 index 70548af632495..0000000000000 --- a/core/src/test/scala/org/apache/spark/shuffle/sort/BlockStoreShuffleReaderBenchmark.scala +++ /dev/null @@ -1,392 +0,0 @@ -/* - * 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.{File, FileOutputStream} - -import com.google.common.io.CountingOutputStream -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.util.Random - -import org.apache.spark.{Aggregator, MapOutputTracker, ShuffleDependency, SparkConf, SparkEnv, TaskContext} -import org.apache.spark.benchmark.{Benchmark, BenchmarkBase} -import org.apache.spark.executor.TaskMetrics -import org.apache.spark.memory.{TaskMemoryManager, TestMemoryManager} -import org.apache.spark.metrics.source.Source -import org.apache.spark.network.BlockTransferService -import org.apache.spark.network.buffer.{FileSegmentManagedBuffer, ManagedBuffer} -import org.apache.spark.network.netty.{NettyBlockTransferService, SparkTransportConf} -import org.apache.spark.network.util.TransportConf -import org.apache.spark.rpc.{RpcEndpoint, RpcEndpointRef, RpcEnv} -import org.apache.spark.serializer.{KryoSerializer, SerializerManager} -import org.apache.spark.shuffle.{BaseShuffleHandle, BlockStoreShuffleReader, FetchFailedException} -import org.apache.spark.storage.{BlockId, BlockManager, BlockManagerId, BlockManagerMaster, ShuffleBlockId} -import org.apache.spark.util.{AccumulatorV2, TaskCompletionListener, TaskFailureListener, Utils} - -/** - * Benchmark to measure performance for aggregate primitives. - * {{{ - * To run this benchmark: - * 1. without sbt: bin/spark-submit --class - * 2. build/sbt "sql/test:runMain " - * 3. generate result: SPARK_GENERATE_BENCHMARK_FILES=1 build/sbt "sql/test:runMain " - * Results will be written to "benchmarks/-results.txt". - * }}} - */ -object BlockStoreShuffleReaderBenchmark extends BenchmarkBase { - - // this is only used to retrieve the aggregator/sorters/serializers, - // so it shouldn't affect the performance significantly - @Mock(answer = RETURNS_SMART_NULLS) private var dependency: - ShuffleDependency[String, String, String] = _ - // only used to retrieve info about the maps at the beginning, doesn't affect perf - @Mock(answer = RETURNS_SMART_NULLS) private var mapOutputTracker: MapOutputTracker = _ - // this is only used when initializing the BlockManager, so doesn't affect perf - @Mock(answer = RETURNS_SMART_NULLS) private var blockManagerMaster: BlockManagerMaster = _ - // this is only used when initiating the BlockManager, for comms between master and executor - @Mock(answer = RETURNS_SMART_NULLS) private var rpcEnv: RpcEnv = _ - @Mock(answer = RETURNS_SMART_NULLS) protected var rpcEndpointRef: RpcEndpointRef = _ - - private var tempDir: File = _ - - private val SHUFFLE_ID: Int = 0 - private val REDUCE_ID: Int = 0 - private val NUM_MAPS: Int = 5 - private val DEFAULT_DATA_STRING_SIZE = 5 - private val TEST_DATA_SIZE: Int = 10000000 - private val SMALLER_DATA_SIZE: Int = 2000000 - private val MIN_NUM_ITERS: Int = 10 - - private val executorId: String = "0" - private val localPort: Int = 17000 - private val remotePort: Int = 17002 - - private val defaultConf: SparkConf = new SparkConf() - .set("spark.shuffle.compress", "false") - .set("spark.shuffle.spill.compress", "false") - .set("spark.authenticate", "false") - .set("spark.app.id", "test-app") - private val serializer: KryoSerializer = new KryoSerializer(defaultConf) - private val serializerManager: SerializerManager = new SerializerManager(serializer, defaultConf) - private val execBlockManagerId: BlockManagerId = - BlockManagerId(executorId, "localhost", localPort) - private val remoteBlockManagerId: BlockManagerId = - BlockManagerId(executorId, "localhost", remotePort) - private val transportConf: TransportConf = - SparkTransportConf.fromSparkConf(defaultConf, "shuffle") - private val securityManager: org.apache.spark.SecurityManager = - new org.apache.spark.SecurityManager(defaultConf) - protected val memoryManager: TestMemoryManager = new TestMemoryManager(defaultConf) - - class TestBlockManager(transferService: BlockTransferService, - blockManagerMaster: BlockManagerMaster, - dataFile: File, - fileLength: Long) extends BlockManager( - executorId, - rpcEnv, - blockManagerMaster, - serializerManager, - defaultConf, - memoryManager, - null, - null, - transferService, - null, - 1) { - blockManagerId = execBlockManagerId - - override def getBlockData(blockId: BlockId): ManagedBuffer = { - new FileSegmentManagedBuffer( - transportConf, - dataFile, - 0, - fileLength - ) - } - } - - private var blockManager : BlockManager = _ - private var externalBlockManager: BlockManager = _ - - def getTestBlockManager(port: Int, dataFile: File, dataFileLength: Long): TestBlockManager = { - val shuffleClient = new NettyBlockTransferService( - defaultConf, - securityManager, - "localhost", - "localhost", - port, - 1 - ) - new TestBlockManager(shuffleClient, - blockManagerMaster, - dataFile, - dataFileLength) - } - - def initializeServers(dataFile: File, dataFileLength: Long): Unit = { - MockitoAnnotations.initMocks(this) - when(blockManagerMaster.registerBlockManager( - any[BlockManagerId], any[Long], any[Long], any[RpcEndpointRef])).thenReturn(null) - when(rpcEnv.setupEndpoint(any[String], any[RpcEndpoint])).thenReturn(rpcEndpointRef) - blockManager = getTestBlockManager(localPort, dataFile, dataFileLength) - blockManager.initialize(defaultConf.getAppId) - externalBlockManager = getTestBlockManager(remotePort, dataFile, dataFileLength) - externalBlockManager.initialize(defaultConf.getAppId) - } - - def stopServers(): Unit = { - blockManager.stop() - externalBlockManager.stop() - } - - def setupReader( - dataFile: File, - dataFileLength: Long, - fetchLocal: Boolean, - aggregator: Option[Aggregator[String, String, String]] = None, - sorter: Option[Ordering[String]] = None): BlockStoreShuffleReader[String, String] = { - SparkEnv.set(new SparkEnv( - "0", - null, - serializer, - null, - serializerManager, - mapOutputTracker, - null, - null, - blockManager, - null, - null, - null, - null, - defaultConf - )) - - val shuffleHandle = new BaseShuffleHandle( - shuffleId = SHUFFLE_ID, - numMaps = NUM_MAPS, - dependency = dependency) - - // We cannot mock the TaskContext because it taskMetrics() gets called at every next() - // call on the reader, and Mockito will try to log all calls to taskMetrics(), thus OOM-ing - // the test - val taskContext = new TaskContext { - private val metrics: TaskMetrics = new TaskMetrics - private val testMemManager = new TestMemoryManager(defaultConf) - private val taskMemManager = new TaskMemoryManager(testMemManager, 0) - testMemManager.limit(PackedRecordPointer.MAXIMUM_PAGE_SIZE_BYTES) - override def isCompleted(): Boolean = false - override def isInterrupted(): Boolean = false - override def addTaskCompletionListener(listener: TaskCompletionListener): - TaskContext = { null } - override def addTaskFailureListener(listener: TaskFailureListener): TaskContext = { null } - override def stageId(): Int = 0 - override def stageAttemptNumber(): Int = 0 - override def partitionId(): Int = 0 - override def attemptNumber(): Int = 0 - override def taskAttemptId(): Long = 0 - override def getLocalProperty(key: String): String = "" - override def taskMetrics(): TaskMetrics = metrics - override def getMetricsSources(sourceName: String): Seq[Source] = Seq.empty - override private[spark] def killTaskIfInterrupted(): Unit = {} - override private[spark] def getKillReason() = None - override private[spark] def taskMemoryManager() = taskMemManager - override private[spark] def registerAccumulator(a: AccumulatorV2[_, _]): Unit = {} - override private[spark] def setFetchFailed(fetchFailed: FetchFailedException): Unit = {} - override private[spark] def markInterrupted(reason: String): Unit = {} - override private[spark] def markTaskFailed(error: Throwable): Unit = {} - override private[spark] def markTaskCompleted(error: Option[Throwable]): Unit = {} - override private[spark] def fetchFailed = None - override private[spark] def getLocalProperties = { null } - } - TaskContext.setTaskContext(taskContext) - - var dataBlockId: BlockManagerId = execBlockManagerId - if (!fetchLocal) { - dataBlockId = remoteBlockManagerId - } - - when(mapOutputTracker.getMapSizesByExecutorId(SHUFFLE_ID, REDUCE_ID, REDUCE_ID + 1)) - .thenReturn { - val shuffleBlockIdsAndSizes = (0 until NUM_MAPS).map { mapId => - val shuffleBlockId = ShuffleBlockId(SHUFFLE_ID, mapId, REDUCE_ID) - (shuffleBlockId, dataFileLength) - } - Seq((dataBlockId, shuffleBlockIdsAndSizes)).toIterator - } - - when(dependency.serializer).thenReturn(serializer) - when(dependency.aggregator).thenReturn(aggregator) - when(dependency.keyOrdering).thenReturn(sorter) - - new BlockStoreShuffleReader[String, String]( - shuffleHandle, - 0, - 1, - taskContext, - taskContext.taskMetrics().createTempShuffleReadMetrics(), - serializerManager, - blockManager, - mapOutputTracker - ) - } - - def generateDataOnDisk(size: Int, file: File): Long = { - // scalastyle:off println - println("Generating test data with num records: " + size) - val random = new Random(123) - val dataOutput = new FileOutputStream(file) - val coutingOutput = new CountingOutputStream(dataOutput) - val serializedOutput = serializer.newInstance().serializeStream(coutingOutput) - try { - (1 to size).foreach { i => { - if (i % 1000000 == 0) { - println("Wrote " + i + " test data points") - } - val x = random.alphanumeric.take(DEFAULT_DATA_STRING_SIZE).mkString - serializedOutput.writeKey(x) - serializedOutput.writeValue(x) - }} - } - finally { - serializedOutput.close() - } - coutingOutput.getCount - // scalastyle:off println - } - - def runWithLargeDataset(): Unit = { - val size = TEST_DATA_SIZE - val tempDataFile: File = File.createTempFile("test-data", "", tempDir) - val dataFileLength = generateDataOnDisk(size, tempDataFile) - val baseBenchmark = - new Benchmark("no aggregation or sorting", - size, - minNumIters = MIN_NUM_ITERS, - output = output, - outputPerIteration = true) - baseBenchmark.addTimerCase("local fetch") { timer => - val reader = setupReader(tempDataFile, dataFileLength, fetchLocal = true) - timer.startTiming() - val numRead = reader.read().length - timer.stopTiming() - assert(numRead == size * NUM_MAPS) - } - baseBenchmark.addTimerCase("remote rpc fetch") { timer => - val reader = setupReader(tempDataFile, dataFileLength, fetchLocal = false) - timer.startTiming() - val numRead = reader.read().length - timer.stopTiming() - assert(numRead == size * NUM_MAPS) - } - baseBenchmark.run() - tempDataFile.delete() - stopServers() - } - - def runWithSmallerDataset(): Unit = { - val size = SMALLER_DATA_SIZE - val smallerDataFile: File = File.createTempFile("test-data", "", tempDir) - val smallerFileLength = generateDataOnDisk(size, smallerDataFile) - initializeServers(smallerDataFile, smallerFileLength) - - 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) - val aggregationBenchmark = - new Benchmark("with aggregation", - size, - minNumIters = MIN_NUM_ITERS, - output = output, - outputPerIteration = true) - aggregationBenchmark.addTimerCase("local fetch") { timer => - val reader = setupReader( - smallerDataFile, - smallerFileLength, - fetchLocal = true, - aggregator = Some(aggregator)) - timer.startTiming() - val numRead = reader.read().length - timer.stopTiming() - assert(numRead > 0) - } - aggregationBenchmark.addTimerCase("remote rpc fetch") { timer => - val reader = setupReader( - smallerDataFile, - smallerFileLength, - fetchLocal = false, - aggregator = Some(aggregator)) - timer.startTiming() - val numRead = reader.read().length - timer.stopTiming() - assert(numRead > 0) - } - aggregationBenchmark.run() - - - val sorter = Ordering.String - val sortingBenchmark = - new Benchmark("with sorting", - size, - minNumIters = MIN_NUM_ITERS, - output = output, - outputPerIteration = true) - sortingBenchmark.addTimerCase("local fetch") { timer => - val reader = setupReader( - smallerDataFile, - smallerFileLength, - fetchLocal = true, - sorter = Some(sorter)) - timer.startTiming() - val numRead = reader.read().length - timer.stopTiming() - assert(numRead == size * NUM_MAPS) - } - sortingBenchmark.addTimerCase("remote rpc fetch") { timer => - val reader = setupReader( - smallerDataFile, - smallerFileLength, - fetchLocal = false, - sorter = Some(sorter)) - timer.startTiming() - val numRead = reader.read().length - timer.stopTiming() - assert(numRead == size * NUM_MAPS) - } - sortingBenchmark.run() - stopServers() - smallerDataFile.delete() - } - - override def runBenchmarkSuite(mainArgs: Array[String]): Unit = { - tempDir = Utils.createTempDir(null, "shuffle") - - runBenchmark("BlockStoreShuffleReader reader") { - runWithLargeDataset() - runWithSmallerDataset() - } - - FileUtils.deleteDirectory(tempDir) - } -} diff --git a/dev/run-spark-25299-benchmarks.sh b/dev/run-spark-25299-benchmarks.sh index 82810899bbd3b..4f055f1f06271 100755 --- a/dev/run-spark-25299-benchmarks.sh +++ b/dev/run-spark-25299-benchmarks.sh @@ -51,12 +51,10 @@ done echo "Running SPARK-25299 benchmarks" SPARK_GENERATE_BENCHMARK_FILES=1 ./build/sbt "sql/test:runMain org.apache.spark.shuffle.sort.SortShuffleWriterBenchmark" -SPARK_GENERATE_BENCHMARK_FILES=1 ./build/sbt "sql/test:runMain org.apache.spark.shuffle.sort.BlockStoreShuffleReaderBenchmark" SPARK_DIR=`pwd` mkdir -p /tmp/artifacts -cp $SPARK_DIR/sql/core/benchmarks/BlockStoreShuffleReaderBenchmark-results.txt /tmp/artifacts/ cp $SPARK_DIR/sql/core/benchmarks/SortShuffleWriterBenchmark-results.txt /tmp/artifacts/ if [ "$UPLOAD" = false ]; then From bcb09c5b4d1bcbb8df1aa8e4421f26707dab7fcb Mon Sep 17 00:00:00 2001 From: Yifei Huang Date: Tue, 5 Mar 2019 13:51:56 -0800 Subject: [PATCH 53/68] add -e to bash script --- dev/run-spark-25299-benchmarks.sh | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/dev/run-spark-25299-benchmarks.sh b/dev/run-spark-25299-benchmarks.sh index 4f055f1f06271..09b20b1ccfd8b 100755 --- a/dev/run-spark-25299-benchmarks.sh +++ b/dev/run-spark-25299-benchmarks.sh @@ -23,7 +23,7 @@ # so it is completely self contained. # It does not contain source or *.class files. -set -ou pipefail +set -oue pipefail function usage { @@ -37,6 +37,7 @@ function usage { exit 1 } +mv blah asdf UPLOAD=false while getopts "hu" opt; do case $opt in From 25da7235292a6fd17859de3c83fa2573dd5e199c Mon Sep 17 00:00:00 2001 From: Yifei Huang Date: Tue, 5 Mar 2019 14:34:13 -0800 Subject: [PATCH 54/68] blah --- dev/run-spark-25299-benchmarks.sh | 1 - 1 file changed, 1 deletion(-) diff --git a/dev/run-spark-25299-benchmarks.sh b/dev/run-spark-25299-benchmarks.sh index 09b20b1ccfd8b..2a0fe2088f219 100755 --- a/dev/run-spark-25299-benchmarks.sh +++ b/dev/run-spark-25299-benchmarks.sh @@ -37,7 +37,6 @@ function usage { exit 1 } -mv blah asdf UPLOAD=false while getopts "hu" opt; do case $opt in From 13703fa476f11955631dd1f2e73be2bf69bbd253 Mon Sep 17 00:00:00 2001 From: Yifei Huang Date: Tue, 5 Mar 2019 17:43:04 -0800 Subject: [PATCH 55/68] enable upload as a PR comment and prevent running benchmarks on this branch --- .circleci/config.yml | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/.circleci/config.yml b/.circleci/config.yml index ccbfdfe318018..c9723929fdfc5 100644 --- a/.circleci/config.yml +++ b/.circleci/config.yml @@ -28,7 +28,6 @@ spark-25299-branch-only: &spark-25299-branch-only branches: only: - spark-25299 - - spark-25299-test-build - yh/add-benchmarks-and-ci deployable-branches-and-tags: &deployable-branches-and-tags @@ -474,7 +473,7 @@ jobs: - *restore-build-binaries-cache - *restore-home-sbt-cache - run: - command: ./dev/run-spark-25299-benchmarks.sh + command: ./dev/run-spark-25299-benchmarks.sh -u - store_artifacts: path: /tmp/artifacts/ From e3751cde60dabc813586806ffe28cdf17b7843f8 Mon Sep 17 00:00:00 2001 From: Yifei Huang Date: Wed, 6 Mar 2019 11:56:49 -0800 Subject: [PATCH 56/68] Revert "enable upload as a PR comment and prevent running benchmarks on this branch" This reverts commit 13703fa476f11955631dd1f2e73be2bf69bbd253. --- .circleci/config.yml | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/.circleci/config.yml b/.circleci/config.yml index c9723929fdfc5..ccbfdfe318018 100644 --- a/.circleci/config.yml +++ b/.circleci/config.yml @@ -28,6 +28,7 @@ spark-25299-branch-only: &spark-25299-branch-only branches: only: - spark-25299 + - spark-25299-test-build - yh/add-benchmarks-and-ci deployable-branches-and-tags: &deployable-branches-and-tags @@ -473,7 +474,7 @@ jobs: - *restore-build-binaries-cache - *restore-home-sbt-cache - run: - command: ./dev/run-spark-25299-benchmarks.sh -u + command: ./dev/run-spark-25299-benchmarks.sh - store_artifacts: path: /tmp/artifacts/ From 33a1b721e77aa06afddca100b1b1608b0c8548f1 Mon Sep 17 00:00:00 2001 From: Yifei Huang Date: Wed, 6 Mar 2019 12:00:54 -0800 Subject: [PATCH 57/68] try machine execution --- .circleci/config.yml | 9 ++++++--- 1 file changed, 6 insertions(+), 3 deletions(-) diff --git a/.circleci/config.yml b/.circleci/config.yml index ccbfdfe318018..1a47cc7167d0a 100644 --- a/.circleci/config.yml +++ b/.circleci/config.yml @@ -8,6 +8,11 @@ defaults: &defaults TERM: dumb BUILD_SBT_CACHE: "/home/circleci/build-sbt-cache" +spark-25299-config: &spark-25299-config + machine: true + environment: &defaults-environment + TERM: dumb + BUILD_SBT_CACHE: "/home/circleci/build-sbt-cache" test-defaults: &test-defaults <<: *defaults @@ -461,9 +466,7 @@ jobs: paths: ~/.m2 run-spark-25299-benchmarks: - <<: *defaults - docker: - - image: palantirtechnologies/circle-spark-r:0.1.3 + <<: *spark-25299-config steps: - *checkout-code - attach_workspace: From fa1b96cceb48e2c46505bc13eb0754fb037bb7a0 Mon Sep 17 00:00:00 2001 From: Yifei Huang Date: Thu, 7 Mar 2019 15:46:24 -0800 Subject: [PATCH 58/68] try uploading benchmarks (#498) --- .circleci/config.yml | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/.circleci/config.yml b/.circleci/config.yml index 1a47cc7167d0a..053b0749fe424 100644 --- a/.circleci/config.yml +++ b/.circleci/config.yml @@ -33,7 +33,6 @@ spark-25299-branch-only: &spark-25299-branch-only branches: only: - spark-25299 - - spark-25299-test-build - yh/add-benchmarks-and-ci deployable-branches-and-tags: &deployable-branches-and-tags @@ -477,7 +476,7 @@ jobs: - *restore-build-binaries-cache - *restore-home-sbt-cache - run: - command: ./dev/run-spark-25299-benchmarks.sh + command: ./dev/run-spark-25299-benchmarks.sh -u - store_artifacts: path: /tmp/artifacts/ From 37cef1fb1c5c34d9ae3e2e2b8e60e90d06f3a462 Mon Sep 17 00:00:00 2001 From: Yifei Huang Date: Mon, 11 Mar 2019 11:37:12 -0700 Subject: [PATCH 59/68] only upload results when merging into the feature branch --- .circleci/config.yml | 1 - 1 file changed, 1 deletion(-) diff --git a/.circleci/config.yml b/.circleci/config.yml index 053b0749fe424..2848d487a6081 100644 --- a/.circleci/config.yml +++ b/.circleci/config.yml @@ -33,7 +33,6 @@ spark-25299-branch-only: &spark-25299-branch-only branches: only: - spark-25299 - - yh/add-benchmarks-and-ci deployable-branches-and-tags: &deployable-branches-and-tags filters: From 459e1b5cfee6b1bc7775202b4870ca148e0bb5a6 Mon Sep 17 00:00:00 2001 From: Yifei Huang Date: Tue, 12 Mar 2019 15:35:46 -0700 Subject: [PATCH 60/68] lock down machine image --- .circleci/config.yml | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/.circleci/config.yml b/.circleci/config.yml index 2848d487a6081..383f3e351c91c 100644 --- a/.circleci/config.yml +++ b/.circleci/config.yml @@ -9,7 +9,8 @@ defaults: &defaults BUILD_SBT_CACHE: "/home/circleci/build-sbt-cache" spark-25299-config: &spark-25299-config - machine: true + machine: + image: circleci/classic:201808-01 environment: &defaults-environment TERM: dumb BUILD_SBT_CACHE: "/home/circleci/build-sbt-cache" From 4cabdbda7249db34e57e8a82f283fb36cf92694b Mon Sep 17 00:00:00 2001 From: Yifei Huang Date: Tue, 12 Mar 2019 18:08:54 -0700 Subject: [PATCH 61/68] don't write input data to disk --- .../sort/ShuffleWriterBenchmarkBase.scala | 51 ++++--------------- .../sort/SortShuffleWriterBenchmark.scala | 44 +++++++--------- 2 files changed, 27 insertions(+), 68 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/shuffle/sort/ShuffleWriterBenchmarkBase.scala b/core/src/test/scala/org/apache/spark/shuffle/sort/ShuffleWriterBenchmarkBase.scala index a9d7e0458c5c5..376205897766c 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/sort/ShuffleWriterBenchmarkBase.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/sort/ShuffleWriterBenchmarkBase.scala @@ -134,59 +134,26 @@ abstract class ShuffleWriterBenchmarkBase extends BenchmarkBase { filenameToFile.clear() } - protected class DataIterator private ( - inputStream: BufferedInputStream, - buffer: Array[Byte]) - extends Iterator[Product2[String, String]] with Closeable { + protected class DataIterator (size: Int, random: Random) + extends Iterator[Product2[String, String]] { + var count = 0 override def hasNext: Boolean = { - inputStream.available() > 0 + count < size } override def next(): Product2[String, String] = { - val read = inputStream.read(buffer) - assert(read == buffer.length) - val string = buffer.mkString + count+=1 + val string = random.nextString(DEFAULT_DATA_STRING_SIZE) (string, string) } - - override def close(): Unit = inputStream.close() - } - - protected object DataIterator { - def apply(inputFile: File, bufferSize: Int): DataIterator = { - val inputStream = new BufferedInputStream( - new FileInputStream(inputFile), DEFAULT_DATA_STRING_SIZE) - val buffer = new Array[Byte](DEFAULT_DATA_STRING_SIZE) - new DataIterator(inputStream, buffer) - } } private val random = new Random(123) - def createDataInMemory(size: Int): Array[(String, String)] = { - (1 to size).map { i => { - val x = random.alphanumeric.take(DEFAULT_DATA_STRING_SIZE).mkString - Tuple2(x, x) - } }.toArray - } - - def createDataOnDisk(size: Int): File = { - // scalastyle:off println - println("Generating test data with num records: " + size) - val tempDataFile = File.createTempFile("test-data", "") - Utils.tryWithResource(new FileOutputStream(tempDataFile)) { - dataOutput => - (1 to size).foreach { i => { - if (i % 1000000 == 0) { - println("Wrote " + i + " test data points") - } - val x = random.alphanumeric.take(DEFAULT_DATA_STRING_SIZE).mkString - dataOutput.write(x.getBytes) - }} - } - // scalastyle:on println - tempDataFile + def createDataIterator(size: Int): DataIterator = { + random.setSeed(123) + new DataIterator(size, random) } } diff --git a/core/src/test/scala/org/apache/spark/shuffle/sort/SortShuffleWriterBenchmark.scala b/core/src/test/scala/org/apache/spark/shuffle/sort/SortShuffleWriterBenchmark.scala index 4862f39425c90..799358db6daef 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/sort/SortShuffleWriterBenchmark.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/sort/SortShuffleWriterBenchmark.scala @@ -93,10 +93,10 @@ object SortShuffleWriterBenchmark extends ShuffleWriterBenchmarkBase { minNumIters = MIN_NUM_ITERS, output = output) addBenchmarkCase(benchmark, "small dataset without spills") { timer => - val writer = getWriter(Option.empty, Option.empty) - val array = createDataInMemory(size) + val shuffleWriter = getWriter(Option.empty, Option.empty) + val dataIterator = createDataIterator(size) timer.startTiming() - writer.write(array.iterator) + shuffleWriter.write(dataIterator) timer.stopTiming() assert(tempFilesCreated.isEmpty) } @@ -105,7 +105,6 @@ object SortShuffleWriterBenchmark extends ShuffleWriterBenchmarkBase { def writeBenchmarkWithSpill(): Unit = { val size = DATA_SIZE_LARGE - val dataFile = createDataOnDisk(size) val benchmark = new Benchmark("SortShuffleWriter with spills", size, @@ -114,13 +113,11 @@ object SortShuffleWriterBenchmark extends ShuffleWriterBenchmarkBase { outputPerIteration = true) addBenchmarkCase(benchmark, "no map side combine") { timer => val shuffleWriter = getWriter(Option.empty, Option.empty) - Utils.tryWithResource(DataIterator(inputFile = dataFile, DEFAULT_DATA_STRING_SIZE)) { - iterator => - timer.startTiming() - shuffleWriter.write(iterator) - timer.stopTiming() - } - assert(tempFilesCreated.length == 8) + val dataIterator = createDataIterator(size) + timer.startTiming() + shuffleWriter.write(dataIterator) + timer.stopTiming() + assert(tempFilesCreated.length == 7) } def createCombiner(i: String): String = i @@ -131,28 +128,23 @@ object SortShuffleWriterBenchmark extends ShuffleWriterBenchmarkBase { new Aggregator[String, String, String](createCombiner, mergeValue, mergeCombiners) addBenchmarkCase(benchmark, "with map side aggregation") { timer => val shuffleWriter = getWriter(Some(aggregator), Option.empty) - Utils.tryWithResource(DataIterator(inputFile = dataFile, DEFAULT_DATA_STRING_SIZE)) { - iterator => - timer.startTiming() - shuffleWriter.write(iterator) - timer.stopTiming() - } - assert(tempFilesCreated.length == 8) + val dataIterator = createDataIterator(size) + timer.startTiming() + shuffleWriter.write(dataIterator) + timer.stopTiming() + assert(tempFilesCreated.length == 7) } val sorter = Ordering.String addBenchmarkCase(benchmark, "with map side sort") { timer => val shuffleWriter = getWriter(Option.empty, Some(sorter)) - Utils.tryWithResource(DataIterator(inputFile = dataFile, DEFAULT_DATA_STRING_SIZE)) { - iterator => - timer.startTiming() - shuffleWriter.write(iterator) - timer.stopTiming() - } - assert(tempFilesCreated.length == 8) + val dataIterator = createDataIterator(size) + timer.startTiming() + shuffleWriter.write(dataIterator) + timer.stopTiming() + assert(tempFilesCreated.length == 7) } benchmark.run() - dataFile.delete() } override def runBenchmarkSuite(mainArgs: Array[String]): Unit = { From 47d2dcf7b2bd4b2eaf21a423e0c27b00e8b44bb2 Mon Sep 17 00:00:00 2001 From: Yifei Huang Date: Tue, 12 Mar 2019 18:32:20 -0700 Subject: [PATCH 62/68] run benchmark test --- .circleci/config.yml | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/.circleci/config.yml b/.circleci/config.yml index 383f3e351c91c..5b9d5ccea2fdb 100644 --- a/.circleci/config.yml +++ b/.circleci/config.yml @@ -34,6 +34,7 @@ spark-25299-branch-only: &spark-25299-branch-only branches: only: - spark-25299 + - yh/add-benchmarks-and-ci deployable-branches-and-tags: &deployable-branches-and-tags filters: @@ -476,7 +477,7 @@ jobs: - *restore-build-binaries-cache - *restore-home-sbt-cache - run: - command: ./dev/run-spark-25299-benchmarks.sh -u + command: ./dev/run-spark-25299-benchmarks.sh - store_artifacts: path: /tmp/artifacts/ From c78e4917b2d4d8bd1b8d7a2215bd70d6b04e9340 Mon Sep 17 00:00:00 2001 From: Yifei Huang Date: Wed, 13 Mar 2019 12:56:21 -0700 Subject: [PATCH 63/68] stop creating file cleanup threads for every block manager --- .../apache/spark/shuffle/sort/ShuffleWriterBenchmarkBase.scala | 1 + 1 file changed, 1 insertion(+) diff --git a/core/src/test/scala/org/apache/spark/shuffle/sort/ShuffleWriterBenchmarkBase.scala b/core/src/test/scala/org/apache/spark/shuffle/sort/ShuffleWriterBenchmarkBase.scala index 376205897766c..4d5d30200fce1 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/sort/ShuffleWriterBenchmarkBase.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/sort/ShuffleWriterBenchmarkBase.scala @@ -92,6 +92,7 @@ abstract class ShuffleWriterBenchmarkBase extends BenchmarkBase { null, 1) { override val diskBlockManager = new TestDiskBlockManager(tempDir) + override val remoteBlockTempFileManager = null } protected var tempDir: File = _ From f28b75c29ba666f00947316b2eec2932d3283bbd Mon Sep 17 00:00:00 2001 From: Yifei Huang Date: Wed, 13 Mar 2019 13:14:18 -0700 Subject: [PATCH 64/68] use alphanumeric again --- .../apache/spark/shuffle/sort/ShuffleWriterBenchmarkBase.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/test/scala/org/apache/spark/shuffle/sort/ShuffleWriterBenchmarkBase.scala b/core/src/test/scala/org/apache/spark/shuffle/sort/ShuffleWriterBenchmarkBase.scala index 4d5d30200fce1..747a7f6e7ece0 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/sort/ShuffleWriterBenchmarkBase.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/sort/ShuffleWriterBenchmarkBase.scala @@ -144,7 +144,7 @@ abstract class ShuffleWriterBenchmarkBase extends BenchmarkBase { override def next(): Product2[String, String] = { count+=1 - val string = random.nextString(DEFAULT_DATA_STRING_SIZE) + val string = random.alphanumeric.take(5).mkString (string, string) } } From a85acf4afbb3794c3fe0c8a63ea5a52d5013c034 Mon Sep 17 00:00:00 2001 From: Yifei Huang Date: Wed, 13 Mar 2019 13:16:41 -0700 Subject: [PATCH 65/68] use a new random everytime --- .../spark/shuffle/sort/ShuffleWriterBenchmarkBase.scala | 8 +++----- 1 file changed, 3 insertions(+), 5 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/shuffle/sort/ShuffleWriterBenchmarkBase.scala b/core/src/test/scala/org/apache/spark/shuffle/sort/ShuffleWriterBenchmarkBase.scala index 747a7f6e7ece0..8e6a69fb7080c 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/sort/ShuffleWriterBenchmarkBase.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/sort/ShuffleWriterBenchmarkBase.scala @@ -135,8 +135,9 @@ abstract class ShuffleWriterBenchmarkBase extends BenchmarkBase { filenameToFile.clear() } - protected class DataIterator (size: Int, random: Random) + protected class DataIterator (size: Int) extends Iterator[Product2[String, String]] { + val random = new Random(123) var count = 0 override def hasNext: Boolean = { count < size @@ -149,12 +150,9 @@ abstract class ShuffleWriterBenchmarkBase extends BenchmarkBase { } } - private val random = new Random(123) - def createDataIterator(size: Int): DataIterator = { - random.setSeed(123) - new DataIterator(size, random) + new DataIterator(size) } } From f26ab4072460656d276cd940f6fce9d3aa6e823b Mon Sep 17 00:00:00 2001 From: Yifei Huang Date: Wed, 13 Mar 2019 15:01:09 -0700 Subject: [PATCH 66/68] close the writers -__________- --- .../spark/shuffle/sort/SortShuffleWriterBenchmark.scala | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/core/src/test/scala/org/apache/spark/shuffle/sort/SortShuffleWriterBenchmark.scala b/core/src/test/scala/org/apache/spark/shuffle/sort/SortShuffleWriterBenchmark.scala index 799358db6daef..0b67fdc18d4a1 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/sort/SortShuffleWriterBenchmark.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/sort/SortShuffleWriterBenchmark.scala @@ -98,6 +98,7 @@ object SortShuffleWriterBenchmark extends ShuffleWriterBenchmarkBase { timer.startTiming() shuffleWriter.write(dataIterator) timer.stopTiming() + shuffleWriter.stop(true) assert(tempFilesCreated.isEmpty) } benchmark.run() @@ -117,6 +118,7 @@ object SortShuffleWriterBenchmark extends ShuffleWriterBenchmarkBase { timer.startTiming() shuffleWriter.write(dataIterator) timer.stopTiming() + shuffleWriter.stop(true) assert(tempFilesCreated.length == 7) } @@ -132,6 +134,7 @@ object SortShuffleWriterBenchmark extends ShuffleWriterBenchmarkBase { timer.startTiming() shuffleWriter.write(dataIterator) timer.stopTiming() + shuffleWriter.stop(true) assert(tempFilesCreated.length == 7) } @@ -142,6 +145,7 @@ object SortShuffleWriterBenchmark extends ShuffleWriterBenchmarkBase { timer.startTiming() shuffleWriter.write(dataIterator) timer.stopTiming() + shuffleWriter.stop(true) assert(tempFilesCreated.length == 7) } benchmark.run() From 103c660858bc35936085d7ec147b2e9e99359fde Mon Sep 17 00:00:00 2001 From: Yifei Huang Date: Wed, 13 Mar 2019 16:11:26 -0700 Subject: [PATCH 67/68] delete branch and publish results as comment --- .circleci/config.yml | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/.circleci/config.yml b/.circleci/config.yml index 5b9d5ccea2fdb..383f3e351c91c 100644 --- a/.circleci/config.yml +++ b/.circleci/config.yml @@ -34,7 +34,6 @@ spark-25299-branch-only: &spark-25299-branch-only branches: only: - spark-25299 - - yh/add-benchmarks-and-ci deployable-branches-and-tags: &deployable-branches-and-tags filters: @@ -477,7 +476,7 @@ jobs: - *restore-build-binaries-cache - *restore-home-sbt-cache - run: - command: ./dev/run-spark-25299-benchmarks.sh + command: ./dev/run-spark-25299-benchmarks.sh -u - store_artifacts: path: /tmp/artifacts/ From c3e58c5f932ac6a4c0b11a8ec5f7e65a8375a406 Mon Sep 17 00:00:00 2001 From: Yifei Huang Date: Wed, 13 Mar 2019 17:05:06 -0700 Subject: [PATCH 68/68] close in finally --- .../sort/SortShuffleWriterBenchmark.scala | 52 ++++++++++++------- 1 file changed, 32 insertions(+), 20 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/shuffle/sort/SortShuffleWriterBenchmark.scala b/core/src/test/scala/org/apache/spark/shuffle/sort/SortShuffleWriterBenchmark.scala index 0b67fdc18d4a1..317cd23279ede 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/sort/SortShuffleWriterBenchmark.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/sort/SortShuffleWriterBenchmark.scala @@ -95,11 +95,14 @@ object SortShuffleWriterBenchmark extends ShuffleWriterBenchmarkBase { addBenchmarkCase(benchmark, "small dataset without spills") { timer => val shuffleWriter = getWriter(Option.empty, Option.empty) val dataIterator = createDataIterator(size) - timer.startTiming() - shuffleWriter.write(dataIterator) - timer.stopTiming() - shuffleWriter.stop(true) - assert(tempFilesCreated.isEmpty) + try { + timer.startTiming() + shuffleWriter.write(dataIterator) + timer.stopTiming() + assert(tempFilesCreated.isEmpty) + } finally { + shuffleWriter.stop(true) + } } benchmark.run() } @@ -115,11 +118,14 @@ object SortShuffleWriterBenchmark extends ShuffleWriterBenchmarkBase { addBenchmarkCase(benchmark, "no map side combine") { timer => val shuffleWriter = getWriter(Option.empty, Option.empty) val dataIterator = createDataIterator(size) - timer.startTiming() - shuffleWriter.write(dataIterator) - timer.stopTiming() - shuffleWriter.stop(true) - assert(tempFilesCreated.length == 7) + try { + timer.startTiming() + shuffleWriter.write(dataIterator) + timer.stopTiming() + assert(tempFilesCreated.length == 7) + } finally { + shuffleWriter.stop(true) + } } def createCombiner(i: String): String = i @@ -131,22 +137,28 @@ object SortShuffleWriterBenchmark extends ShuffleWriterBenchmarkBase { addBenchmarkCase(benchmark, "with map side aggregation") { timer => val shuffleWriter = getWriter(Some(aggregator), Option.empty) val dataIterator = createDataIterator(size) - timer.startTiming() - shuffleWriter.write(dataIterator) - timer.stopTiming() - shuffleWriter.stop(true) - assert(tempFilesCreated.length == 7) + 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) - timer.startTiming() - shuffleWriter.write(dataIterator) - timer.stopTiming() - shuffleWriter.stop(true) - assert(tempFilesCreated.length == 7) + try { + timer.startTiming() + shuffleWriter.write(dataIterator) + timer.stopTiming() + assert(tempFilesCreated.length == 7) + } finally { + shuffleWriter.stop(true) + } } benchmark.run() }