From 3797cb2847e5eda44e841ce22713cb711795684b Mon Sep 17 00:00:00 2001 From: mcheah Date: Fri, 30 Aug 2019 09:43:07 -0700 Subject: [PATCH] [SPARK-28571][CORE][SHUFFLE] Use the shuffle writer plugin for the SortShuffleWriter Use the shuffle writer APIs introduced in SPARK-28209 in the sort shuffle writer. Existing unit tests were changed to use the plugin instead, and they used the local disk version to ensure that there were no regressions. Closes #25342 from mccheah/shuffle-writer-refactor-sort-shuffle-writer. Lead-authored-by: mcheah Co-authored-by: mccheah Signed-off-by: Marcelo Vanzin --- .../shuffle/ShufflePartitionPairsWriter.scala | 126 ++++++++++++++++++ .../shuffle/sort/SortShuffleWriter.scala | 4 +- .../spark/storage/DiskBlockObjectWriter.scala | 2 +- .../util/collection/ExternalSorter.scala | 37 ++--- .../spark/util/collection/PairsWriter.scala | 5 + .../shuffle/sort/SortShuffleWriterSuite.scala | 108 +++++++++++++++ 6 files changed, 262 insertions(+), 20 deletions(-) create mode 100644 core/src/main/scala/org/apache/spark/shuffle/ShufflePartitionPairsWriter.scala create mode 100644 core/src/test/scala/org/apache/spark/shuffle/sort/SortShuffleWriterSuite.scala diff --git a/core/src/main/scala/org/apache/spark/shuffle/ShufflePartitionPairsWriter.scala b/core/src/main/scala/org/apache/spark/shuffle/ShufflePartitionPairsWriter.scala new file mode 100644 index 0000000000000..a988c5e126a76 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/shuffle/ShufflePartitionPairsWriter.scala @@ -0,0 +1,126 @@ +/* + * 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.{Closeable, IOException, OutputStream} + +import org.apache.spark.serializer.{SerializationStream, SerializerInstance, SerializerManager} +import org.apache.spark.shuffle.api.ShufflePartitionWriter +import org.apache.spark.storage.BlockId +import org.apache.spark.util.Utils +import org.apache.spark.util.collection.PairsWriter + +/** + * A key-value writer inspired by {@link DiskBlockObjectWriter} that pushes the bytes to an + * arbitrary partition writer instead of writing to local disk through the block manager. + */ +private[spark] class ShufflePartitionPairsWriter( + partitionWriter: ShufflePartitionWriter, + serializerManager: SerializerManager, + serializerInstance: SerializerInstance, + blockId: BlockId, + writeMetrics: ShuffleWriteMetricsReporter) + extends PairsWriter with Closeable { + + private var isClosed = false + private var partitionStream: OutputStream = _ + private var wrappedStream: OutputStream = _ + private var objOut: SerializationStream = _ + private var numRecordsWritten = 0 + private var curNumBytesWritten = 0L + + override def write(key: Any, value: Any): Unit = { + if (isClosed) { + throw new IOException("Partition pairs writer is already closed.") + } + if (objOut == null) { + open() + } + objOut.writeKey(key) + objOut.writeValue(value) + recordWritten() + } + + private def open(): Unit = { + try { + partitionStream = partitionWriter.openStream + wrappedStream = serializerManager.wrapStream(blockId, partitionStream) + objOut = serializerInstance.serializeStream(wrappedStream) + } catch { + case e: Exception => + Utils.tryLogNonFatalError { + close() + } + throw e + } + } + + override def close(): Unit = { + if (!isClosed) { + Utils.tryWithSafeFinally { + Utils.tryWithSafeFinally { + objOut = closeIfNonNull(objOut) + // Setting these to null will prevent the underlying streams from being closed twice + // just in case any stream's close() implementation is not idempotent. + wrappedStream = null + partitionStream = null + } { + // Normally closing objOut would close the inner streams as well, but just in case there + // was an error in initialization etc. we make sure we clean the other streams up too. + Utils.tryWithSafeFinally { + wrappedStream = closeIfNonNull(wrappedStream) + // Same as above - if wrappedStream closes then assume it closes underlying + // partitionStream and don't close again in the finally + partitionStream = null + } { + partitionStream = closeIfNonNull(partitionStream) + } + } + updateBytesWritten() + } { + isClosed = true + } + } + } + + private def closeIfNonNull[T <: Closeable](closeable: T): T = { + if (closeable != null) { + closeable.close() + } + null.asInstanceOf[T] + } + + /** + * Notify the writer that a record worth of bytes has been written with OutputStream#write. + */ + private def recordWritten(): Unit = { + numRecordsWritten += 1 + writeMetrics.incRecordsWritten(1) + + if (numRecordsWritten % 16384 == 0) { + updateBytesWritten() + } + } + + private def updateBytesWritten(): Unit = { + val numBytesWritten = partitionWriter.getNumBytesWritten + val bytesWrittenDiff = numBytesWritten - curNumBytesWritten + writeMetrics.incBytesWritten(bytesWrittenDiff) + curNumBytesWritten = numBytesWritten + } +} diff --git a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala index 626f5fd91c291..0082b4c9c6b24 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala @@ -67,9 +67,9 @@ private[spark] class SortShuffleWriter[K, V, C]( val mapOutputWriter = shuffleExecutorComponents.createMapOutputWriter( dep.shuffleId, mapId, context.taskAttemptId(), dep.partitioner.numPartitions) sorter.writePartitionedMapOutput(dep.shuffleId, mapId, mapOutputWriter) - val commitMessage = mapOutputWriter.commitAllPartitions + val commitMessage = mapOutputWriter.commitAllPartitions() mapStatus = MapStatus( - commitMessage.getLocation().orElse(null), + commitMessage.getLocation.orElse(null), commitMessage.getPartitionLengths, context.taskAttemptId()) } diff --git a/core/src/main/scala/org/apache/spark/storage/DiskBlockObjectWriter.scala b/core/src/main/scala/org/apache/spark/storage/DiskBlockObjectWriter.scala index f9f4e3594e4f9..758621c52495b 100644 --- a/core/src/main/scala/org/apache/spark/storage/DiskBlockObjectWriter.scala +++ b/core/src/main/scala/org/apache/spark/storage/DiskBlockObjectWriter.scala @@ -234,7 +234,7 @@ private[spark] class DiskBlockObjectWriter( /** * Writes a key-value pair. */ - def write(key: Any, value: Any) { + override def write(key: Any, value: Any) { if (!streamOpen) { open() } diff --git a/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala b/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala index 2f967a3cdfae0..1c8334be9a2bb 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala @@ -24,14 +24,18 @@ import scala.collection.mutable import scala.collection.mutable.ArrayBuffer import com.google.common.io.ByteStreams -import org.apache.spark._ +import com.google.common.io.{ByteStreams, Closeables} -import org.apache.spark.api.shuffle.ShufflePartitionWriter +import org.apache.spark._ import org.apache.spark.executor.ShuffleWriteMetrics import org.apache.spark.internal.{config, Logging} import org.apache.spark.serializer._ import org.apache.spark.shuffle.api.{ShuffleMapOutputWriter, ShufflePartitionWriter} import org.apache.spark.storage.{BlockId, DiskBlockObjectWriter, ShuffleBlockId} +import org.apache.spark.shuffle.ShufflePartitionPairsWriter +import org.apache.spark.shuffle.api.{ShuffleMapOutputWriter, ShufflePartitionWriter} +import org.apache.spark.storage.{BlockId, DiskBlockObjectWriter, ShuffleBlockId} +import org.apache.spark.util.{Utils => TryUtils} /** * Sorts and potentially merges a number of key-value pairs of type (K, V) to produce key-combiner @@ -676,9 +680,9 @@ private[spark] class ExternalSorter[K, V, C]( } /** - * TODO remove this, as this is only used by UnsafeRowSerializerSuite in the SQL project. - * We should figure out an alternative way to test that so that we can remove this otherwise - * unused code path. + * TODO(SPARK-28764): remove this, as this is only used by UnsafeRowSerializerSuite in the SQL + * project. We should figure out an alternative way to test that so that we can remove this + * otherwise unused code path. */ def writePartitionedFile( blockId: BlockId, @@ -729,7 +733,10 @@ private[spark] class ExternalSorter[K, V, C]( * @return array of lengths, in bytes, of each partition of the file (used by map output tracker) */ def writePartitionedMapOutput( - shuffleId: Int, mapId: Int, mapOutputWriter: ShuffleMapOutputWriter): Unit = { + shuffleId: Int, + mapId: Int, + mapOutputWriter: ShuffleMapOutputWriter): Unit = { + var nextPartitionId = 0 if (spills.isEmpty) { // Case where we only have in-memory data val collection = if (aggregator.isDefined) map else buffer @@ -738,7 +745,7 @@ private[spark] class ExternalSorter[K, V, C]( val partitionId = it.nextPartition() var partitionWriter: ShufflePartitionWriter = null var partitionPairsWriter: ShufflePartitionPairsWriter = null - try { + TryUtils.tryWithSafeFinally { partitionWriter = mapOutputWriter.getPartitionWriter(partitionId) val blockId = ShuffleBlockId(shuffleId, mapId, partitionId) partitionPairsWriter = new ShufflePartitionPairsWriter( @@ -750,25 +757,20 @@ private[spark] class ExternalSorter[K, V, C]( while (it.hasNext && it.nextPartition() == partitionId) { it.writeNext(partitionPairsWriter) } - } finally { + } { if (partitionPairsWriter != null) { partitionPairsWriter.close() } } + nextPartitionId = partitionId + 1 } } else { // We must perform merge-sort; get an iterator by partition and write everything directly. for ((id, elements) <- this.partitionedIterator) { - // The contract for the plugin is that we will ask for a writer for every partition - // even if it's empty. However, the external sorter will return non-contiguous - // partition ids. So this loop "backfills" the empty partitions that form the gaps. - - // The algorithm as a whole is correct because the partition ids are returned by the - // iterator in ascending order. val blockId = ShuffleBlockId(shuffleId, mapId, id) var partitionWriter: ShufflePartitionWriter = null var partitionPairsWriter: ShufflePartitionPairsWriter = null - try { + TryUtils.tryWithSafeFinally { partitionWriter = mapOutputWriter.getPartitionWriter(id) partitionPairsWriter = new ShufflePartitionPairsWriter( partitionWriter, @@ -781,11 +783,12 @@ private[spark] class ExternalSorter[K, V, C]( partitionPairsWriter.write(elem._1, elem._2) } } - } finally { - if (partitionPairsWriter!= null) { + } { + if (partitionPairsWriter != null) { partitionPairsWriter.close() } } + nextPartitionId = id + 1 } } diff --git a/core/src/main/scala/org/apache/spark/util/collection/PairsWriter.scala b/core/src/main/scala/org/apache/spark/util/collection/PairsWriter.scala index 9d7c209f242e1..05ed72c3e3778 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/PairsWriter.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/PairsWriter.scala @@ -17,6 +17,11 @@ package org.apache.spark.util.collection +/** + * An abstraction of a consumer of key-value pairs, primarily used when + * persisting partitioned data, either through the shuffle writer plugins + * or via DiskBlockObjectWriter. + */ private[spark] trait PairsWriter { def write(key: Any, value: Any): Unit diff --git a/core/src/test/scala/org/apache/spark/shuffle/sort/SortShuffleWriterSuite.scala b/core/src/test/scala/org/apache/spark/shuffle/sort/SortShuffleWriterSuite.scala new file mode 100644 index 0000000000000..0dd6040808f9e --- /dev/null +++ b/core/src/test/scala/org/apache/spark/shuffle/sort/SortShuffleWriterSuite.scala @@ -0,0 +1,108 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.shuffle.sort + +import org.mockito.{Mock, MockitoAnnotations} +import org.mockito.Answers.RETURNS_SMART_NULLS +import org.mockito.Mockito._ +import org.scalatest.Matchers + +import org.apache.spark.{Partitioner, SharedSparkContext, ShuffleDependency, SparkFunSuite} +import org.apache.spark.memory.MemoryTestingUtils +import org.apache.spark.serializer.JavaSerializer +import org.apache.spark.shuffle.{BaseShuffleHandle, IndexShuffleBlockResolver} +import org.apache.spark.shuffle.api.ShuffleExecutorComponents +import org.apache.spark.shuffle.sort.io.LocalDiskShuffleExecutorComponents +import org.apache.spark.storage.BlockManager +import org.apache.spark.util.Utils + + +class SortShuffleWriterSuite extends SparkFunSuite with SharedSparkContext with Matchers { + + @Mock(answer = RETURNS_SMART_NULLS) + private var blockManager: BlockManager = _ + + private val shuffleId = 0 + private val numMaps = 5 + private var shuffleHandle: BaseShuffleHandle[Int, Int, Int] = _ + private val shuffleBlockResolver = new IndexShuffleBlockResolver(conf) + private val serializer = new JavaSerializer(conf) + private var shuffleExecutorComponents: ShuffleExecutorComponents = _ + + override def beforeEach(): Unit = { + super.beforeEach() + MockitoAnnotations.initMocks(this) + val partitioner = new Partitioner() { + def numPartitions = numMaps + def getPartition(key: Any) = Utils.nonNegativeMod(key.hashCode, numPartitions) + } + shuffleHandle = { + val dependency = mock(classOf[ShuffleDependency[Int, Int, Int]]) + when(dependency.partitioner).thenReturn(partitioner) + when(dependency.serializer).thenReturn(serializer) + when(dependency.aggregator).thenReturn(None) + when(dependency.keyOrdering).thenReturn(None) + new BaseShuffleHandle(shuffleId, numMaps = numMaps, dependency) + } + shuffleExecutorComponents = new LocalDiskShuffleExecutorComponents( + conf, blockManager, shuffleBlockResolver) + } + + override def afterAll(): Unit = { + try { + shuffleBlockResolver.stop() + } finally { + super.afterAll() + } + } + + test("write empty iterator") { + val context = MemoryTestingUtils.fakeTaskContext(sc.env) + val writer = new SortShuffleWriter[Int, Int, Int]( + shuffleBlockResolver, + shuffleHandle, + mapId = 1, + context, + shuffleExecutorComponents) + writer.write(Iterator.empty) + writer.stop(success = true) + val dataFile = shuffleBlockResolver.getDataFile(shuffleId, 1) + val writeMetrics = context.taskMetrics().shuffleWriteMetrics + assert(!dataFile.exists()) + assert(writeMetrics.bytesWritten === 0) + assert(writeMetrics.recordsWritten === 0) + } + + test("write with some records") { + val context = MemoryTestingUtils.fakeTaskContext(sc.env) + val records = List[(Int, Int)]((1, 2), (2, 3), (4, 4), (6, 5)) + val writer = new SortShuffleWriter[Int, Int, Int]( + shuffleBlockResolver, + shuffleHandle, + mapId = 2, + context, + shuffleExecutorComponents) + writer.write(records.toIterator) + writer.stop(success = true) + val dataFile = shuffleBlockResolver.getDataFile(shuffleId, 2) + val writeMetrics = context.taskMetrics().shuffleWriteMetrics + assert(dataFile.exists()) + assert(dataFile.length() === writeMetrics.bytesWritten) + assert(records.size === writeMetrics.recordsWritten) + } +}