From 261c9fcac2bc77204d1751f41fa5192a59210c4e Mon Sep 17 00:00:00 2001 From: "yi.wu" Date: Wed, 9 Jun 2021 00:40:22 +0800 Subject: [PATCH 01/47] update --- .../sort/BypassMergeSortShuffleWriter.java | 27 +++- .../shuffle/sort/ShuffleExternalSorter.java | 30 ++++ .../shuffle/sort/UnsafeShuffleWriter.java | 5 +- .../LocalDiskShuffleExecutorComponents.java | 1 + .../spark/internal/config/package.scala | 8 ++ .../spark/io/MutableCheckedOutputStream.scala | 47 +++++++ .../shuffle/IndexShuffleBlockResolver.scala | 77 +++++++++++ .../shuffle/ShufflePartitionPairsWriter.scala | 27 +++- .../org/apache/spark/storage/BlockId.scala | 5 + .../spark/storage/DiskBlockObjectWriter.scala | 27 +++- .../util/collection/ExternalSorter.scala | 18 ++- .../sort/UnsafeShuffleWriterSuite.java | 129 +++++++++++++++++- .../BypassMergeSortShuffleWriterSuite.scala | 70 +++++++++- .../sort/IndexShuffleBlockResolverSuite.scala | 10 ++ .../sort/ShuffleExternalSorterSuite.scala | 2 + .../shuffle/sort/SortShuffleWriterSuite.scala | 116 +++++++++++++++- 16 files changed, 581 insertions(+), 18 deletions(-) create mode 100644 core/src/main/scala/org/apache/spark/io/MutableCheckedOutputStream.scala diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java b/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java index 3dbee1b13d28..6d8ece9d31e8 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java @@ -23,8 +23,11 @@ import java.io.OutputStream; import java.nio.channels.FileChannel; import java.util.Optional; +import java.util.zip.Adler32; +import java.util.zip.Checksum; import javax.annotation.Nullable; +import org.apache.spark.shuffle.IndexShuffleBlockResolver; import scala.None$; import scala.Option; import scala.Product2; @@ -93,6 +96,8 @@ final class BypassMergeSortShuffleWriter extends ShuffleWriter { private FileSegment[] partitionWriterSegments; @Nullable private MapStatus mapStatus; private long[] partitionLengths; + private Checksum[] partitionChecksums; + private boolean checksumEnabled; /** * Are we in the process of stopping? Because map tasks can call stop() with success = true @@ -120,6 +125,13 @@ final class BypassMergeSortShuffleWriter extends ShuffleWriter { this.writeMetrics = writeMetrics; this.serializer = dep.serializer(); this.shuffleExecutorComponents = shuffleExecutorComponents; + if ((boolean) conf.get(package$.MODULE$.SHUFFLE_CHECKSUM())) { + this.checksumEnabled = true; + this.partitionChecksums = new Adler32[numPartitions]; + for (int i = 0; i < numPartitions; i ++) { + this.partitionChecksums[i] = new Adler32(); + } + } } @Override @@ -143,8 +155,12 @@ public void write(Iterator> records) throws IOException { blockManager.diskBlockManager().createTempShuffleBlock(); final File file = tempShuffleBlockIdPlusFile._2(); final BlockId blockId = tempShuffleBlockIdPlusFile._1(); - partitionWriters[i] = - blockManager.getDiskWriter(blockId, file, serInstance, fileBufferSize, writeMetrics); + DiskBlockObjectWriter writer = + blockManager.getDiskWriter(blockId, file, serInstance, fileBufferSize, writeMetrics); + if (checksumEnabled) { + writer.setChecksum(partitionChecksums[i]); + } + partitionWriters[i] = writer; } // Creating the file to write to and creating a disk writer both involve interacting with // the disk, and can take a long time in aggregate when we open many files, so should be @@ -164,6 +180,13 @@ public void write(Iterator> records) throws IOException { } partitionLengths = writePartitionedData(mapOutputWriter); + if (checksumEnabled) { + long[] checksums = new long[numPartitions]; + for (int i = 0; i < numPartitions; i ++) { + checksums[i] = partitionChecksums[i].getValue(); + } + IndexShuffleBlockResolver.get().writeChecksumFile(shuffleId, mapId, checksums); + } mapStatus = MapStatus$.MODULE$.apply( blockManager.shuffleServerId(), partitionLengths, mapId); } catch (Exception e) { diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleExternalSorter.java b/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleExternalSorter.java index 833744f4777c..6e92f9bff9da 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleExternalSorter.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleExternalSorter.java @@ -21,7 +21,10 @@ import java.io.File; import java.io.IOException; import java.util.LinkedList; +import java.util.zip.Adler32; +import java.util.zip.Checksum; +import org.apache.spark.shuffle.IndexShuffleBlockResolver; import scala.Tuple2; import com.google.common.annotations.VisibleForTesting; @@ -72,6 +75,8 @@ final class ShuffleExternalSorter extends MemoryConsumer { @VisibleForTesting static final int DISK_WRITE_BUFFER_SIZE = 1024 * 1024; + private final int shuffleId; + private final long mapId; private final int numPartitions; private final TaskMemoryManager taskMemoryManager; private final BlockManager blockManager; @@ -107,11 +112,16 @@ final class ShuffleExternalSorter extends MemoryConsumer { @Nullable private MemoryBlock currentPage = null; private long pageCursor = -1; + private boolean checksumEnabled; + private Checksum[] partitionChecksums; + ShuffleExternalSorter( TaskMemoryManager memoryManager, BlockManager blockManager, TaskContext taskContext, int initialSize, + int shuffleId, + long mapId, int numPartitions, SparkConf conf, ShuffleWriteMetricsReporter writeMetrics) { @@ -121,6 +131,8 @@ final class ShuffleExternalSorter extends MemoryConsumer { this.taskMemoryManager = memoryManager; this.blockManager = blockManager; this.taskContext = taskContext; + this.shuffleId = shuffleId; + this.mapId = mapId; this.numPartitions = numPartitions; // Use getSizeAsKb (not bytes) to maintain backwards compatibility if no units are provided this.fileBufferSizeBytes = @@ -133,6 +145,21 @@ final class ShuffleExternalSorter extends MemoryConsumer { this.peakMemoryUsedBytes = getMemoryUsage(); this.diskWriteBufferSize = (int) (long) conf.get(package$.MODULE$.SHUFFLE_DISK_WRITE_BUFFER_SIZE()); + if ((boolean) conf.get(package$.MODULE$.SHUFFLE_CHECKSUM())) { + this.checksumEnabled = true; + this.partitionChecksums = new Adler32[numPartitions]; + for (int i = 0; i < numPartitions; i ++) { + this.partitionChecksums[i] = new Adler32(); + } + } + } + + public void writeChecksumFile() { + long[] checksums = new long[numPartitions]; + for (int i = 0; i < numPartitions; i ++) { + checksums[i] = partitionChecksums[i].getValue(); + } + IndexShuffleBlockResolver.get().writeChecksumFile(shuffleId, mapId, checksums); } /** @@ -204,6 +231,9 @@ private void writeSortedFile(boolean isLastFile) { spillInfo.partitionLengths[currentPartition] = fileSegment.length(); } currentPartition = partition; + if (checksumEnabled) { + writer.setChecksum(partitionChecksums[currentPartition]); + } } final long recordPointer = sortedRecords.packedRecordPointer.getRecordPointer(); diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java b/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java index e8f94ba8ffee..1ff5ea46b745 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java @@ -205,6 +205,8 @@ private void open() { blockManager, taskContext, initialSortBufferSize, + shuffleId, + mapId, partitioner.numPartitions(), sparkConf, writeMetrics); @@ -219,9 +221,10 @@ void closeAndWriteOutput() throws IOException { serBuffer = null; serOutputStream = null; final SpillInfo[] spills = sorter.closeAndGetSpills(); - sorter = null; try { partitionLengths = mergeSpills(spills); + sorter.writeChecksumFile(); + sorter = null; } finally { for (SpillInfo spill : spills) { if (spill.file.exists() && !spill.file.delete()) { diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/io/LocalDiskShuffleExecutorComponents.java b/core/src/main/java/org/apache/spark/shuffle/sort/io/LocalDiskShuffleExecutorComponents.java index eb4d9d9abc8e..9e8c54d9315f 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/io/LocalDiskShuffleExecutorComponents.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/io/LocalDiskShuffleExecutorComponents.java @@ -57,6 +57,7 @@ public void initializeExecutor(String appId, String execId, Map throw new IllegalStateException("No blockManager available from the SparkEnv."); } blockResolver = new IndexShuffleBlockResolver(sparkConf, blockManager); + IndexShuffleBlockResolver.set(blockResolver); } @Override diff --git a/core/src/main/scala/org/apache/spark/internal/config/package.scala b/core/src/main/scala/org/apache/spark/internal/config/package.scala index 613a66d2d5ac..43ebd8d85875 100644 --- a/core/src/main/scala/org/apache/spark/internal/config/package.scala +++ b/core/src/main/scala/org/apache/spark/internal/config/package.scala @@ -1368,6 +1368,14 @@ package object config { s"The buffer size must be greater than 0 and less than or equal to ${Int.MaxValue}.") .createWithDefault(4096) + private[spark] val SHUFFLE_CHECKSUM = + ConfigBuilder("spark.shuffle.checksum") + .doc("Whether to calculate the checksum of shuffle output. If enabled, Spark will try " + + "its best to tell if shuffle data corruption is caused by network or disk or others.") + .version("3.2.0") + .booleanConf + .createWithDefault(true) + private[spark] val SHUFFLE_COMPRESS = ConfigBuilder("spark.shuffle.compress") .doc("Whether to compress shuffle output. Compression will use " + diff --git a/core/src/main/scala/org/apache/spark/io/MutableCheckedOutputStream.scala b/core/src/main/scala/org/apache/spark/io/MutableCheckedOutputStream.scala new file mode 100644 index 000000000000..dd44ad024af2 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/io/MutableCheckedOutputStream.scala @@ -0,0 +1,47 @@ +/* + * 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.io + +import java.io.OutputStream +import java.util.zip.Checksum + +/** + * A variant of [[java.util.zip.CheckedOutputStream]] which can + * change the checksum calculator at runtime. + */ +class MutableCheckedOutputStream(out: OutputStream) extends OutputStream { + private var checksum: Checksum = _ + + def setChecksum(c: Checksum): Unit = this.checksum = c + + override def write(b: Int): Unit = { + assert(checksum != null, "Checksum is not set.") + checksum.update(b) + out.write(b) + } + + override def write(b: Array[Byte], off: Int, len: Int): Unit = { + assert(checksum != null, "Checksum is not set.") + checksum.update(b, off, len) + out.write(b, off, len) + } + + override def flush(): Unit = out.flush() + + override def close(): Unit = out.close() +} diff --git a/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala b/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala index 5d1da19e69f4..5c788f47fc69 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala @@ -22,6 +22,8 @@ import java.nio.ByteBuffer import java.nio.channels.Channels import java.nio.file.Files +import scala.collection.mutable.ArrayBuffer + import org.apache.spark.{SparkConf, SparkEnv, SparkException} import org.apache.spark.internal.{config, Logging} import org.apache.spark.io.NioBufferedFileInputStream @@ -414,6 +416,75 @@ private[spark] class IndexShuffleBlockResolver( new MergedBlockMeta(numChunks, chunkBitMaps) } + private[shuffle] def getChecksums(checksumFile: File, blockNum: Int): Array[Long] = { + if (!checksumFile.exists()) return null + val checksums = new ArrayBuffer[Long] + // Read the checksums of blocks + var in: DataInputStream = null + try { + in = new DataInputStream(new NioBufferedFileInputStream(checksumFile)) + while (checksums.size < blockNum) { + checksums += in.readLong() + } + } catch { + case _: IOException | _: EOFException => + return null + } finally { + in.close() + } + + checksums.toArray + } + + /** + * Get the shuffle checksum file. + * + * When the dirs parameter is None then use the disk manager's local directories. Otherwise, + * read from the specified directories. + */ + def getChecksumFile( + shuffleId: Int, + mapId: Long, + dirs: Option[Array[String]] = None): File = { + val blockId = ShuffleChecksumBlockId(shuffleId, mapId, NOOP_REDUCE_ID) + dirs + .map(ExecutorDiskUtils.getFile(_, blockManager.subDirsPerLocalDir, blockId.name)) + .getOrElse { + blockManager.diskBlockManager.getFile(blockId) + } + } + + def writeChecksumFile(shuffleId: Int, mapId: Long, checksums: Array[Long]): Unit = synchronized { + val checksumFile = getChecksumFile(shuffleId, mapId) + val checksumTmp = Utils.tempFileWith(checksumFile) + val existingChecksums = getChecksums(checksumFile, checksums.length) + if (existingChecksums != null) { + // Another attempt for the same task has already written our checksum file successfully, + // so just use the existing checksums. + System.arraycopy(existingChecksums, 0, checksums, 0, checksums.length) + } else { + val out = new DataOutputStream( + new BufferedOutputStream( + new FileOutputStream(checksumTmp) + ) + ) + Utils.tryWithSafeFinally { + checksums.foreach(out.writeLong) + } { + out.close() + } + + if (checksumFile.exists()) { + checksumFile.delete() + } + if (!checksumTmp.renameTo(checksumFile)) { + // It's not worthwhile to fail here after index file and data file are already + // successfully stored due to checksum is only used for the corner error case. + logWarning("fail to rename file " + checksumTmp + " to " + checksumFile) + } + } + } + override def getBlockData( blockId: BlockId, dirs: Option[Array[String]]): ManagedBuffer = { @@ -466,4 +537,10 @@ private[spark] object IndexShuffleBlockResolver { // The disk store currently expects puts to relate to a (map, reduce) pair, but in the sort // shuffle outputs for several reduces are glommed into a single file. val NOOP_REDUCE_ID = 0 + + @volatile private var _blockResolver: IndexShuffleBlockResolver = _ + + def set(resolver: IndexShuffleBlockResolver): Unit = _blockResolver = resolver + + def get: IndexShuffleBlockResolver = _blockResolver } diff --git a/core/src/main/scala/org/apache/spark/shuffle/ShufflePartitionPairsWriter.scala b/core/src/main/scala/org/apache/spark/shuffle/ShufflePartitionPairsWriter.scala index e0affb858c35..c5da2c8b2fc5 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/ShufflePartitionPairsWriter.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/ShufflePartitionPairsWriter.scala @@ -18,7 +18,9 @@ package org.apache.spark.shuffle import java.io.{Closeable, IOException, OutputStream} +import java.util.zip.Checksum +import org.apache.spark.io.MutableCheckedOutputStream import org.apache.spark.serializer.{SerializationStream, SerializerInstance, SerializerManager} import org.apache.spark.shuffle.api.ShufflePartitionWriter import org.apache.spark.storage.{BlockId, TimeTrackingOutputStream} @@ -45,6 +47,23 @@ private[spark] class ShufflePartitionPairsWriter( private var numRecordsWritten = 0 private var curNumBytesWritten = 0L + // checksum related + private var checksumEnabled = false + private var checksumOutputStream: MutableCheckedOutputStream = _ + private var checksum: Checksum = _ + + /** + * Set the checksum that the checksumOutputStream should use + */ + def setChecksum(checksum: Checksum): Unit = { + if (checksumOutputStream == null) { + this.checksumEnabled = true + this.checksum = checksum + } else { + checksumOutputStream.setChecksum(checksum) + } + } + override def write(key: Any, value: Any): Unit = { if (isClosed) { throw new IOException("Partition pairs writer is already closed.") @@ -61,7 +80,13 @@ private[spark] class ShufflePartitionPairsWriter( try { partitionStream = partitionWriter.openStream timeTrackingStream = new TimeTrackingOutputStream(writeMetrics, partitionStream) - wrappedStream = serializerManager.wrapStream(blockId, timeTrackingStream) + if (checksumEnabled) { + assert(this.checksum != null, "Checksum is not set") + checksumOutputStream = new MutableCheckedOutputStream(timeTrackingStream) + checksumOutputStream.setChecksum(checksum) + } + wrappedStream = serializerManager.wrapStream(blockId, + if (checksumEnabled) checksumOutputStream else timeTrackingStream) objOut = serializerInstance.serializeStream(wrappedStream) } catch { case e: Exception => diff --git a/core/src/main/scala/org/apache/spark/storage/BlockId.scala b/core/src/main/scala/org/apache/spark/storage/BlockId.scala index dc70a9af7e9c..806ce966181f 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockId.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockId.scala @@ -92,6 +92,11 @@ case class ShuffleIndexBlockId(shuffleId: Int, mapId: Long, reduceId: Int) exten override def name: String = "shuffle_" + shuffleId + "_" + mapId + "_" + reduceId + ".index" } +@DeveloperApi +case class ShuffleChecksumBlockId(shuffleId: Int, mapId: Long, reduceId: Int) extends BlockId { + override def name: String = "shuffle_" + shuffleId + "_" + mapId + "_" + reduceId + ".checksum" +} + @Since("3.2.0") @DeveloperApi case class ShufflePushBlockId(shuffleId: Int, mapIndex: Int, reduceId: Int) extends BlockId { 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 e55c09274cd9..f5d8c0219dc8 100644 --- a/core/src/main/scala/org/apache/spark/storage/DiskBlockObjectWriter.scala +++ b/core/src/main/scala/org/apache/spark/storage/DiskBlockObjectWriter.scala @@ -19,8 +19,10 @@ package org.apache.spark.storage import java.io.{BufferedOutputStream, File, FileOutputStream, OutputStream} import java.nio.channels.{ClosedByInterruptException, FileChannel} +import java.util.zip.Checksum import org.apache.spark.internal.Logging +import org.apache.spark.io.MutableCheckedOutputStream import org.apache.spark.serializer.{SerializationStream, SerializerInstance, SerializerManager} import org.apache.spark.shuffle.ShuffleWriteMetricsReporter import org.apache.spark.util.Utils @@ -77,6 +79,11 @@ private[spark] class DiskBlockObjectWriter( private var streamOpen = false private var hasBeenClosed = false + // checksum related + private var checksumEnabled = false + private var checksumOutputStream: MutableCheckedOutputStream = _ + private var checksum: Checksum = _ + /** * Cursors used to represent positions in the file. * @@ -101,12 +108,30 @@ private[spark] class DiskBlockObjectWriter( */ private var numRecordsWritten = 0 + /** + * Set the checksum that the checksumOutputStream should use + */ + def setChecksum(checksum: Checksum): Unit = { + if (checksumOutputStream == null) { + this.checksumEnabled = true + this.checksum = checksum + } else { + checksumOutputStream.setChecksum(checksum) + } + } + private def initialize(): Unit = { fos = new FileOutputStream(file, true) channel = fos.getChannel() ts = new TimeTrackingOutputStream(writeMetrics, fos) + if (checksumEnabled) { + assert(this.checksum != null, "Checksum is not set") + checksumOutputStream = new MutableCheckedOutputStream(ts) + checksumOutputStream.setChecksum(checksum) + } class ManualCloseBufferedOutputStream - extends BufferedOutputStream(ts, bufferSize) with ManualCloseOutputStream + extends BufferedOutputStream(if (checksumEnabled) checksumOutputStream else ts, bufferSize) + with ManualCloseOutputStream mcs = new ManualCloseBufferedOutputStream } 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 1913637371e3..29c3d0c2d471 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 @@ -19,6 +19,8 @@ package org.apache.spark.util.collection import java.io._ import java.util.Comparator +import java.util.zip.Adler32 +import java.util.zip.Checksum import scala.collection.mutable import scala.collection.mutable.ArrayBuffer @@ -29,7 +31,7 @@ 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.ShufflePartitionPairsWriter +import org.apache.spark.shuffle.{IndexShuffleBlockResolver, ShufflePartitionPairsWriter} import org.apache.spark.shuffle.api.{ShuffleMapOutputWriter, ShufflePartitionWriter} import org.apache.spark.storage.{BlockId, DiskBlockObjectWriter, ShuffleBlockId} import org.apache.spark.util.{CompletionIterator, Utils => TryUtils} @@ -141,6 +143,13 @@ private[spark] class ExternalSorter[K, V, C]( private val forceSpillFiles = new ArrayBuffer[SpilledFile] @volatile private var readingIterator: SpillableIterator = null + private val checksumEnabled = conf.get(config.SHUFFLE_CHECKSUM) + private val partitionChecksums = if (checksumEnabled) { + Array.fill[Checksum](numPartitions)(new Adler32()) + } else { + Array.empty + } + // A comparator for keys K that orders them within a partition to allow aggregation or sorting. // Can be a partial ordering by hash code if a total ordering is not provided through by the // user. (A partial ordering means that equal keys have comparator.compare(k, k) = 0, but some @@ -763,6 +772,7 @@ private[spark] class ExternalSorter[K, V, C]( serInstance, blockId, context.taskMetrics().shuffleWriteMetrics) + partitionPairsWriter.setChecksum(partitionChecksums(partitionId)) while (it.hasNext && it.nextPartition() == partitionId) { it.writeNext(partitionPairsWriter) } @@ -787,6 +797,7 @@ private[spark] class ExternalSorter[K, V, C]( serInstance, blockId, context.taskMetrics().shuffleWriteMetrics) + partitionPairsWriter.setChecksum(partitionChecksums(id)) if (elements.hasNext) { for (elem <- elements) { partitionPairsWriter.write(elem._1, elem._2) @@ -801,6 +812,11 @@ private[spark] class ExternalSorter[K, V, C]( } } + if (checksumEnabled) { + IndexShuffleBlockResolver.get.writeChecksumFile( + shuffleId, mapId, partitionChecksums.map(_.getValue)) + } + context.taskMetrics().incMemoryBytesSpilled(memoryBytesSpilled) context.taskMetrics().incDiskBytesSpilled(diskBytesSpilled) context.taskMetrics().incPeakExecutionMemory(peakMemoryUsedBytes) diff --git a/core/src/test/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriterSuite.java b/core/src/test/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriterSuite.java index 5666bb3e5f14..eadbead6738e 100644 --- a/core/src/test/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriterSuite.java +++ b/core/src/test/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriterSuite.java @@ -21,12 +21,11 @@ import java.nio.ByteBuffer; import java.nio.file.Files; import java.util.*; +import java.util.zip.Adler32; +import java.util.zip.CheckedInputStream; import org.mockito.stubbing.Answer; -import scala.Option; -import scala.Product2; -import scala.Tuple2; -import scala.Tuple2$; +import scala.*; import scala.collection.Iterator; import com.google.common.collect.HashMultiset; @@ -134,6 +133,9 @@ public void setUp() throws Exception { ); }); + IndexShuffleBlockResolver.set(shuffleBlockResolver); + doNothing().when(shuffleBlockResolver) + .writeChecksumFile(anyInt(), anyLong(), any(long[].class)); when(shuffleBlockResolver.getDataFile(anyInt(), anyLong())).thenReturn(mergedOutputFile); Answer renameTempAnswer = invocationOnMock -> { @@ -172,6 +174,12 @@ public void setUp() throws Exception { } private UnsafeShuffleWriter createWriter(boolean transferToEnabled) { + return createWriter(transferToEnabled, shuffleBlockResolver); + } + + private UnsafeShuffleWriter createWriter( + boolean transferToEnabled, + IndexShuffleBlockResolver blockResolver) { conf.set("spark.file.transferTo", String.valueOf(transferToEnabled)); return new UnsafeShuffleWriter<>( blockManager, @@ -181,7 +189,7 @@ private UnsafeShuffleWriter createWriter(boolean transferToEnabl taskContext, conf, taskContext.taskMetrics().shuffleWriteMetrics(), - new LocalDiskShuffleExecutorComponents(conf, blockManager, shuffleBlockResolver)); + new LocalDiskShuffleExecutorComponents(conf, blockManager, blockResolver)); } private void assertSpillFilesWereCleanedUp() { @@ -291,6 +299,117 @@ public void writeWithoutSpilling() throws Exception { assertEquals(mergedOutputFile.length(), shuffleWriteMetrics.bytesWritten()); } + @Test + public void writeChecksumFileWithoutSpill() throws Exception { + IndexShuffleBlockResolver blockResolver = new IndexShuffleBlockResolver(conf, blockManager); + IndexShuffleBlockResolver.set(blockResolver); + File checksumFile = new File(tempDir, "checksum"); + File dataFile = new File(tempDir, "data"); + File indexFile = new File(tempDir, "index"); + when(diskBlockManager.getFile(new ShuffleChecksumBlockId(shuffleDep.shuffleId(), 0, 0))) + .thenReturn(checksumFile); + when(diskBlockManager.getFile(new ShuffleDataBlockId(shuffleDep.shuffleId(), 0, 0))) + .thenReturn(dataFile); + when(diskBlockManager.getFile(new ShuffleIndexBlockId(shuffleDep.shuffleId(), 0, 0))) + .thenReturn(indexFile); + + // In this example, each partition should have exactly one record: + final ArrayList> dataToWrite = new ArrayList<>(); + for (int i = 0; i < NUM_PARTITIONS; i ++) { + dataToWrite.add(new Tuple2<>(i, i)); + } + final UnsafeShuffleWriter writer1 = createWriter(true, blockResolver); + writer1.write(dataToWrite.iterator()); + writer1.stop(true); + assertTrue(checksumFile.exists()); + assertEquals(checksumFile.length(), 8 * NUM_PARTITIONS); + long[] expectChecksums = new long[NUM_PARTITIONS]; + DataInputStream in1 = new DataInputStream(new FileInputStream(checksumFile)); + for (int i = 0; i < NUM_PARTITIONS; i ++) { + expectChecksums[i] = in1.readLong(); + } + in1.close(); + checksumFile.delete(); + + assertTrue(dataFile.exists()); + FileInputStream dataIn = new FileInputStream(dataFile); + assertTrue(indexFile.exists()); + DataInputStream indexIn = new DataInputStream(new FileInputStream(indexFile)); + CheckedInputStream checkedIn = null; + long prevOffset = indexIn.readLong(); + for (int i = 0; i < NUM_PARTITIONS; i ++) { + long curOffset = indexIn.readLong(); + int limit = (int) (curOffset - prevOffset); + byte[] bytes = new byte[limit]; + checkedIn = new CheckedInputStream( + new LimitedInputStream(dataIn, curOffset - prevOffset), new Adler32()); + checkedIn.read(bytes, 0, limit); + prevOffset = curOffset; + // checksum must be consistent at both write and read sides + assertEquals(checkedIn.getChecksum().getValue(), expectChecksums[i]); + } + dataIn.close(); + indexIn.close(); + checkedIn.close(); + } + + @Test + public void writeChecksumFileWithSpill() throws Exception { + IndexShuffleBlockResolver blockResolver = new IndexShuffleBlockResolver(conf, blockManager); + IndexShuffleBlockResolver.set(blockResolver); + File checksumFile = new File(tempDir, "checksum"); + File dataFile = new File(tempDir, "data"); + File indexFile = new File(tempDir, "index"); + when(diskBlockManager.getFile((BlockId) any())).thenReturn(checksumFile); + when(diskBlockManager.getFile(new ShuffleDataBlockId(shuffleDep.shuffleId(), 0, 0))) + .thenReturn(dataFile); + when(diskBlockManager.getFile(new ShuffleIndexBlockId(shuffleDep.shuffleId(), 0, 0))) + .thenReturn(indexFile); + + final UnsafeShuffleWriter writer1 = createWriter(true, blockResolver); + writer1.insertRecordIntoSorter(new Tuple2<>(0, 0)); + writer1.forceSorterToSpill(); + writer1.insertRecordIntoSorter(new Tuple2<>(1, 0)); + writer1.insertRecordIntoSorter(new Tuple2<>(2, 0)); + writer1.forceSorterToSpill(); + writer1.insertRecordIntoSorter(new Tuple2<>(0, 1)); + writer1.insertRecordIntoSorter(new Tuple2<>(3, 0)); + writer1.forceSorterToSpill(); + writer1.insertRecordIntoSorter(new Tuple2<>(1, 1)); + writer1.forceSorterToSpill(); + writer1.insertRecordIntoSorter(new Tuple2<>(0, 2)); + writer1.forceSorterToSpill(); + writer1.closeAndWriteOutput(); + assertTrue(checksumFile.exists()); + assertEquals(checksumFile.length(), 8 * NUM_PARTITIONS); + DataInputStream in1 = new DataInputStream(new FileInputStream(checksumFile)); + long[] expectChecksums = new long[NUM_PARTITIONS]; + for (int i = 0; i < NUM_PARTITIONS; i ++) { + expectChecksums[i] = in1.readLong(); + } + in1.close(); + assertTrue(dataFile.exists()); + FileInputStream dataIn = new FileInputStream(dataFile); + assertTrue(indexFile.exists()); + DataInputStream indexIn = new DataInputStream(new FileInputStream(indexFile)); + CheckedInputStream checkedIn = null; + long prevOffset = indexIn.readLong(); + for (int i = 0; i < NUM_PARTITIONS; i ++) { + long curOffset = indexIn.readLong(); + int limit = (int) (curOffset - prevOffset); + byte[] bytes = new byte[limit]; + checkedIn = new CheckedInputStream( + new LimitedInputStream(dataIn, curOffset - prevOffset), new Adler32()); + checkedIn.read(bytes, 0, limit); + prevOffset = curOffset; + // checksum must be consistent at both write and read sides + assertEquals(checkedIn.getChecksum().getValue(), expectChecksums[i]); + } + dataIn.close(); + indexIn.close(); + checkedIn.close(); + } + private void testMergingSpills( final boolean transferToEnabled, String compressionCodecName, diff --git a/core/src/test/scala/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriterSuite.scala b/core/src/test/scala/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriterSuite.scala index 7fd0bf626fda..0320679a4006 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriterSuite.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriterSuite.scala @@ -17,8 +17,9 @@ package org.apache.spark.shuffle.sort -import java.io.File +import java.io.{DataInputStream, File, FileInputStream} import java.util.UUID +import java.util.zip.{Adler32, CheckedInputStream} import scala.collection.mutable import scala.collection.mutable.ArrayBuffer @@ -32,6 +33,7 @@ import org.scalatest.BeforeAndAfterEach import org.apache.spark._ import org.apache.spark.executor.{ShuffleWriteMetrics, TaskMetrics} import org.apache.spark.memory.{TaskMemoryManager, TestMemoryManager} +import org.apache.spark.network.util.LimitedInputStream import org.apache.spark.serializer.{JavaSerializer, SerializerInstance, SerializerManager} import org.apache.spark.shuffle.IndexShuffleBlockResolver import org.apache.spark.shuffle.api.ShuffleExecutorComponents @@ -76,6 +78,7 @@ class BypassMergeSortShuffleWriterSuite extends SparkFunSuite with BeforeAndAfte when(blockManager.diskBlockManager).thenReturn(diskBlockManager) when(taskContext.taskMemoryManager()).thenReturn(taskMemoryManager) + IndexShuffleBlockResolver.set(blockResolver) when(blockResolver.writeIndexFileAndCommit( anyInt, anyLong, any(classOf[Array[Long]]), any(classOf[File]))) .thenAnswer { invocationOnMock => @@ -86,6 +89,7 @@ class BypassMergeSortShuffleWriterSuite extends SparkFunSuite with BeforeAndAfte } null } + doNothing().when(blockResolver).writeChecksumFile(any(), any(), any(classOf[Array[Long]])) when(blockManager.getDiskWriter( any[BlockId], @@ -236,4 +240,68 @@ class BypassMergeSortShuffleWriterSuite extends SparkFunSuite with BeforeAndAfte writer.stop( /* success = */ false) assert(temporaryFilesCreated.count(_.exists()) === 0) } + + test("write checksum file") { + val blockResolver = new IndexShuffleBlockResolver(conf, blockManager) + IndexShuffleBlockResolver.set(blockResolver) + val shuffleId = shuffleHandle.shuffleId + val mapId = 0 + val checksumBlockId = ShuffleChecksumBlockId(shuffleId, mapId, 0) + val dataBlockId = ShuffleDataBlockId(shuffleId, mapId, 0) + val indexBlockId = ShuffleIndexBlockId(shuffleId, mapId, 0) + val checksumFile = new File(tempDir, checksumBlockId.name) + val dataFile = new File(tempDir, dataBlockId.name) + val indexFile = new File(tempDir, indexBlockId.name) + reset(diskBlockManager) + when(diskBlockManager.getFile(checksumBlockId)).thenAnswer(_ => checksumFile) + when(diskBlockManager.getFile(dataBlockId)).thenAnswer(_ => dataFile) + when(diskBlockManager.getFile(indexBlockId)).thenAnswer(_ => indexFile) + when(diskBlockManager.createTempShuffleBlock()) + .thenAnswer { _ => + val blockId = new TempShuffleBlockId(UUID.randomUUID) + val file = new File(tempDir, blockId.name) + temporaryFilesCreated += file + (blockId, file) + } + + val numPartition = shuffleHandle.dependency.partitioner.numPartitions + val writer = new BypassMergeSortShuffleWriter[Int, Int]( + blockManager, + shuffleHandle, + mapId, + conf, + taskContext.taskMetrics().shuffleWriteMetrics, + new LocalDiskShuffleExecutorComponents(conf, blockManager, blockResolver)) + + writer.write(Iterator((0, 0), (1, 1), (2, 2), (3, 3), (4, 4), (5, 5), (6, 6))) + writer.stop( /* success = */ true) + assert(checksumFile.exists()) + assert(checksumFile.length() === 8 * numPartition) + + val in = new DataInputStream(new FileInputStream(checksumFile)) + val expectChecksums = Array.ofDim[Long](numPartition) + (0 until numPartition).foreach(i => expectChecksums(i) = in.readLong()) + in.close() + + assert(dataFile.exists) + val dataIn = new FileInputStream(dataFile) + assert(indexFile.exists) + val indexIn = new DataInputStream(new FileInputStream(indexFile)) + var checkedIn: CheckedInputStream = null + var prevOffset = indexIn.readLong + (0 until numPartition).foreach { i => + val curOffset = indexIn.readLong + val limit = (curOffset - prevOffset).toInt + val bytes = new Array[Byte](limit) + checkedIn = new CheckedInputStream( + new LimitedInputStream(dataIn, curOffset - prevOffset), new Adler32) + checkedIn.read(bytes, 0, limit) + prevOffset = curOffset + // checksum must be consistent at both write and read sides + assert(checkedIn.getChecksum.getValue === expectChecksums(i)) + } + dataIn.close() + indexIn.close() + checkedIn.close() + } } diff --git a/core/src/test/scala/org/apache/spark/shuffle/sort/IndexShuffleBlockResolverSuite.scala b/core/src/test/scala/org/apache/spark/shuffle/sort/IndexShuffleBlockResolverSuite.scala index 5955d442a77f..180b8f37df40 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/sort/IndexShuffleBlockResolverSuite.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/sort/IndexShuffleBlockResolverSuite.scala @@ -248,4 +248,14 @@ class IndexShuffleBlockResolverSuite extends SparkFunSuite with BeforeAndAfterEa outIndex.close() } } + + test("write checksum file") { + val resolver = new IndexShuffleBlockResolver(conf, blockManager) + val checksumsInMemory = Array[Long](0, 1, 2, 3, 4, 5, 6, 7, 8, 9) + resolver.writeChecksumFile(0, 0, checksumsInMemory) + val checksumFile = resolver.getChecksumFile(0, 0) + assert(checksumFile.exists()) + val checksumsFromFile = resolver.getChecksums(checksumFile, 10) + assert(checksumsInMemory === checksumsFromFile) + } } diff --git a/core/src/test/scala/org/apache/spark/shuffle/sort/ShuffleExternalSorterSuite.scala b/core/src/test/scala/org/apache/spark/shuffle/sort/ShuffleExternalSorterSuite.scala index b33708d24e7c..8ae912de6ae1 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/sort/ShuffleExternalSorterSuite.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/sort/ShuffleExternalSorterSuite.scala @@ -74,6 +74,8 @@ class ShuffleExternalSorterSuite extends SparkFunSuite with LocalSparkContext wi sc.env.blockManager, taskContext, 100, // initialSize - This will require ShuffleInMemorySorter to acquire at least 800 bytes + 0, + 0, 1, // numPartitions conf, new ShuffleWriteMetrics) 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 index 4c679fd874c9..807e8f8d212c 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/sort/SortShuffleWriterSuite.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/sort/SortShuffleWriterSuite.scala @@ -17,22 +17,33 @@ package org.apache.spark.shuffle.sort +import java.io.{DataInputStream, FileInputStream} +import java.util.zip.{Adler32, CheckedInputStream} + import org.mockito.{Mock, MockitoAnnotations} import org.mockito.Answers.RETURNS_SMART_NULLS import org.mockito.Mockito._ +import org.scalatest.PrivateMethodTester import org.scalatest.matchers.must.Matchers -import org.apache.spark.{Partitioner, SharedSparkContext, ShuffleDependency, SparkFunSuite} +import org.apache.spark.{Aggregator, DebugFilesystem, Partitioner, SharedSparkContext, ShuffleDependency, SparkContext, SparkFunSuite} import org.apache.spark.memory.MemoryTestingUtils +import org.apache.spark.network.util.LimitedInputStream 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 +import org.apache.spark.util.collection.ExternalSorter + -class SortShuffleWriterSuite extends SparkFunSuite with SharedSparkContext with Matchers { +class SortShuffleWriterSuite + extends SparkFunSuite + with SharedSparkContext + with Matchers + with PrivateMethodTester { @Mock(answer = RETURNS_SMART_NULLS) private var blockManager: BlockManager = _ @@ -44,13 +55,14 @@ class SortShuffleWriterSuite extends SparkFunSuite with SharedSparkContext with private val serializer = new JavaSerializer(conf) private var shuffleExecutorComponents: ShuffleExecutorComponents = _ + private val partitioner = new Partitioner() { + def numPartitions = numMaps + def getPartition(key: Any) = Utils.nonNegativeMod(key.hashCode, numPartitions) + } + override def beforeEach(): Unit = { super.beforeEach() MockitoAnnotations.openMocks(this).close() - 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) @@ -61,6 +73,7 @@ class SortShuffleWriterSuite extends SparkFunSuite with SharedSparkContext with } shuffleExecutorComponents = new LocalDiskShuffleExecutorComponents( conf, blockManager, shuffleBlockResolver) + IndexShuffleBlockResolver.set(shuffleBlockResolver) } override def afterAll(): Unit = { @@ -103,4 +116,95 @@ class SortShuffleWriterSuite extends SparkFunSuite with SharedSparkContext with assert(dataFile.length() === writeMetrics.bytesWritten) assert(records.size === writeMetrics.recordsWritten) } + + Seq((true, false, false), + (true, true, false), + (true, false, true), + (true, true, true), + (false, false, false), + (false, true, false), + (false, false, true), + (false, true, true)).foreach { case (doSpill, doAgg, doOrder) => + test(s"write checksum file (spill=$doSpill, aggregator=$doAgg, order=$doOrder)") { + val aggregator = if (doAgg) { + Some(Aggregator[Int, Int, Int]( + v => v, + (c, v) => c + v, + (c1, c2) => c1 + c2)) + } else None + val order = if (doOrder) { + Some(new Ordering[Int] { + override def compare(x: Int, y: Int): Int = x - y + }) + } else None + + val shuffleHandle = { + val dependency = mock(classOf[ShuffleDependency[Int, Int, Int]]) + when(dependency.partitioner).thenReturn(partitioner) + when(dependency.serializer).thenReturn(serializer) + when(dependency.aggregator).thenReturn(aggregator) + when(dependency.keyOrdering).thenReturn(order) + new BaseShuffleHandle[Int, Int, Int](shuffleId, dependency) + } + + // FIXME: this can affect other tests (if any) after this set of tests + // since `sc` is global. + sc.stop() + conf.set("spark.shuffle.spill.numElementsForceSpillThreshold", + if (doSpill) "0" else Int.MaxValue.toString) + conf.set("spark.hadoop.fs.file.impl", classOf[DebugFilesystem].getName) + val localSC = new SparkContext("local[4]", "test", conf) + val shuffleBlockResolver = new IndexShuffleBlockResolver(conf) + IndexShuffleBlockResolver.set(shuffleBlockResolver) + val context = MemoryTestingUtils.fakeTaskContext(localSC.env) + val records = List[(Int, Int)]( + (0, 1), (1, 2), (0, 2), (1, 3), (2, 3), (3, 4), (4, 5), (3, 5), (4, 6)) + val numPartition = shuffleHandle.dependency.partitioner.numPartitions + val writer = new SortShuffleWriter[Int, Int, Int]( + shuffleHandle, + mapId = 0, + context, + new LocalDiskShuffleExecutorComponents( + conf, shuffleBlockResolver._blockManager, shuffleBlockResolver)) + writer.write(records.toIterator) + val sorterMethod = PrivateMethod[ExternalSorter[_, _, _]](Symbol("sorter")) + val sorter = writer.invokePrivate(sorterMethod()) + val expectSpillSize = if (doSpill) records.size else 0 + assert(sorter.numSpills === expectSpillSize) + writer.stop(success = true) + val checksumFile = shuffleBlockResolver.getChecksumFile(shuffleId, 0) + assert(checksumFile.exists()) + assert(checksumFile.length() === 8 * numPartition) + + val in = new DataInputStream(new FileInputStream(checksumFile)) + val expectChecksums = Array.ofDim[Long](numPartition) + (0 until numPartition).foreach(i => expectChecksums(i) = in.readLong()) + in.close() + + val dataFile = shuffleBlockResolver.getDataFile(shuffleId, 0) + assert(dataFile.exists) + val dataIn = new FileInputStream(dataFile) + val indexFile = shuffleBlockResolver.getIndexFile(shuffleId, 0) + assert(indexFile.exists) + val indexIn = new DataInputStream(new FileInputStream(indexFile)) + var checkedIn: CheckedInputStream = null + var prevOffset = indexIn.readLong + (0 until numPartition).foreach { i => + val curOffset = indexIn.readLong + val limit = (curOffset - prevOffset).toInt + val bytes = new Array[Byte](limit) + checkedIn = new CheckedInputStream( + new LimitedInputStream(dataIn, curOffset - prevOffset), new Adler32) + checkedIn.read(bytes, 0, limit) + prevOffset = curOffset + // checksum must be consistent at both write and read sides + assert(checkedIn.getChecksum.getValue === expectChecksums(i)) + } + dataIn.close() + indexIn.close() + checkedIn.close() + localSC.stop() + } + } + } From 7a70312d8951832ca22edbd5f46ac38f81e72ddc Mon Sep 17 00:00:00 2001 From: "yi.wu" Date: Wed, 9 Jun 2021 17:59:29 +0800 Subject: [PATCH 02/47] update test --- .../shuffle/IndexShuffleBlockResolver.scala | 7 ++ .../sort/UnsafeShuffleWriterSuite.java | 61 ++-------------- .../spark/shuffle/ShuffleChecksumTester.scala | 72 +++++++++++++++++++ .../BypassMergeSortShuffleWriterSuite.scala | 38 ++-------- .../shuffle/sort/SortShuffleWriterSuite.scala | 37 ++-------- .../util/collection/ExternalSorterSuite.scala | 14 ++-- 6 files changed, 101 insertions(+), 128 deletions(-) create mode 100644 core/src/test/scala/org/apache/spark/shuffle/ShuffleChecksumTester.scala diff --git a/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala b/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala index 5c788f47fc69..712f5b90e25d 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala @@ -156,6 +156,13 @@ private[spark] class IndexShuffleBlockResolver( logWarning(s"Error deleting index ${file.getPath()}") } } + + file = getChecksumFile(shuffleId, mapId) + if (file.exists()) { + if (!file.delete()) { + logWarning(s"Error deleting checksum ${file.getPath()}") + } + } } /** diff --git a/core/src/test/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriterSuite.java b/core/src/test/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriterSuite.java index eadbead6738e..4fa30a4fc5c9 100644 --- a/core/src/test/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriterSuite.java +++ b/core/src/test/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriterSuite.java @@ -21,9 +21,8 @@ import java.nio.ByteBuffer; import java.nio.file.Files; import java.util.*; -import java.util.zip.Adler32; -import java.util.zip.CheckedInputStream; +import org.apache.spark.shuffle.ShuffleChecksumTester; import org.mockito.stubbing.Answer; import scala.*; import scala.collection.Iterator; @@ -64,7 +63,7 @@ import static org.mockito.Answers.RETURNS_SMART_NULLS; import static org.mockito.Mockito.*; -public class UnsafeShuffleWriterSuite { +public class UnsafeShuffleWriterSuite implements ShuffleChecksumTester { static final int DEFAULT_INITIAL_SORT_BUFFER_SIZE = 4096; static final int NUM_PARTITIONS = 4; @@ -323,34 +322,7 @@ public void writeChecksumFileWithoutSpill() throws Exception { writer1.stop(true); assertTrue(checksumFile.exists()); assertEquals(checksumFile.length(), 8 * NUM_PARTITIONS); - long[] expectChecksums = new long[NUM_PARTITIONS]; - DataInputStream in1 = new DataInputStream(new FileInputStream(checksumFile)); - for (int i = 0; i < NUM_PARTITIONS; i ++) { - expectChecksums[i] = in1.readLong(); - } - in1.close(); - checksumFile.delete(); - - assertTrue(dataFile.exists()); - FileInputStream dataIn = new FileInputStream(dataFile); - assertTrue(indexFile.exists()); - DataInputStream indexIn = new DataInputStream(new FileInputStream(indexFile)); - CheckedInputStream checkedIn = null; - long prevOffset = indexIn.readLong(); - for (int i = 0; i < NUM_PARTITIONS; i ++) { - long curOffset = indexIn.readLong(); - int limit = (int) (curOffset - prevOffset); - byte[] bytes = new byte[limit]; - checkedIn = new CheckedInputStream( - new LimitedInputStream(dataIn, curOffset - prevOffset), new Adler32()); - checkedIn.read(bytes, 0, limit); - prevOffset = curOffset; - // checksum must be consistent at both write and read sides - assertEquals(checkedIn.getChecksum().getValue(), expectChecksums[i]); - } - dataIn.close(); - indexIn.close(); - checkedIn.close(); + compareChecksums(NUM_PARTITIONS, checksumFile, dataFile, indexFile); } @Test @@ -382,32 +354,7 @@ public void writeChecksumFileWithSpill() throws Exception { writer1.closeAndWriteOutput(); assertTrue(checksumFile.exists()); assertEquals(checksumFile.length(), 8 * NUM_PARTITIONS); - DataInputStream in1 = new DataInputStream(new FileInputStream(checksumFile)); - long[] expectChecksums = new long[NUM_PARTITIONS]; - for (int i = 0; i < NUM_PARTITIONS; i ++) { - expectChecksums[i] = in1.readLong(); - } - in1.close(); - assertTrue(dataFile.exists()); - FileInputStream dataIn = new FileInputStream(dataFile); - assertTrue(indexFile.exists()); - DataInputStream indexIn = new DataInputStream(new FileInputStream(indexFile)); - CheckedInputStream checkedIn = null; - long prevOffset = indexIn.readLong(); - for (int i = 0; i < NUM_PARTITIONS; i ++) { - long curOffset = indexIn.readLong(); - int limit = (int) (curOffset - prevOffset); - byte[] bytes = new byte[limit]; - checkedIn = new CheckedInputStream( - new LimitedInputStream(dataIn, curOffset - prevOffset), new Adler32()); - checkedIn.read(bytes, 0, limit); - prevOffset = curOffset; - // checksum must be consistent at both write and read sides - assertEquals(checkedIn.getChecksum().getValue(), expectChecksums[i]); - } - dataIn.close(); - indexIn.close(); - checkedIn.close(); + compareChecksums(NUM_PARTITIONS, checksumFile, dataFile, indexFile); } private void testMergingSpills( diff --git a/core/src/test/scala/org/apache/spark/shuffle/ShuffleChecksumTester.scala b/core/src/test/scala/org/apache/spark/shuffle/ShuffleChecksumTester.scala new file mode 100644 index 000000000000..a9fa77b9dee2 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/shuffle/ShuffleChecksumTester.scala @@ -0,0 +1,72 @@ +/* + * 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.{DataInputStream, File, FileInputStream} +import java.util.zip.{Adler32, CheckedInputStream} + +import org.apache.spark.network.util.LimitedInputStream + +trait ShuffleChecksumTester { + def compareChecksums(numPartition: Int, checksum: File, data: File, index: File): Unit = { + assert(checksum.exists(), "Checksum file doesn't exist") + assert(data.exists(), "Data file doesn't exist") + assert(index.exists(), "Index file doesn't exist") + + var checksumIn: DataInputStream = null + val expectChecksums = Array.ofDim[Long](numPartition) + try { + checksumIn = new DataInputStream(new FileInputStream(checksum)) + (0 until numPartition).foreach(i => expectChecksums(i) = checksumIn.readLong()) + } finally { + if (checksumIn != null) { + checksumIn.close() + } + } + + var dataIn: FileInputStream = null + var indexIn: DataInputStream = null + var checkedIn: CheckedInputStream = null + try { + dataIn = new FileInputStream(data) + indexIn = new DataInputStream(new FileInputStream(index)) + var prevOffset = indexIn.readLong + (0 until numPartition).foreach { i => + val curOffset = indexIn.readLong + val limit = (curOffset - prevOffset).toInt + val bytes = new Array[Byte](limit) + checkedIn = new CheckedInputStream( + new LimitedInputStream(dataIn, curOffset - prevOffset), new Adler32) + checkedIn.read(bytes, 0, limit) + prevOffset = curOffset + // checksum must be consistent at both write and read sides + assert(checkedIn.getChecksum.getValue == expectChecksums(i)) + } + } finally { + if (dataIn != null) { + dataIn.close() + } + if (indexIn != null) { + indexIn.close() + } + if (checkedIn != null) { + checkedIn.close() + } + } + } +} diff --git a/core/src/test/scala/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriterSuite.scala b/core/src/test/scala/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriterSuite.scala index 0320679a4006..91e599e67727 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriterSuite.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriterSuite.scala @@ -17,9 +17,8 @@ package org.apache.spark.shuffle.sort -import java.io.{DataInputStream, File, FileInputStream} +import java.io.File import java.util.UUID -import java.util.zip.{Adler32, CheckedInputStream} import scala.collection.mutable import scala.collection.mutable.ArrayBuffer @@ -33,15 +32,17 @@ import org.scalatest.BeforeAndAfterEach import org.apache.spark._ import org.apache.spark.executor.{ShuffleWriteMetrics, TaskMetrics} import org.apache.spark.memory.{TaskMemoryManager, TestMemoryManager} -import org.apache.spark.network.util.LimitedInputStream import org.apache.spark.serializer.{JavaSerializer, SerializerInstance, SerializerManager} -import org.apache.spark.shuffle.IndexShuffleBlockResolver +import org.apache.spark.shuffle.{IndexShuffleBlockResolver, ShuffleChecksumTester} import org.apache.spark.shuffle.api.ShuffleExecutorComponents import org.apache.spark.shuffle.sort.io.LocalDiskShuffleExecutorComponents import org.apache.spark.storage._ import org.apache.spark.util.Utils -class BypassMergeSortShuffleWriterSuite extends SparkFunSuite with BeforeAndAfterEach { +class BypassMergeSortShuffleWriterSuite + extends SparkFunSuite + with BeforeAndAfterEach + with ShuffleChecksumTester { @Mock(answer = RETURNS_SMART_NULLS) private var blockManager: BlockManager = _ @Mock(answer = RETURNS_SMART_NULLS) private var diskBlockManager: DiskBlockManager = _ @@ -277,31 +278,6 @@ class BypassMergeSortShuffleWriterSuite extends SparkFunSuite with BeforeAndAfte writer.stop( /* success = */ true) assert(checksumFile.exists()) assert(checksumFile.length() === 8 * numPartition) - - val in = new DataInputStream(new FileInputStream(checksumFile)) - val expectChecksums = Array.ofDim[Long](numPartition) - (0 until numPartition).foreach(i => expectChecksums(i) = in.readLong()) - in.close() - - assert(dataFile.exists) - val dataIn = new FileInputStream(dataFile) - assert(indexFile.exists) - val indexIn = new DataInputStream(new FileInputStream(indexFile)) - var checkedIn: CheckedInputStream = null - var prevOffset = indexIn.readLong - (0 until numPartition).foreach { i => - val curOffset = indexIn.readLong - val limit = (curOffset - prevOffset).toInt - val bytes = new Array[Byte](limit) - checkedIn = new CheckedInputStream( - new LimitedInputStream(dataIn, curOffset - prevOffset), new Adler32) - checkedIn.read(bytes, 0, limit) - prevOffset = curOffset - // checksum must be consistent at both write and read sides - assert(checkedIn.getChecksum.getValue === expectChecksums(i)) - } - dataIn.close() - indexIn.close() - checkedIn.close() + compareChecksums(numPartition, checksumFile, dataFile, indexFile) } } 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 index 807e8f8d212c..0caa5579296b 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/sort/SortShuffleWriterSuite.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/sort/SortShuffleWriterSuite.scala @@ -17,9 +17,6 @@ package org.apache.spark.shuffle.sort -import java.io.{DataInputStream, FileInputStream} -import java.util.zip.{Adler32, CheckedInputStream} - import org.mockito.{Mock, MockitoAnnotations} import org.mockito.Answers.RETURNS_SMART_NULLS import org.mockito.Mockito._ @@ -28,9 +25,8 @@ import org.scalatest.matchers.must.Matchers import org.apache.spark.{Aggregator, DebugFilesystem, Partitioner, SharedSparkContext, ShuffleDependency, SparkContext, SparkFunSuite} import org.apache.spark.memory.MemoryTestingUtils -import org.apache.spark.network.util.LimitedInputStream import org.apache.spark.serializer.JavaSerializer -import org.apache.spark.shuffle.{BaseShuffleHandle, IndexShuffleBlockResolver} +import org.apache.spark.shuffle.{BaseShuffleHandle, IndexShuffleBlockResolver, ShuffleChecksumTester} import org.apache.spark.shuffle.api.ShuffleExecutorComponents import org.apache.spark.shuffle.sort.io.LocalDiskShuffleExecutorComponents import org.apache.spark.storage.BlockManager @@ -43,7 +39,8 @@ class SortShuffleWriterSuite extends SparkFunSuite with SharedSparkContext with Matchers - with PrivateMethodTester { + with PrivateMethodTester + with ShuffleChecksumTester { @Mock(answer = RETURNS_SMART_NULLS) private var blockManager: BlockManager = _ @@ -175,36 +172,10 @@ class SortShuffleWriterSuite val checksumFile = shuffleBlockResolver.getChecksumFile(shuffleId, 0) assert(checksumFile.exists()) assert(checksumFile.length() === 8 * numPartition) - - val in = new DataInputStream(new FileInputStream(checksumFile)) - val expectChecksums = Array.ofDim[Long](numPartition) - (0 until numPartition).foreach(i => expectChecksums(i) = in.readLong()) - in.close() - val dataFile = shuffleBlockResolver.getDataFile(shuffleId, 0) - assert(dataFile.exists) - val dataIn = new FileInputStream(dataFile) val indexFile = shuffleBlockResolver.getIndexFile(shuffleId, 0) - assert(indexFile.exists) - val indexIn = new DataInputStream(new FileInputStream(indexFile)) - var checkedIn: CheckedInputStream = null - var prevOffset = indexIn.readLong - (0 until numPartition).foreach { i => - val curOffset = indexIn.readLong - val limit = (curOffset - prevOffset).toInt - val bytes = new Array[Byte](limit) - checkedIn = new CheckedInputStream( - new LimitedInputStream(dataIn, curOffset - prevOffset), new Adler32) - checkedIn.read(bytes, 0, limit) - prevOffset = curOffset - // checksum must be consistent at both write and read sides - assert(checkedIn.getChecksum.getValue === expectChecksums(i)) - } - dataIn.close() - indexIn.close() - checkedIn.close() + compareChecksums(numPartition, checksumFile, dataFile, indexFile) localSC.stop() } } - } diff --git a/core/src/test/scala/org/apache/spark/util/collection/ExternalSorterSuite.scala b/core/src/test/scala/org/apache/spark/util/collection/ExternalSorterSuite.scala index a6de64b6c68a..7bec96121875 100644 --- a/core/src/test/scala/org/apache/spark/util/collection/ExternalSorterSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/collection/ExternalSorterSuite.scala @@ -500,15 +500,15 @@ class ExternalSorterSuite extends SparkFunSuite with LocalSparkContext { intercept[SparkException] { data.reduceByKey(_ + _).count() } - // After the shuffle, there should be only 2 files on disk: the output of task 1 and - // its index. All other files (map 2's output and intermediate merge files) should - // have been deleted. - assert(diskBlockManager.getAllFiles().length === 2) + // After the shuffle, there should be only 3 files on disk: the output of task 1 and + // its index and checksum. All other files (map 2's output and intermediate merge files) + // should have been deleted. + assert(diskBlockManager.getAllFiles().length === 3) } else { assert(data.reduceByKey(_ + _).count() === size) - // After the shuffle, there should be only 4 files on disk: the output of both tasks - // and their indices. All intermediate merge files should have been deleted. - assert(diskBlockManager.getAllFiles().length === 4) + // After the shuffle, there should be only 6 files on disk: the output of both tasks + // and their indices/checksums. All intermediate merge files should have been deleted. + assert(diskBlockManager.getAllFiles().length === 6) } } } From 248cf2bd9e1e7e7cbc062bb6ad73b30833ad41fc Mon Sep 17 00:00:00 2001 From: "yi.wu" Date: Wed, 9 Jun 2021 18:00:58 +0800 Subject: [PATCH 03/47] add comment --- .../org/apache/spark/shuffle/ShuffleChecksumTester.scala | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/core/src/test/scala/org/apache/spark/shuffle/ShuffleChecksumTester.scala b/core/src/test/scala/org/apache/spark/shuffle/ShuffleChecksumTester.scala index a9fa77b9dee2..b3470fbdcc99 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/ShuffleChecksumTester.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/ShuffleChecksumTester.scala @@ -23,6 +23,10 @@ import java.util.zip.{Adler32, CheckedInputStream} import org.apache.spark.network.util.LimitedInputStream trait ShuffleChecksumTester { + + /** + * Ensure that the checksum values are consistent between write and read side. + */ def compareChecksums(numPartition: Int, checksum: File, data: File, index: File): Unit = { assert(checksum.exists(), "Checksum file doesn't exist") assert(data.exists(), "Data file doesn't exist") From a4326948adb7b20620fa5c74194918e3587f52c6 Mon Sep 17 00:00:00 2001 From: "yi.wu" Date: Thu, 10 Jun 2021 23:06:14 +0800 Subject: [PATCH 04/47] update commitAllPartitions to include checksums --- .../shuffle/api/ShuffleMapOutputWriter.java | 9 +- .../SingleSpillShuffleMapOutputWriter.java | 5 +- .../sort/BypassMergeSortShuffleWriter.java | 21 ++-- .../shuffle/sort/ShuffleExternalSorter.java | 16 ++- .../shuffle/sort/UnsafeShuffleWriter.java | 9 +- .../LocalDiskShuffleExecutorComponents.java | 1 - .../io/LocalDiskShuffleMapOutputWriter.java | 5 +- .../LocalDiskSingleSpillMapOutputWriter.java | 6 +- .../shuffle/IndexShuffleBlockResolver.scala | 103 ++++++++++-------- .../shuffle/sort/SortShuffleWriter.scala | 2 +- .../util/collection/ExternalSorter.scala | 15 ++- .../sort/UnsafeShuffleWriterSuite.java | 13 +-- .../BypassMergeSortShuffleWriterSuite.scala | 9 +- .../sort/IndexShuffleBlockResolverSuite.scala | 10 +- .../shuffle/sort/SortShuffleWriterSuite.scala | 4 - ...LocalDiskShuffleMapOutputWriterSuite.scala | 7 +- 16 files changed, 131 insertions(+), 104 deletions(-) diff --git a/core/src/main/java/org/apache/spark/shuffle/api/ShuffleMapOutputWriter.java b/core/src/main/java/org/apache/spark/shuffle/api/ShuffleMapOutputWriter.java index 0167002ceedb..d484af2589b1 100644 --- a/core/src/main/java/org/apache/spark/shuffle/api/ShuffleMapOutputWriter.java +++ b/core/src/main/java/org/apache/spark/shuffle/api/ShuffleMapOutputWriter.java @@ -59,6 +59,10 @@ public interface ShuffleMapOutputWriter { * available to downstream reduce tasks. If this method throws any exception, this module's * {@link #abort(Throwable)} method will be invoked before propagating the exception. *

+ * Shuffle extension who cares about the cause of shuffle data corruption should store + * the {@param checksums} properly. When corruption happens, Spark would provide the checksum + * of the fetched partition to the shuffle extension to help diagnose the cause of corruption. + *

* This can also close any resources and clean up temporary state if necessary. *

* The returned commit message is a structure with two components: @@ -68,8 +72,11 @@ public interface ShuffleMapOutputWriter { * for that partition id. *

* 2) An optional metadata blob that can be used by shuffle readers. + * + * @param checksums The checksum values for each partition if shuffle checksum enabled. + * Otherwise, it's empty. */ - MapOutputCommitMessage commitAllPartitions() throws IOException; + MapOutputCommitMessage commitAllPartitions(long[] checksums) throws IOException; /** * Abort all of the writes done by any writers returned by {@link #getPartitionWriter(int)}. diff --git a/core/src/main/java/org/apache/spark/shuffle/api/SingleSpillShuffleMapOutputWriter.java b/core/src/main/java/org/apache/spark/shuffle/api/SingleSpillShuffleMapOutputWriter.java index cad8dcfda52b..ba3d5a603e05 100644 --- a/core/src/main/java/org/apache/spark/shuffle/api/SingleSpillShuffleMapOutputWriter.java +++ b/core/src/main/java/org/apache/spark/shuffle/api/SingleSpillShuffleMapOutputWriter.java @@ -32,5 +32,8 @@ public interface SingleSpillShuffleMapOutputWriter { /** * Transfer a file that contains the bytes of all the partitions written by this map task. */ - void transferMapSpillFile(File mapOutputFile, long[] partitionLengths) throws IOException; + void transferMapSpillFile( + File mapOutputFile, + long[] partitionLengths, + long[] checksums) throws IOException; } diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java b/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java index 6d8ece9d31e8..dacf6e8006b8 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java @@ -27,7 +27,6 @@ import java.util.zip.Checksum; import javax.annotation.Nullable; -import org.apache.spark.shuffle.IndexShuffleBlockResolver; import scala.None$; import scala.Option; import scala.Product2; @@ -141,7 +140,7 @@ public void write(Iterator> records) throws IOException { .createMapOutputWriter(shuffleId, mapId, numPartitions); try { if (!records.hasNext()) { - partitionLengths = mapOutputWriter.commitAllPartitions().getPartitionLengths(); + partitionLengths = mapOutputWriter.commitAllPartitions(new long[0]).getPartitionLengths(); mapStatus = MapStatus$.MODULE$.apply( blockManager.shuffleServerId(), partitionLengths, mapId); return; @@ -180,13 +179,6 @@ public void write(Iterator> records) throws IOException { } partitionLengths = writePartitionedData(mapOutputWriter); - if (checksumEnabled) { - long[] checksums = new long[numPartitions]; - for (int i = 0; i < numPartitions; i ++) { - checksums[i] = partitionChecksums[i].getValue(); - } - IndexShuffleBlockResolver.get().writeChecksumFile(shuffleId, mapId, checksums); - } mapStatus = MapStatus$.MODULE$.apply( blockManager.shuffleServerId(), partitionLengths, mapId); } catch (Exception e) { @@ -241,7 +233,16 @@ private long[] writePartitionedData(ShuffleMapOutputWriter mapOutputWriter) thro } partitionWriters = null; } - return mapOutputWriter.commitAllPartitions().getPartitionLengths(); + long[] checksums; + if (checksumEnabled) { + checksums = new long[numPartitions]; + for (int i = 0; i < numPartitions; i ++) { + checksums[i] = partitionChecksums[i].getValue(); + } + } else { + checksums = new long[0]; + } + return mapOutputWriter.commitAllPartitions(checksums).getPartitionLengths(); } private void writePartitionedDataWithChannel( diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleExternalSorter.java b/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleExternalSorter.java index 6e92f9bff9da..2c693659ac6c 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleExternalSorter.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleExternalSorter.java @@ -24,7 +24,6 @@ import java.util.zip.Adler32; import java.util.zip.Checksum; -import org.apache.spark.shuffle.IndexShuffleBlockResolver; import scala.Tuple2; import com.google.common.annotations.VisibleForTesting; @@ -154,12 +153,17 @@ final class ShuffleExternalSorter extends MemoryConsumer { } } - public void writeChecksumFile() { - long[] checksums = new long[numPartitions]; - for (int i = 0; i < numPartitions; i ++) { - checksums[i] = partitionChecksums[i].getValue(); + public long[] getChecksums() { + long[] checksums; + if (checksumEnabled) { + checksums = new long[numPartitions]; + for (int i = 0; i < numPartitions; i ++) { + checksums[i] = partitionChecksums[i].getValue(); + } + } else { + checksums = new long[0]; } - IndexShuffleBlockResolver.get().writeChecksumFile(shuffleId, mapId, checksums); + return checksums; } /** diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java b/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java index 1ff5ea46b745..cd269d79866f 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java @@ -223,7 +223,7 @@ void closeAndWriteOutput() throws IOException { final SpillInfo[] spills = sorter.closeAndGetSpills(); try { partitionLengths = mergeSpills(spills); - sorter.writeChecksumFile(); + sorter.getChecksums(); sorter = null; } finally { for (SpillInfo spill : spills) { @@ -270,7 +270,7 @@ private long[] mergeSpills(SpillInfo[] spills) throws IOException { if (spills.length == 0) { final ShuffleMapOutputWriter mapWriter = shuffleExecutorComponents .createMapOutputWriter(shuffleId, mapId, partitioner.numPartitions()); - return mapWriter.commitAllPartitions().getPartitionLengths(); + return mapWriter.commitAllPartitions(new long[0]).getPartitionLengths(); } else if (spills.length == 1) { Optional maybeSingleFileWriter = shuffleExecutorComponents.createSingleFileMapOutputWriter(shuffleId, mapId); @@ -280,7 +280,8 @@ private long[] mergeSpills(SpillInfo[] spills) throws IOException { partitionLengths = spills[0].partitionLengths; logger.debug("Merge shuffle spills for mapId {} with length {}", mapId, partitionLengths.length); - maybeSingleFileWriter.get().transferMapSpillFile(spills[0].file, partitionLengths); + maybeSingleFileWriter.get() + .transferMapSpillFile(spills[0].file, partitionLengths, sorter.getChecksums()); } else { partitionLengths = mergeSpillsUsingStandardWriter(spills); } @@ -333,7 +334,7 @@ private long[] mergeSpillsUsingStandardWriter(SpillInfo[] spills) throws IOExcep // to be counted as shuffle write, but this will lead to double-counting of the final // SpillInfo's bytes. writeMetrics.decBytesWritten(spills[spills.length - 1].file.length()); - partitionLengths = mapWriter.commitAllPartitions().getPartitionLengths(); + partitionLengths = mapWriter.commitAllPartitions(sorter.getChecksums()).getPartitionLengths(); } catch (Exception e) { try { mapWriter.abort(e); diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/io/LocalDiskShuffleExecutorComponents.java b/core/src/main/java/org/apache/spark/shuffle/sort/io/LocalDiskShuffleExecutorComponents.java index 9e8c54d9315f..eb4d9d9abc8e 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/io/LocalDiskShuffleExecutorComponents.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/io/LocalDiskShuffleExecutorComponents.java @@ -57,7 +57,6 @@ public void initializeExecutor(String appId, String execId, Map throw new IllegalStateException("No blockManager available from the SparkEnv."); } blockResolver = new IndexShuffleBlockResolver(sparkConf, blockManager); - IndexShuffleBlockResolver.set(blockResolver); } @Override diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/io/LocalDiskShuffleMapOutputWriter.java b/core/src/main/java/org/apache/spark/shuffle/sort/io/LocalDiskShuffleMapOutputWriter.java index 0b286264be43..6c5025d1822f 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/io/LocalDiskShuffleMapOutputWriter.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/io/LocalDiskShuffleMapOutputWriter.java @@ -98,7 +98,7 @@ public ShufflePartitionWriter getPartitionWriter(int reducePartitionId) throws I } @Override - public MapOutputCommitMessage commitAllPartitions() throws IOException { + public MapOutputCommitMessage commitAllPartitions(long[] checksums) throws IOException { // Check the position after transferTo loop to see if it is in the right position and raise a // exception if it is incorrect. The position will not be increased to the expected length // after calling transferTo in kernel version 2.6.32. This issue is described at @@ -115,7 +115,8 @@ public MapOutputCommitMessage commitAllPartitions() throws IOException { File resolvedTmp = outputTempFile != null && outputTempFile.isFile() ? outputTempFile : null; log.debug("Writing shuffle index file for mapId {} with length {}", mapId, partitionLengths.length); - blockResolver.writeIndexFileAndCommit(shuffleId, mapId, partitionLengths, resolvedTmp); + blockResolver + .writeMetadataFileAndCommit(shuffleId, mapId, partitionLengths, checksums, resolvedTmp); return MapOutputCommitMessage.of(partitionLengths); } diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/io/LocalDiskSingleSpillMapOutputWriter.java b/core/src/main/java/org/apache/spark/shuffle/sort/io/LocalDiskSingleSpillMapOutputWriter.java index c8b41992a891..6a994b49d3a2 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/io/LocalDiskSingleSpillMapOutputWriter.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/io/LocalDiskSingleSpillMapOutputWriter.java @@ -44,12 +44,14 @@ public LocalDiskSingleSpillMapOutputWriter( @Override public void transferMapSpillFile( File mapSpillFile, - long[] partitionLengths) throws IOException { + long[] partitionLengths, + long[] checksums) throws IOException { // The map spill file already has the proper format, and it contains all of the partition data. // So just transfer it directly to the destination without any merging. File outputFile = blockResolver.getDataFile(shuffleId, mapId); File tempFile = Utils.tempFileWith(outputFile); Files.move(mapSpillFile.toPath(), tempFile.toPath()); - blockResolver.writeIndexFileAndCommit(shuffleId, mapId, partitionLengths, tempFile); + blockResolver + .writeMetadataFileAndCommit(shuffleId, mapId, partitionLengths, checksums, tempFile); } } diff --git a/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala b/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala index 712f5b90e25d..23330565afe3 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala @@ -312,32 +312,54 @@ private[spark] class IndexShuffleBlockResolver( /** - * Write an index file with the offsets of each block, plus a final offset at the end for the - * end of the output file. This will be used by getBlockData to figure out where each block - * begins and ends. + * Commit the data and metadata files as an atomic operation, use the existing ones, or + * replace them with new ones. Note that the metadata parameters (`lengths`, `checksums`) + * will be updated to match the existing ones if use the existing ones. * - * It will commit the data and index file as an atomic operation, use the existing ones, or - * replace them with new ones. + * There're two kinds of metadata files: * - * Note: the `lengths` will be updated to match the existing index file if use the existing ones. + * - index file + * An index file contains the offsets of each block, plus a final offset at the end + * for the end of the output file. It will be used by [[getBlockData]] to figure out + * where each block begins and ends. + * + * - checksum file (optional) + * An checksum file contains the checksum of each block. It will be used to diagnose + * the cause when a block is corrupted. Note that empty `checksums` indicate that + * checksum is disabled. */ - def writeIndexFileAndCommit( + def writeMetadataFileAndCommit( shuffleId: Int, mapId: Long, lengths: Array[Long], + checksums: Array[Long], dataTmp: File): Unit = { val indexFile = getIndexFile(shuffleId, mapId) val indexTmp = Utils.tempFileWith(indexFile) + + val checksumEnabled = checksums.nonEmpty + val (checksumFileOpt, checksumTmpOpt) = if (checksumEnabled) { + assert(lengths.length == checksums.length, + "The size of partition lengths and checksums should be equal") + val checksumFile = getChecksumFile(shuffleId, mapId) + (Some(checksumFile), Some(Utils.tempFileWith(checksumFile))) + } else { + (None, None) + } + try { val dataFile = getDataFile(shuffleId, mapId) // There is only one IndexShuffleBlockResolver per executor, this synchronization make sure // the following check and rename are atomic. this.synchronized { val existingLengths = checkIndexAndDataFile(indexFile, dataFile, lengths.length) - if (existingLengths != null) { + val existingChecksums = + checksumFileOpt.map(getChecksums(_, checksums.length)).getOrElse(checksums) + if (existingLengths != null && existingChecksums != null) { // Another attempt for the same task has already written our map outputs successfully, // so just use the existing partition lengths and delete our temporary map outputs. System.arraycopy(existingLengths, 0, lengths, 0, lengths.length) + System.arraycopy(existingChecksums, 0, checksums, 0, lengths.length) if (dataTmp != null && dataTmp.exists()) { dataTmp.delete() } @@ -369,6 +391,29 @@ private[spark] class IndexShuffleBlockResolver( if (dataTmp != null && dataTmp.exists() && !dataTmp.renameTo(dataFile)) { throw new IOException("fail to rename file " + dataTmp + " to " + dataFile) } + + // write the checksum file + checksumTmpOpt.zip(checksumFileOpt).foreach { case (checksumTmp, checksumFile) => + val out = new DataOutputStream( + new BufferedOutputStream( + new FileOutputStream(checksumTmp) + ) + ) + Utils.tryWithSafeFinally { + checksums.foreach(out.writeLong) + } { + out.close() + } + + if (checksumFile.exists()) { + checksumFile.delete() + } + if (!checksumTmp.renameTo(checksumFile)) { + // It's not worthwhile to fail here after index file and data file are already + // successfully stored due to checksum is only used for the corner error case. + logWarning("fail to rename file " + checksumTmp + " to " + checksumFile) + } + } } } } finally { @@ -376,6 +421,11 @@ private[spark] class IndexShuffleBlockResolver( if (indexTmp.exists() && !indexTmp.delete()) { logError(s"Failed to delete temporary index file at ${indexTmp.getAbsolutePath}") } + checksumTmpOpt.foreach { checksumTmp => + if (checksumTmp.exists() && !checksumTmp.delete()) { + logError(s"Failed to delete temporary checksum file at ${checksumTmp.getAbsolutePath}") + } + } } } @@ -461,37 +511,6 @@ private[spark] class IndexShuffleBlockResolver( } } - def writeChecksumFile(shuffleId: Int, mapId: Long, checksums: Array[Long]): Unit = synchronized { - val checksumFile = getChecksumFile(shuffleId, mapId) - val checksumTmp = Utils.tempFileWith(checksumFile) - val existingChecksums = getChecksums(checksumFile, checksums.length) - if (existingChecksums != null) { - // Another attempt for the same task has already written our checksum file successfully, - // so just use the existing checksums. - System.arraycopy(existingChecksums, 0, checksums, 0, checksums.length) - } else { - val out = new DataOutputStream( - new BufferedOutputStream( - new FileOutputStream(checksumTmp) - ) - ) - Utils.tryWithSafeFinally { - checksums.foreach(out.writeLong) - } { - out.close() - } - - if (checksumFile.exists()) { - checksumFile.delete() - } - if (!checksumTmp.renameTo(checksumFile)) { - // It's not worthwhile to fail here after index file and data file are already - // successfully stored due to checksum is only used for the corner error case. - logWarning("fail to rename file " + checksumTmp + " to " + checksumFile) - } - } - } - override def getBlockData( blockId: BlockId, dirs: Option[Array[String]]): ManagedBuffer = { @@ -544,10 +563,4 @@ private[spark] object IndexShuffleBlockResolver { // The disk store currently expects puts to relate to a (map, reduce) pair, but in the sort // shuffle outputs for several reduces are glommed into a single file. val NOOP_REDUCE_ID = 0 - - @volatile private var _blockResolver: IndexShuffleBlockResolver = _ - - def set(resolver: IndexShuffleBlockResolver): Unit = _blockResolver = resolver - - def get: IndexShuffleBlockResolver = _blockResolver } 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 adbe6eca5800..3cbf30160efb 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 @@ -68,7 +68,7 @@ private[spark] class SortShuffleWriter[K, V, C]( val mapOutputWriter = shuffleExecutorComponents.createMapOutputWriter( dep.shuffleId, mapId, dep.partitioner.numPartitions) sorter.writePartitionedMapOutput(dep.shuffleId, mapId, mapOutputWriter) - partitionLengths = mapOutputWriter.commitAllPartitions().getPartitionLengths + partitionLengths = mapOutputWriter.commitAllPartitions(sorter.getChecksums).getPartitionLengths mapStatus = MapStatus(blockManager.shuffleServerId, partitionLengths, mapId) } 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 29c3d0c2d471..381a77376993 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 @@ -31,7 +31,7 @@ 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.{IndexShuffleBlockResolver, ShufflePartitionPairsWriter} +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.{CompletionIterator, Utils => TryUtils} @@ -150,6 +150,14 @@ private[spark] class ExternalSorter[K, V, C]( Array.empty } + def getChecksums: Array[Long] = { + if (checksumEnabled) { + partitionChecksums.map(_.getValue) + } else { + Array.empty + } + } + // A comparator for keys K that orders them within a partition to allow aggregation or sorting. // Can be a partial ordering by hash code if a total ordering is not provided through by the // user. (A partial ordering means that equal keys have comparator.compare(k, k) = 0, but some @@ -812,11 +820,6 @@ private[spark] class ExternalSorter[K, V, C]( } } - if (checksumEnabled) { - IndexShuffleBlockResolver.get.writeChecksumFile( - shuffleId, mapId, partitionChecksums.map(_.getValue)) - } - context.taskMetrics().incMemoryBytesSpilled(memoryBytesSpilled) context.taskMetrics().incDiskBytesSpilled(diskBytesSpilled) context.taskMetrics().incPeakExecutionMemory(peakMemoryUsedBytes) diff --git a/core/src/test/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriterSuite.java b/core/src/test/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriterSuite.java index 4fa30a4fc5c9..48102c9e0902 100644 --- a/core/src/test/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriterSuite.java +++ b/core/src/test/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriterSuite.java @@ -132,14 +132,11 @@ public void setUp() throws Exception { ); }); - IndexShuffleBlockResolver.set(shuffleBlockResolver); - doNothing().when(shuffleBlockResolver) - .writeChecksumFile(anyInt(), anyLong(), any(long[].class)); when(shuffleBlockResolver.getDataFile(anyInt(), anyLong())).thenReturn(mergedOutputFile); Answer renameTempAnswer = invocationOnMock -> { partitionSizesInMergedFile = (long[]) invocationOnMock.getArguments()[2]; - File tmp = (File) invocationOnMock.getArguments()[3]; + File tmp = (File) invocationOnMock.getArguments()[4]; if (!mergedOutputFile.delete()) { throw new RuntimeException("Failed to delete old merged output file."); } @@ -153,11 +150,13 @@ public void setUp() throws Exception { doAnswer(renameTempAnswer) .when(shuffleBlockResolver) - .writeIndexFileAndCommit(anyInt(), anyLong(), any(long[].class), any(File.class)); + .writeMetadataFileAndCommit( + anyInt(), anyLong(), any(long[].class), any(long[].class), any(File.class)); doAnswer(renameTempAnswer) .when(shuffleBlockResolver) - .writeIndexFileAndCommit(anyInt(), anyLong(), any(long[].class), eq(null)); + .writeMetadataFileAndCommit( + anyInt(), anyLong(), any(long[].class), any(long[].class), eq(null)); when(diskBlockManager.createTempShuffleBlock()).thenAnswer(invocationOnMock -> { TempShuffleBlockId blockId = new TempShuffleBlockId(UUID.randomUUID()); @@ -301,7 +300,6 @@ public void writeWithoutSpilling() throws Exception { @Test public void writeChecksumFileWithoutSpill() throws Exception { IndexShuffleBlockResolver blockResolver = new IndexShuffleBlockResolver(conf, blockManager); - IndexShuffleBlockResolver.set(blockResolver); File checksumFile = new File(tempDir, "checksum"); File dataFile = new File(tempDir, "data"); File indexFile = new File(tempDir, "index"); @@ -328,7 +326,6 @@ public void writeChecksumFileWithoutSpill() throws Exception { @Test public void writeChecksumFileWithSpill() throws Exception { IndexShuffleBlockResolver blockResolver = new IndexShuffleBlockResolver(conf, blockManager); - IndexShuffleBlockResolver.set(blockResolver); File checksumFile = new File(tempDir, "checksum"); File dataFile = new File(tempDir, "data"); File indexFile = new File(tempDir, "index"); diff --git a/core/src/test/scala/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriterSuite.scala b/core/src/test/scala/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriterSuite.scala index 91e599e67727..8bb604f1b47f 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriterSuite.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriterSuite.scala @@ -79,18 +79,16 @@ class BypassMergeSortShuffleWriterSuite when(blockManager.diskBlockManager).thenReturn(diskBlockManager) when(taskContext.taskMemoryManager()).thenReturn(taskMemoryManager) - IndexShuffleBlockResolver.set(blockResolver) - when(blockResolver.writeIndexFileAndCommit( - anyInt, anyLong, any(classOf[Array[Long]]), any(classOf[File]))) + when(blockResolver.writeMetadataFileAndCommit( + anyInt, anyLong, any(classOf[Array[Long]]), any(classOf[Array[Long]]), any(classOf[File]))) .thenAnswer { invocationOnMock => - val tmp = invocationOnMock.getArguments()(3).asInstanceOf[File] + val tmp = invocationOnMock.getArguments()(4).asInstanceOf[File] if (tmp != null) { outputFile.delete tmp.renameTo(outputFile) } null } - doNothing().when(blockResolver).writeChecksumFile(any(), any(), any(classOf[Array[Long]])) when(blockManager.getDiskWriter( any[BlockId], @@ -244,7 +242,6 @@ class BypassMergeSortShuffleWriterSuite test("write checksum file") { val blockResolver = new IndexShuffleBlockResolver(conf, blockManager) - IndexShuffleBlockResolver.set(blockResolver) val shuffleId = shuffleHandle.shuffleId val mapId = 0 val checksumBlockId = ShuffleChecksumBlockId(shuffleId, mapId, 0) diff --git a/core/src/test/scala/org/apache/spark/shuffle/sort/IndexShuffleBlockResolverSuite.scala b/core/src/test/scala/org/apache/spark/shuffle/sort/IndexShuffleBlockResolverSuite.scala index 180b8f37df40..c0f91dad5dc3 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/sort/IndexShuffleBlockResolverSuite.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/sort/IndexShuffleBlockResolverSuite.scala @@ -77,7 +77,7 @@ class IndexShuffleBlockResolverSuite extends SparkFunSuite with BeforeAndAfterEa } { out.close() } - resolver.writeIndexFileAndCommit(shuffleId, mapId, lengths, dataTmp) + resolver.writeMetadataFileAndCommit(shuffleId, mapId, lengths, Array.empty, dataTmp) val indexFile = new File(tempDir.getAbsolutePath, idxName) val dataFile = resolver.getDataFile(shuffleId, mapId) @@ -97,7 +97,7 @@ class IndexShuffleBlockResolverSuite extends SparkFunSuite with BeforeAndAfterEa } { out2.close() } - resolver.writeIndexFileAndCommit(shuffleId, mapId, lengths2, dataTmp2) + resolver.writeMetadataFileAndCommit(shuffleId, mapId, lengths2, Array.empty, dataTmp2) assert(indexFile.length() === (lengths.length + 1) * 8) assert(lengths2.toSeq === lengths.toSeq) @@ -136,7 +136,7 @@ class IndexShuffleBlockResolverSuite extends SparkFunSuite with BeforeAndAfterEa } { out3.close() } - resolver.writeIndexFileAndCommit(shuffleId, mapId, lengths3, dataTmp3) + resolver.writeMetadataFileAndCommit(shuffleId, mapId, lengths3, Array.empty, dataTmp3) assert(indexFile.length() === (lengths3.length + 1) * 8) assert(lengths3.toSeq != lengths.toSeq) assert(dataFile.exists()) @@ -251,8 +251,10 @@ class IndexShuffleBlockResolverSuite extends SparkFunSuite with BeforeAndAfterEa test("write checksum file") { val resolver = new IndexShuffleBlockResolver(conf, blockManager) + val dataTmp = File.createTempFile("shuffle", null, tempDir) + val indexInMemory = Array[Long](0, 1, 2, 3, 4, 5, 6, 7, 8, 9) val checksumsInMemory = Array[Long](0, 1, 2, 3, 4, 5, 6, 7, 8, 9) - resolver.writeChecksumFile(0, 0, checksumsInMemory) + resolver.writeMetadataFileAndCommit(0, 0, indexInMemory, checksumsInMemory, dataTmp) val checksumFile = resolver.getChecksumFile(0, 0) assert(checksumFile.exists()) val checksumsFromFile = resolver.getChecksums(checksumFile, 10) 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 index 0caa5579296b..6e98bc008953 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/sort/SortShuffleWriterSuite.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/sort/SortShuffleWriterSuite.scala @@ -33,8 +33,6 @@ import org.apache.spark.storage.BlockManager import org.apache.spark.util.Utils import org.apache.spark.util.collection.ExternalSorter - - class SortShuffleWriterSuite extends SparkFunSuite with SharedSparkContext @@ -70,7 +68,6 @@ class SortShuffleWriterSuite } shuffleExecutorComponents = new LocalDiskShuffleExecutorComponents( conf, blockManager, shuffleBlockResolver) - IndexShuffleBlockResolver.set(shuffleBlockResolver) } override def afterAll(): Unit = { @@ -152,7 +149,6 @@ class SortShuffleWriterSuite conf.set("spark.hadoop.fs.file.impl", classOf[DebugFilesystem].getName) val localSC = new SparkContext("local[4]", "test", conf) val shuffleBlockResolver = new IndexShuffleBlockResolver(conf) - IndexShuffleBlockResolver.set(shuffleBlockResolver) val context = MemoryTestingUtils.fakeTaskContext(localSC.env) val records = List[(Int, Int)]( (0, 1), (1, 2), (0, 2), (1, 3), (2, 3), (3, 4), (4, 5), (3, 5), (4, 6)) diff --git a/core/src/test/scala/org/apache/spark/shuffle/sort/io/LocalDiskShuffleMapOutputWriterSuite.scala b/core/src/test/scala/org/apache/spark/shuffle/sort/io/LocalDiskShuffleMapOutputWriterSuite.scala index ef5c615bf759..23f011f9fb9a 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/sort/io/LocalDiskShuffleMapOutputWriterSuite.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/sort/io/LocalDiskShuffleMapOutputWriterSuite.scala @@ -74,8 +74,8 @@ class LocalDiskShuffleMapOutputWriterSuite extends SparkFunSuite with BeforeAndA .set("spark.app.id", "example.spark.app") .set("spark.shuffle.unsafe.file.output.buffer", "16k") when(blockResolver.getDataFile(anyInt, anyLong)).thenReturn(mergedOutputFile) - when(blockResolver.writeIndexFileAndCommit( - anyInt, anyLong, any(classOf[Array[Long]]), any(classOf[File]))) + when(blockResolver.writeMetadataFileAndCommit( + anyInt, anyLong, any(classOf[Array[Long]]), any(classOf[Array[Long]]), any(classOf[File]))) .thenAnswer { invocationOnMock => partitionSizesInMergedFile = invocationOnMock.getArguments()(2).asInstanceOf[Array[Long]] val tmp: File = invocationOnMock.getArguments()(3).asInstanceOf[File] @@ -136,7 +136,8 @@ class LocalDiskShuffleMapOutputWriterSuite extends SparkFunSuite with BeforeAndA } private def verifyWrittenRecords(): Unit = { - val committedLengths = mapOutputWriter.commitAllPartitions().getPartitionLengths + val committedLengths = + mapOutputWriter.commitAllPartitions(Array.empty[Long]).getPartitionLengths assert(partitionSizesInMergedFile === partitionLengths) assert(committedLengths === partitionLengths) assert(mergedOutputFile.length() === partitionLengths.sum) From dab8c4d1e3e8404e27793f7a4a368dd6d2d388ca Mon Sep 17 00:00:00 2001 From: "yi.wu" Date: Thu, 17 Jun 2021 22:39:19 +0800 Subject: [PATCH 05/47] use final for checksumEnabled --- .../spark/shuffle/sort/BypassMergeSortShuffleWriter.java | 6 +++--- .../apache/spark/shuffle/sort/ShuffleExternalSorter.java | 6 +++--- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java b/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java index dacf6e8006b8..d0d418f5c6d2 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java @@ -96,7 +96,7 @@ final class BypassMergeSortShuffleWriter extends ShuffleWriter { @Nullable private MapStatus mapStatus; private long[] partitionLengths; private Checksum[] partitionChecksums; - private boolean checksumEnabled; + private final boolean checksumEnabled; /** * Are we in the process of stopping? Because map tasks can call stop() with success = true @@ -124,8 +124,8 @@ final class BypassMergeSortShuffleWriter extends ShuffleWriter { this.writeMetrics = writeMetrics; this.serializer = dep.serializer(); this.shuffleExecutorComponents = shuffleExecutorComponents; - if ((boolean) conf.get(package$.MODULE$.SHUFFLE_CHECKSUM())) { - this.checksumEnabled = true; + this.checksumEnabled = (boolean) conf.get(package$.MODULE$.SHUFFLE_CHECKSUM()); + if (this.checksumEnabled) { this.partitionChecksums = new Adler32[numPartitions]; for (int i = 0; i < numPartitions; i ++) { this.partitionChecksums[i] = new Adler32(); diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleExternalSorter.java b/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleExternalSorter.java index 2c693659ac6c..dba173e33df2 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleExternalSorter.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleExternalSorter.java @@ -111,7 +111,7 @@ final class ShuffleExternalSorter extends MemoryConsumer { @Nullable private MemoryBlock currentPage = null; private long pageCursor = -1; - private boolean checksumEnabled; + private final boolean checksumEnabled; private Checksum[] partitionChecksums; ShuffleExternalSorter( @@ -144,8 +144,8 @@ final class ShuffleExternalSorter extends MemoryConsumer { this.peakMemoryUsedBytes = getMemoryUsage(); this.diskWriteBufferSize = (int) (long) conf.get(package$.MODULE$.SHUFFLE_DISK_WRITE_BUFFER_SIZE()); - if ((boolean) conf.get(package$.MODULE$.SHUFFLE_CHECKSUM())) { - this.checksumEnabled = true; + this.checksumEnabled = (boolean) conf.get(package$.MODULE$.SHUFFLE_CHECKSUM()); + if (this.checksumEnabled) { this.partitionChecksums = new Adler32[numPartitions]; for (int i = 0; i < numPartitions; i ++) { this.partitionChecksums[i] = new Adler32(); From cd25f8aa7801051461bcdb575eb972af927b2a70 Mon Sep 17 00:00:00 2001 From: "yi.wu" Date: Thu, 17 Jun 2021 22:40:58 +0800 Subject: [PATCH 06/47] remove dead code --- .../java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java | 1 - 1 file changed, 1 deletion(-) diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java b/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java index cd269d79866f..06a4a36abfe8 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java @@ -223,7 +223,6 @@ void closeAndWriteOutput() throws IOException { final SpillInfo[] spills = sorter.closeAndGetSpills(); try { partitionLengths = mergeSpills(spills); - sorter.getChecksums(); sorter = null; } finally { for (SpillInfo spill : spills) { From ebd1c80112597c84734b30f6d33655819cd0f4f2 Mon Sep 17 00:00:00 2001 From: "yi.wu" Date: Thu, 17 Jun 2021 22:41:56 +0800 Subject: [PATCH 07/47] move sorter null set to finally --- .../java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java b/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java index 06a4a36abfe8..0fd84d2c36b0 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java @@ -223,8 +223,8 @@ void closeAndWriteOutput() throws IOException { final SpillInfo[] spills = sorter.closeAndGetSpills(); try { partitionLengths = mergeSpills(spills); - sorter = null; } finally { + sorter = null; for (SpillInfo spill : spills) { if (spill.file.exists() && !spill.file.delete()) { logger.error("Error while deleting spill file {}", spill.file.getPath()); From 9257f2b620d07a76ed7c34055f0596992ee36f00 Mon Sep 17 00:00:00 2001 From: "yi.wu" Date: Thu, 17 Jun 2021 22:46:51 +0800 Subject: [PATCH 08/47] combine if conditions in removeDataByMap --- .../shuffle/IndexShuffleBlockResolver.scala | 18 ++++++------------ 1 file changed, 6 insertions(+), 12 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala b/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala index 23330565afe3..52f562879747 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala @@ -144,24 +144,18 @@ private[spark] class IndexShuffleBlockResolver( */ def removeDataByMap(shuffleId: Int, mapId: Long): Unit = { var file = getDataFile(shuffleId, mapId) - if (file.exists()) { - if (!file.delete()) { - logWarning(s"Error deleting data ${file.getPath()}") - } + if (file.exists() && !file.delete()) { + logWarning(s"Error deleting data ${file.getPath()}") } file = getIndexFile(shuffleId, mapId) - if (file.exists()) { - if (!file.delete()) { - logWarning(s"Error deleting index ${file.getPath()}") - } + if (file.exists() && !file.delete()) { + logWarning(s"Error deleting index ${file.getPath()}") } file = getChecksumFile(shuffleId, mapId) - if (file.exists()) { - if (!file.delete()) { - logWarning(s"Error deleting checksum ${file.getPath()}") - } + if (file.exists() && !file.delete()) { + logWarning(s"Error deleting checksum ${file.getPath()}") } } From 52610caae956cb6db20995007ceb9099277580b0 Mon Sep 17 00:00:00 2001 From: "yi.wu" Date: Thu, 17 Jun 2021 22:49:44 +0800 Subject: [PATCH 09/47] simplify the getChecksums in ExternalSorter --- .../org/apache/spark/util/collection/ExternalSorter.scala | 6 +----- 1 file changed, 1 insertion(+), 5 deletions(-) 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 381a77376993..a960b3279c7e 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 @@ -151,11 +151,7 @@ private[spark] class ExternalSorter[K, V, C]( } def getChecksums: Array[Long] = { - if (checksumEnabled) { - partitionChecksums.map(_.getValue) - } else { - Array.empty - } + partitionChecksums.map(_.getValue) } // A comparator for keys K that orders them within a partition to allow aggregation or sorting. From 3942aa186133341947345402aa740584c6af0649 Mon Sep 17 00:00:00 2001 From: "yi.wu" Date: Thu, 17 Jun 2021 22:50:43 +0800 Subject: [PATCH 10/47] add brackets to setChecksum --- .../org/apache/spark/io/MutableCheckedOutputStream.scala | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/io/MutableCheckedOutputStream.scala b/core/src/main/scala/org/apache/spark/io/MutableCheckedOutputStream.scala index dd44ad024af2..754b4a87720a 100644 --- a/core/src/main/scala/org/apache/spark/io/MutableCheckedOutputStream.scala +++ b/core/src/main/scala/org/apache/spark/io/MutableCheckedOutputStream.scala @@ -27,7 +27,9 @@ import java.util.zip.Checksum class MutableCheckedOutputStream(out: OutputStream) extends OutputStream { private var checksum: Checksum = _ - def setChecksum(c: Checksum): Unit = this.checksum = c + def setChecksum(c: Checksum): Unit = { + this.checksum = c + } override def write(b: Int): Unit = { assert(checksum != null, "Checksum is not set.") From 7aa5c5c8bd5b76f068cfe4642b1503daccdeb566 Mon Sep 17 00:00:00 2001 From: "yi.wu" Date: Fri, 18 Jun 2021 00:09:34 +0800 Subject: [PATCH 11/47] fix mima --- project/MimaExcludes.scala | 9 ++++++++- 1 file changed, 8 insertions(+), 1 deletion(-) diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index 032dd19fd594..01b7bed4f280 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -95,7 +95,14 @@ object MimaExcludes { ProblemFilters.exclude[IncompatibleTemplateDefProblem]("org.apache.spark.util.collection.WritablePartitionedIterator"), // [SPARK-35757][CORE] Add bitwise AND operation and functionality for intersecting bloom filters - ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.util.sketch.BloomFilter.intersectInPlace") + ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.util.sketch.BloomFilter.intersectInPlace"), + + // [SPARK-35276][CORE] Calculate checksum for shuffle data and write as checksum file + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.shuffle.sort.io.LocalDiskShuffleMapOutputWriter.commitAllPartitions"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.shuffle.sort.io.LocalDiskSingleSpillMapOutputWriter.transferMapSpillFile"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.shuffle.api.ShuffleMapOutputWriter.commitAllPartitions"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.shuffle.api.SingleSpillShuffleMapOutputWriter.transferMapSpillFile"), + ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.shuffle.api.SingleSpillShuffleMapOutputWriter.transferMapSpillFile") ) def excludes(version: String) = version match { From 3788e77a28b0e51e1ba2d885a4ac70f6f364d35b Mon Sep 17 00:00:00 2001 From: "yi.wu" Date: Mon, 5 Jul 2021 16:04:26 +0800 Subject: [PATCH 12/47] add enabled suffix --- .../main/scala/org/apache/spark/internal/config/package.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/internal/config/package.scala b/core/src/main/scala/org/apache/spark/internal/config/package.scala index 43ebd8d85875..3ee36123f716 100644 --- a/core/src/main/scala/org/apache/spark/internal/config/package.scala +++ b/core/src/main/scala/org/apache/spark/internal/config/package.scala @@ -1369,7 +1369,7 @@ package object config { .createWithDefault(4096) private[spark] val SHUFFLE_CHECKSUM = - ConfigBuilder("spark.shuffle.checksum") + ConfigBuilder("spark.shuffle.checksum.enabled") .doc("Whether to calculate the checksum of shuffle output. If enabled, Spark will try " + "its best to tell if shuffle data corruption is caused by network or disk or others.") .version("3.2.0") From 627c597cd92a94721177684130bdcac49ebe32df Mon Sep 17 00:00:00 2001 From: "yi.wu" Date: Mon, 5 Jul 2021 16:04:47 +0800 Subject: [PATCH 13/47] change version to 3.3.0 --- .../main/scala/org/apache/spark/internal/config/package.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/internal/config/package.scala b/core/src/main/scala/org/apache/spark/internal/config/package.scala index 3ee36123f716..a44b350028e6 100644 --- a/core/src/main/scala/org/apache/spark/internal/config/package.scala +++ b/core/src/main/scala/org/apache/spark/internal/config/package.scala @@ -1372,7 +1372,7 @@ package object config { ConfigBuilder("spark.shuffle.checksum.enabled") .doc("Whether to calculate the checksum of shuffle output. If enabled, Spark will try " + "its best to tell if shuffle data corruption is caused by network or disk or others.") - .version("3.2.0") + .version("3.3.0") .booleanConf .createWithDefault(true) From dae2dca2333b7a2bd4567f5449bde7796c7af6d6 Mon Sep 17 00:00:00 2001 From: "yi.wu" Date: Mon, 5 Jul 2021 16:17:48 +0800 Subject: [PATCH 14/47] rename to SHUFFLE_CHECKSUM_ENABLED --- .../apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java | 2 +- .../org/apache/spark/shuffle/sort/ShuffleExternalSorter.java | 2 +- .../main/scala/org/apache/spark/internal/config/package.scala | 2 +- .../scala/org/apache/spark/util/collection/ExternalSorter.scala | 2 +- 4 files changed, 4 insertions(+), 4 deletions(-) diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java b/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java index d0d418f5c6d2..ea7ad55c4028 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java @@ -124,7 +124,7 @@ final class BypassMergeSortShuffleWriter extends ShuffleWriter { this.writeMetrics = writeMetrics; this.serializer = dep.serializer(); this.shuffleExecutorComponents = shuffleExecutorComponents; - this.checksumEnabled = (boolean) conf.get(package$.MODULE$.SHUFFLE_CHECKSUM()); + this.checksumEnabled = (boolean) conf.get(package$.MODULE$.SHUFFLE_CHECKSUM_ENABLED()); if (this.checksumEnabled) { this.partitionChecksums = new Adler32[numPartitions]; for (int i = 0; i < numPartitions; i ++) { diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleExternalSorter.java b/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleExternalSorter.java index dba173e33df2..2c6601f858c4 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleExternalSorter.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleExternalSorter.java @@ -144,7 +144,7 @@ final class ShuffleExternalSorter extends MemoryConsumer { this.peakMemoryUsedBytes = getMemoryUsage(); this.diskWriteBufferSize = (int) (long) conf.get(package$.MODULE$.SHUFFLE_DISK_WRITE_BUFFER_SIZE()); - this.checksumEnabled = (boolean) conf.get(package$.MODULE$.SHUFFLE_CHECKSUM()); + this.checksumEnabled = (boolean) conf.get(package$.MODULE$.SHUFFLE_CHECKSUM_ENABLED()); if (this.checksumEnabled) { this.partitionChecksums = new Adler32[numPartitions]; for (int i = 0; i < numPartitions; i ++) { diff --git a/core/src/main/scala/org/apache/spark/internal/config/package.scala b/core/src/main/scala/org/apache/spark/internal/config/package.scala index a44b350028e6..cfea9fa8d703 100644 --- a/core/src/main/scala/org/apache/spark/internal/config/package.scala +++ b/core/src/main/scala/org/apache/spark/internal/config/package.scala @@ -1368,7 +1368,7 @@ package object config { s"The buffer size must be greater than 0 and less than or equal to ${Int.MaxValue}.") .createWithDefault(4096) - private[spark] val SHUFFLE_CHECKSUM = + private[spark] val SHUFFLE_CHECKSUM_ENABLED = ConfigBuilder("spark.shuffle.checksum.enabled") .doc("Whether to calculate the checksum of shuffle output. If enabled, Spark will try " + "its best to tell if shuffle data corruption is caused by network or disk or others.") 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 a960b3279c7e..d8f1896bb805 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 @@ -143,7 +143,7 @@ private[spark] class ExternalSorter[K, V, C]( private val forceSpillFiles = new ArrayBuffer[SpilledFile] @volatile private var readingIterator: SpillableIterator = null - private val checksumEnabled = conf.get(config.SHUFFLE_CHECKSUM) + private val checksumEnabled = conf.get(config.SHUFFLE_CHECKSUM_ENABLED) private val partitionChecksums = if (checksumEnabled) { Array.fill[Checksum](numPartitions)(new Adler32()) } else { From c82710e43f23863337743ed22e0ce3d5e824466a Mon Sep 17 00:00:00 2001 From: "yi.wu" Date: Mon, 5 Jul 2021 17:05:51 +0800 Subject: [PATCH 15/47] add ShuffleChecksumHelper to support different checksum algo --- .../checksum/ShuffleChecksumHelper.java | 40 +++++++++++++++++++ .../sort/BypassMergeSortShuffleWriter.java | 13 +++--- .../shuffle/sort/ShuffleExternalSorter.java | 11 +++-- .../shuffle/sort/UnsafeShuffleWriter.java | 4 +- .../spark/internal/config/package.scala | 10 +++++ .../util/collection/ExternalSorter.scala | 9 ++--- 6 files changed, 67 insertions(+), 20 deletions(-) create mode 100644 core/src/main/java/org/apache/spark/shuffle/checksum/ShuffleChecksumHelper.java diff --git a/core/src/main/java/org/apache/spark/shuffle/checksum/ShuffleChecksumHelper.java b/core/src/main/java/org/apache/spark/shuffle/checksum/ShuffleChecksumHelper.java new file mode 100644 index 000000000000..123348864015 --- /dev/null +++ b/core/src/main/java/org/apache/spark/shuffle/checksum/ShuffleChecksumHelper.java @@ -0,0 +1,40 @@ +package org.apache.spark.shuffle.checksum; + +import org.apache.spark.SparkConf; +import org.apache.spark.SparkException; +import org.apache.spark.internal.config.package$; + +import java.util.zip.Adler32; +import java.util.zip.CRC32; +import java.util.zip.Checksum; + +public class ShuffleChecksumHelper { + + public static boolean isShuffleChecksumEnabled(SparkConf conf) { + return (boolean) conf.get(package$.MODULE$.SHUFFLE_CHECKSUM_ENABLED()); + } + + public static Checksum[] createPartitionChecksums(int numPartitions, SparkConf conf) + throws SparkException { + Checksum[] partitionChecksums; + String checksumAlgo = conf.get(package$.MODULE$.SHUFFLE_CHECKSUM_ALGORITHM()); + switch (checksumAlgo) { + case "Adler32": + partitionChecksums = new Adler32[numPartitions]; + for (int i = 0; i < numPartitions; i ++) { + partitionChecksums[i] = new Adler32(); + } + return partitionChecksums; + + case "CRC32": + partitionChecksums = new CRC32[numPartitions]; + for (int i = 0; i < numPartitions; i ++) { + partitionChecksums[i] = new CRC32(); + } + return partitionChecksums; + + default: + throw new SparkException("Unsupported shuffle checksum algorithm: " + checksumAlgo); + } + } +} diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java b/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java index ea7ad55c4028..a126fd67856d 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java @@ -23,10 +23,11 @@ import java.io.OutputStream; import java.nio.channels.FileChannel; import java.util.Optional; -import java.util.zip.Adler32; import java.util.zip.Checksum; import javax.annotation.Nullable; +import org.apache.spark.SparkException; +import org.apache.spark.shuffle.checksum.ShuffleChecksumHelper; import scala.None$; import scala.Option; import scala.Product2; @@ -44,6 +45,7 @@ import org.apache.spark.shuffle.api.ShuffleMapOutputWriter; import org.apache.spark.shuffle.api.ShufflePartitionWriter; import org.apache.spark.shuffle.api.WritableByteChannelWrapper; +import org.apache.spark.shuffle.checksum.ShuffleChecksumHelper.*; import org.apache.spark.internal.config.package$; import org.apache.spark.scheduler.MapStatus; import org.apache.spark.scheduler.MapStatus$; @@ -111,7 +113,7 @@ final class BypassMergeSortShuffleWriter extends ShuffleWriter { long mapId, SparkConf conf, ShuffleWriteMetricsReporter writeMetrics, - ShuffleExecutorComponents shuffleExecutorComponents) { + ShuffleExecutorComponents shuffleExecutorComponents) throws SparkException { // Use getSizeAsKb (not bytes) to maintain backwards compatibility if no units are provided this.fileBufferSize = (int) (long) conf.get(package$.MODULE$.SHUFFLE_FILE_BUFFER_SIZE()) * 1024; this.transferToEnabled = conf.getBoolean("spark.file.transferTo", true); @@ -124,12 +126,9 @@ final class BypassMergeSortShuffleWriter extends ShuffleWriter { this.writeMetrics = writeMetrics; this.serializer = dep.serializer(); this.shuffleExecutorComponents = shuffleExecutorComponents; - this.checksumEnabled = (boolean) conf.get(package$.MODULE$.SHUFFLE_CHECKSUM_ENABLED()); + this.checksumEnabled = ShuffleChecksumHelper.isShuffleChecksumEnabled(conf); if (this.checksumEnabled) { - this.partitionChecksums = new Adler32[numPartitions]; - for (int i = 0; i < numPartitions; i ++) { - this.partitionChecksums[i] = new Adler32(); - } + this.partitionChecksums = ShuffleChecksumHelper.createPartitionChecksums(numPartitions, conf); } } diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleExternalSorter.java b/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleExternalSorter.java index 2c6601f858c4..a505c1b756c3 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleExternalSorter.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleExternalSorter.java @@ -24,6 +24,7 @@ import java.util.zip.Adler32; import java.util.zip.Checksum; +import org.apache.spark.SparkException; import scala.Tuple2; import com.google.common.annotations.VisibleForTesting; @@ -41,6 +42,7 @@ import org.apache.spark.serializer.DummySerializerInstance; import org.apache.spark.serializer.SerializerInstance; import org.apache.spark.shuffle.ShuffleWriteMetricsReporter; +import org.apache.spark.shuffle.checksum.ShuffleChecksumHelper; import org.apache.spark.storage.BlockManager; import org.apache.spark.storage.DiskBlockObjectWriter; import org.apache.spark.storage.FileSegment; @@ -123,7 +125,7 @@ final class ShuffleExternalSorter extends MemoryConsumer { long mapId, int numPartitions, SparkConf conf, - ShuffleWriteMetricsReporter writeMetrics) { + ShuffleWriteMetricsReporter writeMetrics) throws SparkException { super(memoryManager, (int) Math.min(PackedRecordPointer.MAXIMUM_PAGE_SIZE_BYTES, memoryManager.pageSizeBytes()), memoryManager.getTungstenMemoryMode()); @@ -144,12 +146,9 @@ final class ShuffleExternalSorter extends MemoryConsumer { this.peakMemoryUsedBytes = getMemoryUsage(); this.diskWriteBufferSize = (int) (long) conf.get(package$.MODULE$.SHUFFLE_DISK_WRITE_BUFFER_SIZE()); - this.checksumEnabled = (boolean) conf.get(package$.MODULE$.SHUFFLE_CHECKSUM_ENABLED()); + this.checksumEnabled = ShuffleChecksumHelper.isShuffleChecksumEnabled(conf); if (this.checksumEnabled) { - this.partitionChecksums = new Adler32[numPartitions]; - for (int i = 0; i < numPartitions; i ++) { - this.partitionChecksums[i] = new Adler32(); - } + this.partitionChecksums = ShuffleChecksumHelper.createPartitionChecksums(numPartitions, conf); } } diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java b/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java index 0fd84d2c36b0..a49bcc3696fe 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java @@ -115,7 +115,7 @@ public UnsafeShuffleWriter( TaskContext taskContext, SparkConf sparkConf, ShuffleWriteMetricsReporter writeMetrics, - ShuffleExecutorComponents shuffleExecutorComponents) { + ShuffleExecutorComponents shuffleExecutorComponents) throws SparkException { final int numPartitions = handle.dependency().partitioner().numPartitions(); if (numPartitions > SortShuffleManager.MAX_SHUFFLE_OUTPUT_PARTITIONS_FOR_SERIALIZED_MODE()) { throw new IllegalArgumentException( @@ -198,7 +198,7 @@ public void write(scala.collection.Iterator> records) throws IOEx } } - private void open() { + private void open() throws SparkException { assert (sorter == null); sorter = new ShuffleExternalSorter( memoryManager, diff --git a/core/src/main/scala/org/apache/spark/internal/config/package.scala b/core/src/main/scala/org/apache/spark/internal/config/package.scala index cfea9fa8d703..a24d9c1a2d9a 100644 --- a/core/src/main/scala/org/apache/spark/internal/config/package.scala +++ b/core/src/main/scala/org/apache/spark/internal/config/package.scala @@ -1376,6 +1376,16 @@ package object config { .booleanConf .createWithDefault(true) + private[spark] val SHUFFLE_CHECKSUM_ALGORITHM = + ConfigBuilder("spark.shuffle.checksum.algorithm") + .doc("The algorithm used to calculate the checksum. Currently, it only supports" + + " built-in algorithms of JDK.") + .version("3.3.0") + .stringConf + .checkValue(Set("Adler32", "CRC32").contains, "Shuffle checksum algorithm " + + "should be either Adler32 or CRC32.") + .createWithDefault("Adler32") + private[spark] val SHUFFLE_COMPRESS = ConfigBuilder("spark.shuffle.compress") .doc("Whether to compress shuffle output. Compression will use " + 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 d8f1896bb805..f6b0ef15ddd4 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 @@ -19,8 +19,6 @@ package org.apache.spark.util.collection import java.io._ import java.util.Comparator -import java.util.zip.Adler32 -import java.util.zip.Checksum import scala.collection.mutable import scala.collection.mutable.ArrayBuffer @@ -31,8 +29,9 @@ 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.{ShufflePartitionPairsWriter} +import org.apache.spark.shuffle.ShufflePartitionPairsWriter import org.apache.spark.shuffle.api.{ShuffleMapOutputWriter, ShufflePartitionWriter} +import org.apache.spark.shuffle.checksum.ShuffleChecksumHelper import org.apache.spark.storage.{BlockId, DiskBlockObjectWriter, ShuffleBlockId} import org.apache.spark.util.{CompletionIterator, Utils => TryUtils} @@ -143,9 +142,9 @@ private[spark] class ExternalSorter[K, V, C]( private val forceSpillFiles = new ArrayBuffer[SpilledFile] @volatile private var readingIterator: SpillableIterator = null - private val checksumEnabled = conf.get(config.SHUFFLE_CHECKSUM_ENABLED) + private val checksumEnabled = ShuffleChecksumHelper.isShuffleChecksumEnabled(conf) private val partitionChecksums = if (checksumEnabled) { - Array.fill[Checksum](numPartitions)(new Adler32()) + ShuffleChecksumHelper.createPartitionChecksums(numPartitions, conf) } else { Array.empty } From d6998c25642e4bf4d0d3e122fe14572eab9107cb Mon Sep 17 00:00:00 2001 From: "yi.wu" Date: Mon, 5 Jul 2021 17:34:17 +0800 Subject: [PATCH 16/47] add algo to file extension --- .../checksum/ShuffleChecksumHelper.java | 13 ++++++++++++- .../shuffle/IndexShuffleBlockResolver.scala | 6 ++++-- .../shuffle/sort/UnsafeShuffleWriterSuite.java | 18 ++++++++---------- .../sort/IndexShuffleBlockResolverSuite.scala | 6 ++++++ 4 files changed, 30 insertions(+), 13 deletions(-) diff --git a/core/src/main/java/org/apache/spark/shuffle/checksum/ShuffleChecksumHelper.java b/core/src/main/java/org/apache/spark/shuffle/checksum/ShuffleChecksumHelper.java index 123348864015..de7a67f88738 100644 --- a/core/src/main/java/org/apache/spark/shuffle/checksum/ShuffleChecksumHelper.java +++ b/core/src/main/java/org/apache/spark/shuffle/checksum/ShuffleChecksumHelper.java @@ -3,6 +3,8 @@ import org.apache.spark.SparkConf; import org.apache.spark.SparkException; import org.apache.spark.internal.config.package$; +import org.apache.spark.storage.BlockId; +import org.apache.spark.storage.ShuffleChecksumBlockId; import java.util.zip.Adler32; import java.util.zip.CRC32; @@ -17,7 +19,7 @@ public static boolean isShuffleChecksumEnabled(SparkConf conf) { public static Checksum[] createPartitionChecksums(int numPartitions, SparkConf conf) throws SparkException { Checksum[] partitionChecksums; - String checksumAlgo = conf.get(package$.MODULE$.SHUFFLE_CHECKSUM_ALGORITHM()); + String checksumAlgo = shuffleChecksumAlgorithm(conf); switch (checksumAlgo) { case "Adler32": partitionChecksums = new Adler32[numPartitions]; @@ -37,4 +39,13 @@ public static Checksum[] createPartitionChecksums(int numPartitions, SparkConf c throw new SparkException("Unsupported shuffle checksum algorithm: " + checksumAlgo); } } + + public static String shuffleChecksumAlgorithm(SparkConf conf) { + return conf.get(package$.MODULE$.SHUFFLE_CHECKSUM_ALGORITHM()); + } + + public static String getChecksumFileName(ShuffleChecksumBlockId blockId, SparkConf conf) { + // append the shuffle checksum algorithm as the file extension + return String.format("%s.%s", blockId.name(), shuffleChecksumAlgorithm(conf)); + } } diff --git a/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala b/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala index 52f562879747..140a495551c3 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala @@ -33,6 +33,7 @@ import org.apache.spark.network.netty.SparkTransportConf import org.apache.spark.network.shuffle.{ExecutorDiskUtils, MergedBlockMeta} import org.apache.spark.serializer.SerializerManager import org.apache.spark.shuffle.IndexShuffleBlockResolver.NOOP_REDUCE_ID +import org.apache.spark.shuffle.checksum.ShuffleChecksumHelper import org.apache.spark.storage._ import org.apache.spark.util.Utils @@ -498,10 +499,11 @@ private[spark] class IndexShuffleBlockResolver( mapId: Long, dirs: Option[Array[String]] = None): File = { val blockId = ShuffleChecksumBlockId(shuffleId, mapId, NOOP_REDUCE_ID) + val fileName = ShuffleChecksumHelper.getChecksumFileName(blockId, conf) dirs - .map(ExecutorDiskUtils.getFile(_, blockManager.subDirsPerLocalDir, blockId.name)) + .map(ExecutorDiskUtils.getFile(_, blockManager.subDirsPerLocalDir, fileName)) .getOrElse { - blockManager.diskBlockManager.getFile(blockId) + blockManager.diskBlockManager.getFile(fileName) } } diff --git a/core/src/test/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriterSuite.java b/core/src/test/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriterSuite.java index 48102c9e0902..71a3459bad2e 100644 --- a/core/src/test/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriterSuite.java +++ b/core/src/test/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriterSuite.java @@ -22,6 +22,7 @@ import java.nio.file.Files; import java.util.*; +import org.apache.spark.*; import org.apache.spark.shuffle.ShuffleChecksumTester; import org.mockito.stubbing.Answer; import scala.*; @@ -34,10 +35,6 @@ import org.mockito.Mock; import org.mockito.MockitoAnnotations; -import org.apache.spark.HashPartitioner; -import org.apache.spark.ShuffleDependency; -import org.apache.spark.SparkConf; -import org.apache.spark.TaskContext; import org.apache.spark.executor.ShuffleWriteMetrics; import org.apache.spark.executor.TaskMetrics; import org.apache.spark.io.CompressionCodec$; @@ -171,13 +168,14 @@ public void setUp() throws Exception { when(taskContext.taskMemoryManager()).thenReturn(taskMemoryManager); } - private UnsafeShuffleWriter createWriter(boolean transferToEnabled) { + private UnsafeShuffleWriter createWriter(boolean transferToEnabled) + throws SparkException { return createWriter(transferToEnabled, shuffleBlockResolver); } private UnsafeShuffleWriter createWriter( boolean transferToEnabled, - IndexShuffleBlockResolver blockResolver) { + IndexShuffleBlockResolver blockResolver) throws SparkException { conf.set("spark.file.transferTo", String.valueOf(transferToEnabled)); return new UnsafeShuffleWriter<>( blockManager, @@ -225,12 +223,12 @@ private List> readRecordsFromFile() throws IOException { } @Test(expected=IllegalStateException.class) - public void mustCallWriteBeforeSuccessfulStop() throws IOException { + public void mustCallWriteBeforeSuccessfulStop() throws IOException, SparkException { createWriter(false).stop(true); } @Test - public void doNotNeedToCallWriteBeforeUnsuccessfulStop() throws IOException { + public void doNotNeedToCallWriteBeforeUnsuccessfulStop() throws IOException, SparkException { createWriter(false).stop(false); } @@ -380,7 +378,7 @@ private void testMergingSpills( private void testMergingSpills( boolean transferToEnabled, - boolean encrypted) throws IOException { + boolean encrypted) throws IOException, SparkException { final UnsafeShuffleWriter writer = createWriter(transferToEnabled); final ArrayList> dataToWrite = new ArrayList<>(); for (int i : new int[] { 1, 2, 3, 4, 4, 2 }) { @@ -578,7 +576,7 @@ public void writeRecordsThatAreBiggerThanMaxRecordSize() throws Exception { } @Test - public void spillFilesAreDeletedWhenStoppingAfterError() throws IOException { + public void spillFilesAreDeletedWhenStoppingAfterError() throws IOException, SparkException { final UnsafeShuffleWriter writer = createWriter(false); writer.insertRecordIntoSorter(new Tuple2<>(1, 1)); writer.insertRecordIntoSorter(new Tuple2<>(2, 2)); diff --git a/core/src/test/scala/org/apache/spark/shuffle/sort/IndexShuffleBlockResolverSuite.scala b/core/src/test/scala/org/apache/spark/shuffle/sort/IndexShuffleBlockResolverSuite.scala index c0f91dad5dc3..49c079cd4fce 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/sort/IndexShuffleBlockResolverSuite.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/sort/IndexShuffleBlockResolverSuite.scala @@ -28,6 +28,7 @@ import org.roaringbitmap.RoaringBitmap import org.scalatest.BeforeAndAfterEach import org.apache.spark.{MapOutputTracker, SparkConf, SparkFunSuite} +import org.apache.spark.internal.config import org.apache.spark.shuffle.{IndexShuffleBlockResolver, ShuffleBlockInfo} import org.apache.spark.storage._ import org.apache.spark.util.Utils @@ -49,6 +50,8 @@ class IndexShuffleBlockResolverSuite extends SparkFunSuite with BeforeAndAfterEa when(blockManager.diskBlockManager).thenReturn(diskBlockManager) when(diskBlockManager.getFile(any[BlockId])).thenAnswer( (invocation: InvocationOnMock) => new File(tempDir, invocation.getArguments.head.toString)) + when(diskBlockManager.getFile(any[String])).thenAnswer( + (invocation: InvocationOnMock) => new File(tempDir, invocation.getArguments.head.toString)) when(diskBlockManager.getMergedShuffleFile( any[BlockId], any[Option[Array[String]]])).thenAnswer( (invocation: InvocationOnMock) => new File(tempDir, invocation.getArguments.head.toString)) @@ -257,6 +260,9 @@ class IndexShuffleBlockResolverSuite extends SparkFunSuite with BeforeAndAfterEa resolver.writeMetadataFileAndCommit(0, 0, indexInMemory, checksumsInMemory, dataTmp) val checksumFile = resolver.getChecksumFile(0, 0) assert(checksumFile.exists()) + val checksumFileName = checksumFile.toString + val checksumAlgo = checksumFileName.substring(checksumFileName.lastIndexOf(".") + 1) + assert(checksumAlgo === conf.get(config.SHUFFLE_CHECKSUM_ALGORITHM)) val checksumsFromFile = resolver.getChecksums(checksumFile, 10) assert(checksumsInMemory === checksumsFromFile) } From 9745479d6535548b03420204b4ade29edeb628a6 Mon Sep 17 00:00:00 2001 From: "yi.wu" Date: Mon, 5 Jul 2021 17:49:43 +0800 Subject: [PATCH 17/47] update comment --- .../org/apache/spark/shuffle/api/ShuffleMapOutputWriter.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/java/org/apache/spark/shuffle/api/ShuffleMapOutputWriter.java b/core/src/main/java/org/apache/spark/shuffle/api/ShuffleMapOutputWriter.java index d484af2589b1..22b6e13196da 100644 --- a/core/src/main/java/org/apache/spark/shuffle/api/ShuffleMapOutputWriter.java +++ b/core/src/main/java/org/apache/spark/shuffle/api/ShuffleMapOutputWriter.java @@ -73,8 +73,8 @@ public interface ShuffleMapOutputWriter { *

* 2) An optional metadata blob that can be used by shuffle readers. * - * @param checksums The checksum values for each partition if shuffle checksum enabled. - * Otherwise, it's empty. + * @param checksums The checksum values for each partition (where checksum index is equivalent to + * partition id) if shuffle checksum enabled. Otherwise, it's empty. */ MapOutputCommitMessage commitAllPartitions(long[] checksums) throws IOException; From 48de1521b9c6fddc7083a639b849994fd314e9fa Mon Sep 17 00:00:00 2001 From: "yi.wu" Date: Mon, 5 Jul 2021 17:50:46 +0800 Subject: [PATCH 18/47] remove unused import --- .../org/apache/spark/shuffle/checksum/ShuffleChecksumHelper.java | 1 - .../org/apache/spark/shuffle/sort/ShuffleExternalSorter.java | 1 - 2 files changed, 2 deletions(-) diff --git a/core/src/main/java/org/apache/spark/shuffle/checksum/ShuffleChecksumHelper.java b/core/src/main/java/org/apache/spark/shuffle/checksum/ShuffleChecksumHelper.java index de7a67f88738..e6a6fd8182f1 100644 --- a/core/src/main/java/org/apache/spark/shuffle/checksum/ShuffleChecksumHelper.java +++ b/core/src/main/java/org/apache/spark/shuffle/checksum/ShuffleChecksumHelper.java @@ -3,7 +3,6 @@ import org.apache.spark.SparkConf; import org.apache.spark.SparkException; import org.apache.spark.internal.config.package$; -import org.apache.spark.storage.BlockId; import org.apache.spark.storage.ShuffleChecksumBlockId; import java.util.zip.Adler32; diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleExternalSorter.java b/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleExternalSorter.java index a505c1b756c3..27f3d9b81d2e 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleExternalSorter.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleExternalSorter.java @@ -21,7 +21,6 @@ import java.io.File; import java.io.IOException; import java.util.LinkedList; -import java.util.zip.Adler32; import java.util.zip.Checksum; import org.apache.spark.SparkException; From c28e7e7ebb14f5c44938b5382d2b1f56821cd716 Mon Sep 17 00:00:00 2001 From: "yi.wu" Date: Tue, 6 Jul 2021 16:32:18 +0800 Subject: [PATCH 19/47] update ShuffleChecksumHelper --- .../checksum/ShuffleChecksumHelper.java | 17 ++++++++++- .../sort/BypassMergeSortShuffleWriter.java | 29 ++++++------------- .../shuffle/sort/ShuffleExternalSorter.java | 23 ++++----------- .../util/collection/ExternalSorter.scala | 12 ++------ 4 files changed, 34 insertions(+), 47 deletions(-) diff --git a/core/src/main/java/org/apache/spark/shuffle/checksum/ShuffleChecksumHelper.java b/core/src/main/java/org/apache/spark/shuffle/checksum/ShuffleChecksumHelper.java index e6a6fd8182f1..76db3211c57a 100644 --- a/core/src/main/java/org/apache/spark/shuffle/checksum/ShuffleChecksumHelper.java +++ b/core/src/main/java/org/apache/spark/shuffle/checksum/ShuffleChecksumHelper.java @@ -15,9 +15,15 @@ public static boolean isShuffleChecksumEnabled(SparkConf conf) { return (boolean) conf.get(package$.MODULE$.SHUFFLE_CHECKSUM_ENABLED()); } - public static Checksum[] createPartitionChecksums(int numPartitions, SparkConf conf) + public static Checksum[] createPartitionChecksumsIfEnabled(int numPartitions, SparkConf conf) throws SparkException { Checksum[] partitionChecksums; + + if (!isShuffleChecksumEnabled(conf)) { + partitionChecksums = new Checksum[0]; + return partitionChecksums; + } + String checksumAlgo = shuffleChecksumAlgorithm(conf); switch (checksumAlgo) { case "Adler32": @@ -39,6 +45,15 @@ public static Checksum[] createPartitionChecksums(int numPartitions, SparkConf c } } + public static long[] getChecksumValues(Checksum[] partitionChecksums) { + int numPartitions = partitionChecksums.length; + long[] checksumValues = new long[numPartitions]; + for (int i = 0; i < numPartitions; i ++) { + checksumValues[i] = partitionChecksums[i].getValue(); + } + return checksumValues; + } + public static String shuffleChecksumAlgorithm(SparkConf conf) { return conf.get(package$.MODULE$.SHUFFLE_CHECKSUM_ALGORITHM()); } diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java b/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java index a126fd67856d..12dfb8f5f4b6 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java @@ -26,8 +26,6 @@ import java.util.zip.Checksum; import javax.annotation.Nullable; -import org.apache.spark.SparkException; -import org.apache.spark.shuffle.checksum.ShuffleChecksumHelper; import scala.None$; import scala.Option; import scala.Product2; @@ -41,11 +39,11 @@ import org.apache.spark.Partitioner; import org.apache.spark.ShuffleDependency; import org.apache.spark.SparkConf; +import org.apache.spark.SparkException; import org.apache.spark.shuffle.api.ShuffleExecutorComponents; import org.apache.spark.shuffle.api.ShuffleMapOutputWriter; import org.apache.spark.shuffle.api.ShufflePartitionWriter; import org.apache.spark.shuffle.api.WritableByteChannelWrapper; -import org.apache.spark.shuffle.checksum.ShuffleChecksumHelper.*; import org.apache.spark.internal.config.package$; import org.apache.spark.scheduler.MapStatus; import org.apache.spark.scheduler.MapStatus$; @@ -53,6 +51,7 @@ import org.apache.spark.serializer.SerializerInstance; import org.apache.spark.shuffle.ShuffleWriteMetricsReporter; import org.apache.spark.shuffle.ShuffleWriter; +import org.apache.spark.shuffle.checksum.ShuffleChecksumHelper; import org.apache.spark.storage.*; import org.apache.spark.util.Utils; @@ -97,8 +96,8 @@ final class BypassMergeSortShuffleWriter extends ShuffleWriter { private FileSegment[] partitionWriterSegments; @Nullable private MapStatus mapStatus; private long[] partitionLengths; - private Checksum[] partitionChecksums; - private final boolean checksumEnabled; + /** Checksum calculator for each partition. Empty when shuffle checksum disabled. */ + private final Checksum[] partitionChecksums; /** * Are we in the process of stopping? Because map tasks can call stop() with success = true @@ -126,10 +125,7 @@ final class BypassMergeSortShuffleWriter extends ShuffleWriter { this.writeMetrics = writeMetrics; this.serializer = dep.serializer(); this.shuffleExecutorComponents = shuffleExecutorComponents; - this.checksumEnabled = ShuffleChecksumHelper.isShuffleChecksumEnabled(conf); - if (this.checksumEnabled) { - this.partitionChecksums = ShuffleChecksumHelper.createPartitionChecksums(numPartitions, conf); - } + this.partitionChecksums = ShuffleChecksumHelper.createPartitionChecksumsIfEnabled(numPartitions, conf); } @Override @@ -155,7 +151,7 @@ public void write(Iterator> records) throws IOException { final BlockId blockId = tempShuffleBlockIdPlusFile._1(); DiskBlockObjectWriter writer = blockManager.getDiskWriter(blockId, file, serInstance, fileBufferSize, writeMetrics); - if (checksumEnabled) { + if (partitionChecksums.length > 0) { writer.setChecksum(partitionChecksums[i]); } partitionWriters[i] = writer; @@ -232,16 +228,9 @@ private long[] writePartitionedData(ShuffleMapOutputWriter mapOutputWriter) thro } partitionWriters = null; } - long[] checksums; - if (checksumEnabled) { - checksums = new long[numPartitions]; - for (int i = 0; i < numPartitions; i ++) { - checksums[i] = partitionChecksums[i].getValue(); - } - } else { - checksums = new long[0]; - } - return mapOutputWriter.commitAllPartitions(checksums).getPartitionLengths(); + return mapOutputWriter.commitAllPartitions( + ShuffleChecksumHelper.getChecksumValues(partitionChecksums) + ).getPartitionLengths(); } private void writePartitionedDataWithChannel( diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleExternalSorter.java b/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleExternalSorter.java index 27f3d9b81d2e..cb5bd4ad08b3 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleExternalSorter.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleExternalSorter.java @@ -112,8 +112,8 @@ final class ShuffleExternalSorter extends MemoryConsumer { @Nullable private MemoryBlock currentPage = null; private long pageCursor = -1; - private final boolean checksumEnabled; - private Checksum[] partitionChecksums; + // Checksum calculator for each partition. Empty when shuffle checksum disabled. + private final Checksum[] partitionChecksums; ShuffleExternalSorter( TaskMemoryManager memoryManager, @@ -145,23 +145,12 @@ final class ShuffleExternalSorter extends MemoryConsumer { this.peakMemoryUsedBytes = getMemoryUsage(); this.diskWriteBufferSize = (int) (long) conf.get(package$.MODULE$.SHUFFLE_DISK_WRITE_BUFFER_SIZE()); - this.checksumEnabled = ShuffleChecksumHelper.isShuffleChecksumEnabled(conf); - if (this.checksumEnabled) { - this.partitionChecksums = ShuffleChecksumHelper.createPartitionChecksums(numPartitions, conf); - } + this.partitionChecksums = + ShuffleChecksumHelper.createPartitionChecksumsIfEnabled(numPartitions, conf); } public long[] getChecksums() { - long[] checksums; - if (checksumEnabled) { - checksums = new long[numPartitions]; - for (int i = 0; i < numPartitions; i ++) { - checksums[i] = partitionChecksums[i].getValue(); - } - } else { - checksums = new long[0]; - } - return checksums; + return ShuffleChecksumHelper.getChecksumValues(partitionChecksums); } /** @@ -233,7 +222,7 @@ private void writeSortedFile(boolean isLastFile) { spillInfo.partitionLengths[currentPartition] = fileSegment.length(); } currentPartition = partition; - if (checksumEnabled) { + if (partitionChecksums.length > 0) { writer.setChecksum(partitionChecksums[currentPartition]); } } 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 f6b0ef15ddd4..0e862948edfb 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 @@ -142,16 +142,10 @@ private[spark] class ExternalSorter[K, V, C]( private val forceSpillFiles = new ArrayBuffer[SpilledFile] @volatile private var readingIterator: SpillableIterator = null - private val checksumEnabled = ShuffleChecksumHelper.isShuffleChecksumEnabled(conf) - private val partitionChecksums = if (checksumEnabled) { - ShuffleChecksumHelper.createPartitionChecksums(numPartitions, conf) - } else { - Array.empty - } + private val partitionChecksums = + ShuffleChecksumHelper.createPartitionChecksumsIfEnabled(numPartitions, conf) - def getChecksums: Array[Long] = { - partitionChecksums.map(_.getValue) - } + def getChecksums: Array[Long] = ShuffleChecksumHelper.getChecksumValues(partitionChecksums) // A comparator for keys K that orders them within a partition to allow aggregation or sorting. // Can be a partial ordering by hash code if a total ordering is not provided through by the From 69250ffa7404e4528cd1236818f9ecec16920cfe Mon Sep 17 00:00:00 2001 From: "yi.wu" Date: Tue, 6 Jul 2021 16:33:41 +0800 Subject: [PATCH 20/47] only set checksum for partitionPairsWriter when enabled --- .../org/apache/spark/util/collection/ExternalSorter.scala | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) 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 0e862948edfb..33d154c543c4 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 @@ -769,7 +769,9 @@ private[spark] class ExternalSorter[K, V, C]( serInstance, blockId, context.taskMetrics().shuffleWriteMetrics) - partitionPairsWriter.setChecksum(partitionChecksums(partitionId)) + if (partitionChecksums.nonEmpty) { + partitionPairsWriter.setChecksum(partitionChecksums(partitionId)) + } while (it.hasNext && it.nextPartition() == partitionId) { it.writeNext(partitionPairsWriter) } @@ -794,7 +796,9 @@ private[spark] class ExternalSorter[K, V, C]( serInstance, blockId, context.taskMetrics().shuffleWriteMetrics) - partitionPairsWriter.setChecksum(partitionChecksums(id)) + if (partitionChecksums.nonEmpty) { + partitionPairsWriter.setChecksum(partitionChecksums(id)) + } if (elements.hasNext) { for (elem <- elements) { partitionPairsWriter.write(elem._1, elem._2) From 7102e7fa9a341fca688e17525c769c0f4d7c7494 Mon Sep 17 00:00:00 2001 From: "yi.wu" Date: Tue, 6 Jul 2021 16:45:12 +0800 Subject: [PATCH 21/47] set checksum in ShufflePartitionPairsWriter's constructor --- .../shuffle/ShufflePartitionPairsWriter.scala | 26 +++++-------------- .../util/collection/ExternalSorter.scala | 12 +++------ 2 files changed, 10 insertions(+), 28 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/shuffle/ShufflePartitionPairsWriter.scala b/core/src/main/scala/org/apache/spark/shuffle/ShufflePartitionPairsWriter.scala index c5da2c8b2fc5..9843ae18bba7 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/ShufflePartitionPairsWriter.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/ShufflePartitionPairsWriter.scala @@ -36,7 +36,8 @@ private[spark] class ShufflePartitionPairsWriter( serializerManager: SerializerManager, serializerInstance: SerializerInstance, blockId: BlockId, - writeMetrics: ShuffleWriteMetricsReporter) + writeMetrics: ShuffleWriteMetricsReporter, + checksum: Checksum) extends PairsWriter with Closeable { private var isClosed = false @@ -46,23 +47,9 @@ private[spark] class ShufflePartitionPairsWriter( private var objOut: SerializationStream = _ private var numRecordsWritten = 0 private var curNumBytesWritten = 0L - - // checksum related - private var checksumEnabled = false + // this would be only initialized when checksum != null, + // which indicates shuffle checksum is enabled. private var checksumOutputStream: MutableCheckedOutputStream = _ - private var checksum: Checksum = _ - - /** - * Set the checksum that the checksumOutputStream should use - */ - def setChecksum(checksum: Checksum): Unit = { - if (checksumOutputStream == null) { - this.checksumEnabled = true - this.checksum = checksum - } else { - checksumOutputStream.setChecksum(checksum) - } - } override def write(key: Any, value: Any): Unit = { if (isClosed) { @@ -80,13 +67,12 @@ private[spark] class ShufflePartitionPairsWriter( try { partitionStream = partitionWriter.openStream timeTrackingStream = new TimeTrackingOutputStream(writeMetrics, partitionStream) - if (checksumEnabled) { - assert(this.checksum != null, "Checksum is not set") + if (checksum != null) { checksumOutputStream = new MutableCheckedOutputStream(timeTrackingStream) checksumOutputStream.setChecksum(checksum) } wrappedStream = serializerManager.wrapStream(blockId, - if (checksumEnabled) checksumOutputStream else timeTrackingStream) + if (checksumOutputStream != null) checksumOutputStream else timeTrackingStream) objOut = serializerInstance.serializeStream(wrappedStream) } catch { case e: Exception => 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 33d154c543c4..dba9e749a573 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 @@ -768,10 +768,8 @@ private[spark] class ExternalSorter[K, V, C]( serializerManager, serInstance, blockId, - context.taskMetrics().shuffleWriteMetrics) - if (partitionChecksums.nonEmpty) { - partitionPairsWriter.setChecksum(partitionChecksums(partitionId)) - } + context.taskMetrics().shuffleWriteMetrics, + if (partitionChecksums.nonEmpty) partitionChecksums(partitionId) else null) while (it.hasNext && it.nextPartition() == partitionId) { it.writeNext(partitionPairsWriter) } @@ -795,10 +793,8 @@ private[spark] class ExternalSorter[K, V, C]( serializerManager, serInstance, blockId, - context.taskMetrics().shuffleWriteMetrics) - if (partitionChecksums.nonEmpty) { - partitionPairsWriter.setChecksum(partitionChecksums(id)) - } + context.taskMetrics().shuffleWriteMetrics, + if (partitionChecksums.nonEmpty) partitionChecksums(id) else null) if (elements.hasNext) { for (elem <- elements) { partitionPairsWriter.write(elem._1, elem._2) From 02f074abde970c608de44a87ee0e736fb6f9a7ac Mon Sep 17 00:00:00 2001 From: "yi.wu" Date: Tue, 6 Jul 2021 16:46:19 +0800 Subject: [PATCH 22/47] remove unused mapId&shuffleId --- .../org/apache/spark/shuffle/sort/ShuffleExternalSorter.java | 4 ---- 1 file changed, 4 deletions(-) diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleExternalSorter.java b/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleExternalSorter.java index cb5bd4ad08b3..8a36b1155929 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleExternalSorter.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleExternalSorter.java @@ -75,8 +75,6 @@ final class ShuffleExternalSorter extends MemoryConsumer { @VisibleForTesting static final int DISK_WRITE_BUFFER_SIZE = 1024 * 1024; - private final int shuffleId; - private final long mapId; private final int numPartitions; private final TaskMemoryManager taskMemoryManager; private final BlockManager blockManager; @@ -131,8 +129,6 @@ final class ShuffleExternalSorter extends MemoryConsumer { this.taskMemoryManager = memoryManager; this.blockManager = blockManager; this.taskContext = taskContext; - this.shuffleId = shuffleId; - this.mapId = mapId; this.numPartitions = numPartitions; // Use getSizeAsKb (not bytes) to maintain backwards compatibility if no units are provided this.fileBufferSizeBytes = From dcc8dde81ac05ab10dde66a6d4dae890d5be5aa9 Mon Sep 17 00:00:00 2001 From: "yi.wu" Date: Tue, 6 Jul 2021 16:50:54 +0800 Subject: [PATCH 23/47] insensitive cheksum algo --- .../shuffle/checksum/ShuffleChecksumHelper.java | 15 ++++++++------- .../apache/spark/internal/config/package.scala | 3 ++- 2 files changed, 10 insertions(+), 8 deletions(-) diff --git a/core/src/main/java/org/apache/spark/shuffle/checksum/ShuffleChecksumHelper.java b/core/src/main/java/org/apache/spark/shuffle/checksum/ShuffleChecksumHelper.java index 76db3211c57a..fe9bcf1c9c4d 100644 --- a/core/src/main/java/org/apache/spark/shuffle/checksum/ShuffleChecksumHelper.java +++ b/core/src/main/java/org/apache/spark/shuffle/checksum/ShuffleChecksumHelper.java @@ -1,14 +1,15 @@ package org.apache.spark.shuffle.checksum; +import java.util.Locale; +import java.util.zip.Adler32; +import java.util.zip.CRC32; +import java.util.zip.Checksum; + import org.apache.spark.SparkConf; import org.apache.spark.SparkException; import org.apache.spark.internal.config.package$; import org.apache.spark.storage.ShuffleChecksumBlockId; -import java.util.zip.Adler32; -import java.util.zip.CRC32; -import java.util.zip.Checksum; - public class ShuffleChecksumHelper { public static boolean isShuffleChecksumEnabled(SparkConf conf) { @@ -24,16 +25,16 @@ public static Checksum[] createPartitionChecksumsIfEnabled(int numPartitions, Sp return partitionChecksums; } - String checksumAlgo = shuffleChecksumAlgorithm(conf); + String checksumAlgo = shuffleChecksumAlgorithm(conf).toLowerCase(Locale.ROOT); switch (checksumAlgo) { - case "Adler32": + case "adler32": partitionChecksums = new Adler32[numPartitions]; for (int i = 0; i < numPartitions; i ++) { partitionChecksums[i] = new Adler32(); } return partitionChecksums; - case "CRC32": + case "crc32": partitionChecksums = new CRC32[numPartitions]; for (int i = 0; i < numPartitions; i ++) { partitionChecksums[i] = new CRC32(); diff --git a/core/src/main/scala/org/apache/spark/internal/config/package.scala b/core/src/main/scala/org/apache/spark/internal/config/package.scala index a24d9c1a2d9a..5e8502acd8e4 100644 --- a/core/src/main/scala/org/apache/spark/internal/config/package.scala +++ b/core/src/main/scala/org/apache/spark/internal/config/package.scala @@ -1382,7 +1382,8 @@ package object config { " built-in algorithms of JDK.") .version("3.3.0") .stringConf - .checkValue(Set("Adler32", "CRC32").contains, "Shuffle checksum algorithm " + + .transform(_.toLowerCase(Locale.ROOT)) + .checkValue(Set("adler32", "crc32").contains, "Shuffle checksum algorithm " + "should be either Adler32 or CRC32.") .createWithDefault("Adler32") From ac8025fb27727365999ed6e484a9e08b177e8efe Mon Sep 17 00:00:00 2001 From: "yi.wu" Date: Tue, 6 Jul 2021 17:27:27 +0800 Subject: [PATCH 24/47] refactor existingChecksums --- .../spark/shuffle/IndexShuffleBlockResolver.scala | 11 +++++++---- 1 file changed, 7 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala b/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala index 140a495551c3..0e642b6257b7 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala @@ -348,13 +348,16 @@ private[spark] class IndexShuffleBlockResolver( // the following check and rename are atomic. this.synchronized { val existingLengths = checkIndexAndDataFile(indexFile, dataFile, lengths.length) - val existingChecksums = - checksumFileOpt.map(getChecksums(_, checksums.length)).getOrElse(checksums) - if (existingLengths != null && existingChecksums != null) { + // `existingChecksums` could be null either because we're the first task attempt + // reaches here or shuffle checksum is disabled. + val existingChecksums = getChecksums(checksumFileOpt.get, checksums.length) + if (existingLengths != null && (!checksumEnabled || existingChecksums != null)) { // Another attempt for the same task has already written our map outputs successfully, // so just use the existing partition lengths and delete our temporary map outputs. System.arraycopy(existingLengths, 0, lengths, 0, lengths.length) - System.arraycopy(existingChecksums, 0, checksums, 0, lengths.length) + if (checksumEnabled) { + System.arraycopy(existingChecksums, 0, checksums, 0, lengths.length) + } if (dataTmp != null && dataTmp.exists()) { dataTmp.delete() } From 1e9876d739226e8ef870efb657bed110b96dc0af Mon Sep 17 00:00:00 2001 From: "yi.wu" Date: Mon, 12 Jul 2021 22:32:55 +0800 Subject: [PATCH 25/47] remove unsued shuffleId/mapId --- .../org/apache/spark/shuffle/sort/ShuffleExternalSorter.java | 2 -- 1 file changed, 2 deletions(-) diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleExternalSorter.java b/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleExternalSorter.java index 8a36b1155929..0307027c6f26 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleExternalSorter.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleExternalSorter.java @@ -118,8 +118,6 @@ final class ShuffleExternalSorter extends MemoryConsumer { BlockManager blockManager, TaskContext taskContext, int initialSize, - int shuffleId, - long mapId, int numPartitions, SparkConf conf, ShuffleWriteMetricsReporter writeMetrics) throws SparkException { From 1573fb6742aa972492ccd445ff55febb871afdc0 Mon Sep 17 00:00:00 2001 From: "yi.wu" Date: Mon, 12 Jul 2021 22:37:03 +0800 Subject: [PATCH 26/47] use uppercase --- .../spark/shuffle/checksum/ShuffleChecksumHelper.java | 6 +++--- .../scala/org/apache/spark/internal/config/package.scala | 6 +++--- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/core/src/main/java/org/apache/spark/shuffle/checksum/ShuffleChecksumHelper.java b/core/src/main/java/org/apache/spark/shuffle/checksum/ShuffleChecksumHelper.java index fe9bcf1c9c4d..a0afb8c1ce99 100644 --- a/core/src/main/java/org/apache/spark/shuffle/checksum/ShuffleChecksumHelper.java +++ b/core/src/main/java/org/apache/spark/shuffle/checksum/ShuffleChecksumHelper.java @@ -25,16 +25,16 @@ public static Checksum[] createPartitionChecksumsIfEnabled(int numPartitions, Sp return partitionChecksums; } - String checksumAlgo = shuffleChecksumAlgorithm(conf).toLowerCase(Locale.ROOT); + String checksumAlgo = shuffleChecksumAlgorithm(conf); switch (checksumAlgo) { - case "adler32": + case "ADLER32": partitionChecksums = new Adler32[numPartitions]; for (int i = 0; i < numPartitions; i ++) { partitionChecksums[i] = new Adler32(); } return partitionChecksums; - case "crc32": + case "CRC32": partitionChecksums = new CRC32[numPartitions]; for (int i = 0; i < numPartitions; i ++) { partitionChecksums[i] = new CRC32(); diff --git a/core/src/main/scala/org/apache/spark/internal/config/package.scala b/core/src/main/scala/org/apache/spark/internal/config/package.scala index 5e8502acd8e4..3ef964fcb8fd 100644 --- a/core/src/main/scala/org/apache/spark/internal/config/package.scala +++ b/core/src/main/scala/org/apache/spark/internal/config/package.scala @@ -1382,10 +1382,10 @@ package object config { " built-in algorithms of JDK.") .version("3.3.0") .stringConf - .transform(_.toLowerCase(Locale.ROOT)) - .checkValue(Set("adler32", "crc32").contains, "Shuffle checksum algorithm " + + .transform(_.toUpperCase(Locale.ROOT)) + .checkValue(Set("ADLER32", "CRC32").contains, "Shuffle checksum algorithm " + "should be either Adler32 or CRC32.") - .createWithDefault("Adler32") + .createWithDefault("ADLER32") private[spark] val SHUFFLE_COMPRESS = ConfigBuilder("spark.shuffle.compress") From 5446bb1b529d76fc63b938058341311e43e54f65 Mon Sep 17 00:00:00 2001 From: "yi.wu" Date: Mon, 12 Jul 2021 22:37:48 +0800 Subject: [PATCH 27/47] rename to testhelper --- .../apache/spark/shuffle/sort/UnsafeShuffleWriterSuite.java | 4 ++-- ...leChecksumTester.scala => ShuffleChecksumTestHelper.scala} | 2 +- .../shuffle/sort/BypassMergeSortShuffleWriterSuite.scala | 4 ++-- .../apache/spark/shuffle/sort/SortShuffleWriterSuite.scala | 4 ++-- 4 files changed, 7 insertions(+), 7 deletions(-) rename core/src/test/scala/org/apache/spark/shuffle/{ShuffleChecksumTester.scala => ShuffleChecksumTestHelper.scala} (98%) diff --git a/core/src/test/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriterSuite.java b/core/src/test/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriterSuite.java index 71a3459bad2e..b2bd9a78834e 100644 --- a/core/src/test/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriterSuite.java +++ b/core/src/test/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriterSuite.java @@ -23,7 +23,7 @@ import java.util.*; import org.apache.spark.*; -import org.apache.spark.shuffle.ShuffleChecksumTester; +import org.apache.spark.shuffle.ShuffleChecksumTestHelper; import org.mockito.stubbing.Answer; import scala.*; import scala.collection.Iterator; @@ -60,7 +60,7 @@ import static org.mockito.Answers.RETURNS_SMART_NULLS; import static org.mockito.Mockito.*; -public class UnsafeShuffleWriterSuite implements ShuffleChecksumTester { +public class UnsafeShuffleWriterSuite implements ShuffleChecksumTestHelper { static final int DEFAULT_INITIAL_SORT_BUFFER_SIZE = 4096; static final int NUM_PARTITIONS = 4; diff --git a/core/src/test/scala/org/apache/spark/shuffle/ShuffleChecksumTester.scala b/core/src/test/scala/org/apache/spark/shuffle/ShuffleChecksumTestHelper.scala similarity index 98% rename from core/src/test/scala/org/apache/spark/shuffle/ShuffleChecksumTester.scala rename to core/src/test/scala/org/apache/spark/shuffle/ShuffleChecksumTestHelper.scala index b3470fbdcc99..85b5e1df075d 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/ShuffleChecksumTester.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/ShuffleChecksumTestHelper.scala @@ -22,7 +22,7 @@ import java.util.zip.{Adler32, CheckedInputStream} import org.apache.spark.network.util.LimitedInputStream -trait ShuffleChecksumTester { +trait ShuffleChecksumTestHelper { /** * Ensure that the checksum values are consistent between write and read side. diff --git a/core/src/test/scala/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriterSuite.scala b/core/src/test/scala/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriterSuite.scala index 8bb604f1b47f..87d0994dc1f5 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriterSuite.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriterSuite.scala @@ -33,7 +33,7 @@ import org.apache.spark._ import org.apache.spark.executor.{ShuffleWriteMetrics, TaskMetrics} import org.apache.spark.memory.{TaskMemoryManager, TestMemoryManager} import org.apache.spark.serializer.{JavaSerializer, SerializerInstance, SerializerManager} -import org.apache.spark.shuffle.{IndexShuffleBlockResolver, ShuffleChecksumTester} +import org.apache.spark.shuffle.{IndexShuffleBlockResolver, ShuffleChecksumTestHelper} import org.apache.spark.shuffle.api.ShuffleExecutorComponents import org.apache.spark.shuffle.sort.io.LocalDiskShuffleExecutorComponents import org.apache.spark.storage._ @@ -42,7 +42,7 @@ import org.apache.spark.util.Utils class BypassMergeSortShuffleWriterSuite extends SparkFunSuite with BeforeAndAfterEach - with ShuffleChecksumTester { + with ShuffleChecksumTestHelper { @Mock(answer = RETURNS_SMART_NULLS) private var blockManager: BlockManager = _ @Mock(answer = RETURNS_SMART_NULLS) private var diskBlockManager: DiskBlockManager = _ 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 index 6e98bc008953..e3457367d9ba 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/sort/SortShuffleWriterSuite.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/sort/SortShuffleWriterSuite.scala @@ -26,7 +26,7 @@ import org.scalatest.matchers.must.Matchers import org.apache.spark.{Aggregator, DebugFilesystem, Partitioner, SharedSparkContext, ShuffleDependency, SparkContext, SparkFunSuite} import org.apache.spark.memory.MemoryTestingUtils import org.apache.spark.serializer.JavaSerializer -import org.apache.spark.shuffle.{BaseShuffleHandle, IndexShuffleBlockResolver, ShuffleChecksumTester} +import org.apache.spark.shuffle.{BaseShuffleHandle, IndexShuffleBlockResolver, ShuffleChecksumTestHelper} import org.apache.spark.shuffle.api.ShuffleExecutorComponents import org.apache.spark.shuffle.sort.io.LocalDiskShuffleExecutorComponents import org.apache.spark.storage.BlockManager @@ -38,7 +38,7 @@ class SortShuffleWriterSuite with SharedSparkContext with Matchers with PrivateMethodTester - with ShuffleChecksumTester { + with ShuffleChecksumTestHelper { @Mock(answer = RETURNS_SMART_NULLS) private var blockManager: BlockManager = _ From 0908076ae559c67d8a6ed1380311bf6eac4c7353 Mon Sep 17 00:00:00 2001 From: "yi.wu" Date: Mon, 12 Jul 2021 23:01:35 +0800 Subject: [PATCH 28/47] pick the checksum by index file ext --- .../checksum/ShuffleChecksumHelper.java | 32 +++++++++++++------ .../shuffle/ShuffleChecksumTestHelper.scala | 4 ++- 2 files changed, 25 insertions(+), 11 deletions(-) diff --git a/core/src/main/java/org/apache/spark/shuffle/checksum/ShuffleChecksumHelper.java b/core/src/main/java/org/apache/spark/shuffle/checksum/ShuffleChecksumHelper.java index a0afb8c1ce99..dbeb53aad822 100644 --- a/core/src/main/java/org/apache/spark/shuffle/checksum/ShuffleChecksumHelper.java +++ b/core/src/main/java/org/apache/spark/shuffle/checksum/ShuffleChecksumHelper.java @@ -26,23 +26,29 @@ public static Checksum[] createPartitionChecksumsIfEnabled(int numPartitions, Sp } String checksumAlgo = shuffleChecksumAlgorithm(conf); - switch (checksumAlgo) { + return getChecksumByAlgorithm(numPartitions, checksumAlgo); + } + + private static Checksum[] getChecksumByAlgorithm(int num, String algorithm) + throws SparkException { + Checksum[] checksums; + switch (algorithm) { case "ADLER32": - partitionChecksums = new Adler32[numPartitions]; - for (int i = 0; i < numPartitions; i ++) { - partitionChecksums[i] = new Adler32(); + checksums = new Adler32[num]; + for (int i = 0; i < num; i ++) { + checksums[i] = new Adler32(); } - return partitionChecksums; + return checksums; case "CRC32": - partitionChecksums = new CRC32[numPartitions]; - for (int i = 0; i < numPartitions; i ++) { - partitionChecksums[i] = new CRC32(); + checksums = new CRC32[num]; + for (int i = 0; i < num; i ++) { + checksums[i] = new CRC32(); } - return partitionChecksums; + return checksums; default: - throw new SparkException("Unsupported shuffle checksum algorithm: " + checksumAlgo); + throw new SparkException("Unsupported shuffle checksum algorithm: " + algorithm); } } @@ -59,6 +65,12 @@ public static String shuffleChecksumAlgorithm(SparkConf conf) { return conf.get(package$.MODULE$.SHUFFLE_CHECKSUM_ALGORITHM()); } + public static Checksum getChecksumByFileExtension(String fileName) throws SparkException { + int index = fileName.lastIndexOf("."); + String algorithm = fileName.substring(index + 1); + return getChecksumByAlgorithm(1, algorithm)[1]; + } + public static String getChecksumFileName(ShuffleChecksumBlockId blockId, SparkConf conf) { // append the shuffle checksum algorithm as the file extension return String.format("%s.%s", blockId.name(), shuffleChecksumAlgorithm(conf)); diff --git a/core/src/test/scala/org/apache/spark/shuffle/ShuffleChecksumTestHelper.scala b/core/src/test/scala/org/apache/spark/shuffle/ShuffleChecksumTestHelper.scala index 85b5e1df075d..005499c422a5 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/ShuffleChecksumTestHelper.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/ShuffleChecksumTestHelper.scala @@ -21,6 +21,7 @@ import java.io.{DataInputStream, File, FileInputStream} import java.util.zip.{Adler32, CheckedInputStream} import org.apache.spark.network.util.LimitedInputStream +import org.apache.spark.shuffle.checksum.ShuffleChecksumHelper trait ShuffleChecksumTestHelper { @@ -54,8 +55,9 @@ trait ShuffleChecksumTestHelper { val curOffset = indexIn.readLong val limit = (curOffset - prevOffset).toInt val bytes = new Array[Byte](limit) + val checksum = ShuffleChecksumHelper.getChecksumByFileExtension(index.getName) checkedIn = new CheckedInputStream( - new LimitedInputStream(dataIn, curOffset - prevOffset), new Adler32) + new LimitedInputStream(dataIn, curOffset - prevOffset), checksum) checkedIn.read(bytes, 0, limit) prevOffset = curOffset // checksum must be consistent at both write and read sides From 09f901509f4c23fcead4a5b71976ad484ed3efbe Mon Sep 17 00:00:00 2001 From: "yi.wu" Date: Mon, 12 Jul 2021 23:03:34 +0800 Subject: [PATCH 29/47] remove unused import --- .../org/apache/spark/shuffle/checksum/ShuffleChecksumHelper.java | 1 - 1 file changed, 1 deletion(-) diff --git a/core/src/main/java/org/apache/spark/shuffle/checksum/ShuffleChecksumHelper.java b/core/src/main/java/org/apache/spark/shuffle/checksum/ShuffleChecksumHelper.java index dbeb53aad822..5ebb15ddc85f 100644 --- a/core/src/main/java/org/apache/spark/shuffle/checksum/ShuffleChecksumHelper.java +++ b/core/src/main/java/org/apache/spark/shuffle/checksum/ShuffleChecksumHelper.java @@ -1,6 +1,5 @@ package org.apache.spark.shuffle.checksum; -import java.util.Locale; import java.util.zip.Adler32; import java.util.zip.CRC32; import java.util.zip.Checksum; From 62b479e145b2762f2298f2433a663ca88bd3a644 Mon Sep 17 00:00:00 2001 From: "yi.wu" Date: Mon, 12 Jul 2021 23:08:09 +0800 Subject: [PATCH 30/47] pull empty checksum as a static final value --- .../apache/spark/shuffle/checksum/ShuffleChecksumHelper.java | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/core/src/main/java/org/apache/spark/shuffle/checksum/ShuffleChecksumHelper.java b/core/src/main/java/org/apache/spark/shuffle/checksum/ShuffleChecksumHelper.java index 5ebb15ddc85f..64cd7568c533 100644 --- a/core/src/main/java/org/apache/spark/shuffle/checksum/ShuffleChecksumHelper.java +++ b/core/src/main/java/org/apache/spark/shuffle/checksum/ShuffleChecksumHelper.java @@ -11,6 +11,9 @@ public class ShuffleChecksumHelper { + /** Used when the checksum is disabled for shuffle. */ + private static final Checksum[] EMPTY_CHECKSUM = new Checksum[0]; + public static boolean isShuffleChecksumEnabled(SparkConf conf) { return (boolean) conf.get(package$.MODULE$.SHUFFLE_CHECKSUM_ENABLED()); } @@ -20,7 +23,7 @@ public static Checksum[] createPartitionChecksumsIfEnabled(int numPartitions, Sp Checksum[] partitionChecksums; if (!isShuffleChecksumEnabled(conf)) { - partitionChecksums = new Checksum[0]; + partitionChecksums = EMPTY_CHECKSUM; return partitionChecksums; } From d71283c1284ff73703b1da428534d82cc86abccc Mon Sep 17 00:00:00 2001 From: "yi.wu" Date: Mon, 12 Jul 2021 23:15:07 +0800 Subject: [PATCH 31/47] pull empty checksum value into a static final value --- .../apache/spark/shuffle/checksum/ShuffleChecksumHelper.java | 1 + .../spark/shuffle/sort/BypassMergeSortShuffleWriter.java | 3 ++- .../org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java | 4 +++- 3 files changed, 6 insertions(+), 2 deletions(-) diff --git a/core/src/main/java/org/apache/spark/shuffle/checksum/ShuffleChecksumHelper.java b/core/src/main/java/org/apache/spark/shuffle/checksum/ShuffleChecksumHelper.java index 64cd7568c533..c69b2ddfee0a 100644 --- a/core/src/main/java/org/apache/spark/shuffle/checksum/ShuffleChecksumHelper.java +++ b/core/src/main/java/org/apache/spark/shuffle/checksum/ShuffleChecksumHelper.java @@ -13,6 +13,7 @@ public class ShuffleChecksumHelper { /** Used when the checksum is disabled for shuffle. */ private static final Checksum[] EMPTY_CHECKSUM = new Checksum[0]; + public static final long[] EMPTY_CHECKSUM_VALUE = new long[0]; public static boolean isShuffleChecksumEnabled(SparkConf conf) { return (boolean) conf.get(package$.MODULE$.SHUFFLE_CHECKSUM_ENABLED()); diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java b/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java index 12dfb8f5f4b6..b8822d7e1325 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java @@ -135,7 +135,8 @@ public void write(Iterator> records) throws IOException { .createMapOutputWriter(shuffleId, mapId, numPartitions); try { if (!records.hasNext()) { - partitionLengths = mapOutputWriter.commitAllPartitions(new long[0]).getPartitionLengths(); + partitionLengths = mapOutputWriter.commitAllPartitions( + ShuffleChecksumHelper.EMPTY_CHECKSUM_VALUE).getPartitionLengths(); mapStatus = MapStatus$.MODULE$.apply( blockManager.shuffleServerId(), partitionLengths, mapId); return; diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java b/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java index a49bcc3696fe..e60e3bf0466e 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java @@ -57,6 +57,7 @@ import org.apache.spark.shuffle.api.ShufflePartitionWriter; import org.apache.spark.shuffle.api.SingleSpillShuffleMapOutputWriter; import org.apache.spark.shuffle.api.WritableByteChannelWrapper; +import org.apache.spark.shuffle.checksum.ShuffleChecksumHelper; import org.apache.spark.storage.BlockManager; import org.apache.spark.storage.TimeTrackingOutputStream; import org.apache.spark.unsafe.Platform; @@ -269,7 +270,8 @@ private long[] mergeSpills(SpillInfo[] spills) throws IOException { if (spills.length == 0) { final ShuffleMapOutputWriter mapWriter = shuffleExecutorComponents .createMapOutputWriter(shuffleId, mapId, partitioner.numPartitions()); - return mapWriter.commitAllPartitions(new long[0]).getPartitionLengths(); + return mapWriter.commitAllPartitions( + ShuffleChecksumHelper.EMPTY_CHECKSUM_VALUE).getPartitionLengths(); } else if (spills.length == 1) { Optional maybeSingleFileWriter = shuffleExecutorComponents.createSingleFileMapOutputWriter(shuffleId, mapId); From 302ca76db433dc3ea4dcab8294329ad2cfc405b1 Mon Sep 17 00:00:00 2001 From: "yi.wu" Date: Mon, 12 Jul 2021 23:16:01 +0800 Subject: [PATCH 32/47] fix unsafe write due to shuffleId/map --- .../java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java | 2 -- 1 file changed, 2 deletions(-) diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java b/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java index e60e3bf0466e..2659b172bf68 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java @@ -206,8 +206,6 @@ private void open() throws SparkException { blockManager, taskContext, initialSortBufferSize, - shuffleId, - mapId, partitioner.numPartitions(), sparkConf, writeMetrics); From b81e5567d689182dbab8acd439e9a4a8143f51d0 Mon Sep 17 00:00:00 2001 From: "yi.wu" Date: Tue, 13 Jul 2021 11:46:53 +0800 Subject: [PATCH 33/47] update comment --- .../org/apache/spark/shuffle/api/ShuffleMapOutputWriter.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/java/org/apache/spark/shuffle/api/ShuffleMapOutputWriter.java b/core/src/main/java/org/apache/spark/shuffle/api/ShuffleMapOutputWriter.java index 22b6e13196da..daa60028e7ad 100644 --- a/core/src/main/java/org/apache/spark/shuffle/api/ShuffleMapOutputWriter.java +++ b/core/src/main/java/org/apache/spark/shuffle/api/ShuffleMapOutputWriter.java @@ -59,7 +59,7 @@ public interface ShuffleMapOutputWriter { * available to downstream reduce tasks. If this method throws any exception, this module's * {@link #abort(Throwable)} method will be invoked before propagating the exception. *

- * Shuffle extension who cares about the cause of shuffle data corruption should store + * Shuffle extensions which care about the cause of shuffle data corruption should store * the {@param checksums} properly. When corruption happens, Spark would provide the checksum * of the fetched partition to the shuffle extension to help diagnose the cause of corruption. *

From eeb3ef751bccdf42ee65c4b1ddb1694923da2957 Mon Sep 17 00:00:00 2001 From: "yi.wu" Date: Tue, 13 Jul 2021 11:51:35 +0800 Subject: [PATCH 34/47] add doc for ShuffleChecksumHelper --- .../apache/spark/shuffle/checksum/ShuffleChecksumHelper.java | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/core/src/main/java/org/apache/spark/shuffle/checksum/ShuffleChecksumHelper.java b/core/src/main/java/org/apache/spark/shuffle/checksum/ShuffleChecksumHelper.java index c69b2ddfee0a..1738cb312d16 100644 --- a/core/src/main/java/org/apache/spark/shuffle/checksum/ShuffleChecksumHelper.java +++ b/core/src/main/java/org/apache/spark/shuffle/checksum/ShuffleChecksumHelper.java @@ -6,9 +6,14 @@ import org.apache.spark.SparkConf; import org.apache.spark.SparkException; +import org.apache.spark.annotation.Private; import org.apache.spark.internal.config.package$; import org.apache.spark.storage.ShuffleChecksumBlockId; +/** + * A set of utility functions for the shuffle checksum. + */ +@Private public class ShuffleChecksumHelper { /** Used when the checksum is disabled for shuffle. */ From bf3ca61dc9137e66355c72f1522d104aa2112f8d Mon Sep 17 00:00:00 2001 From: "yi.wu" Date: Tue, 13 Jul 2021 11:52:32 +0800 Subject: [PATCH 35/47] remove unncessary partitionChecksums --- .../apache/spark/shuffle/checksum/ShuffleChecksumHelper.java | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) diff --git a/core/src/main/java/org/apache/spark/shuffle/checksum/ShuffleChecksumHelper.java b/core/src/main/java/org/apache/spark/shuffle/checksum/ShuffleChecksumHelper.java index 1738cb312d16..91e7b060d052 100644 --- a/core/src/main/java/org/apache/spark/shuffle/checksum/ShuffleChecksumHelper.java +++ b/core/src/main/java/org/apache/spark/shuffle/checksum/ShuffleChecksumHelper.java @@ -26,11 +26,8 @@ public static boolean isShuffleChecksumEnabled(SparkConf conf) { public static Checksum[] createPartitionChecksumsIfEnabled(int numPartitions, SparkConf conf) throws SparkException { - Checksum[] partitionChecksums; - if (!isShuffleChecksumEnabled(conf)) { - partitionChecksums = EMPTY_CHECKSUM; - return partitionChecksums; + return EMPTY_CHECKSUM; } String checksumAlgo = shuffleChecksumAlgorithm(conf); From d8c70dc8f0a0d2190f2975bc03dc4857202d259b Mon Sep 17 00:00:00 2001 From: "yi.wu" Date: Tue, 13 Jul 2021 11:53:20 +0800 Subject: [PATCH 36/47] add since for ShuffleChecksumBlockId --- core/src/main/scala/org/apache/spark/storage/BlockId.scala | 1 + 1 file changed, 1 insertion(+) diff --git a/core/src/main/scala/org/apache/spark/storage/BlockId.scala b/core/src/main/scala/org/apache/spark/storage/BlockId.scala index 806ce966181f..db5862dec2fb 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockId.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockId.scala @@ -92,6 +92,7 @@ case class ShuffleIndexBlockId(shuffleId: Int, mapId: Long, reduceId: Int) exten override def name: String = "shuffle_" + shuffleId + "_" + mapId + "_" + reduceId + ".index" } +@Since("3.3.0") @DeveloperApi case class ShuffleChecksumBlockId(shuffleId: Int, mapId: Long, reduceId: Int) extends BlockId { override def name: String = "shuffle_" + shuffleId + "_" + mapId + "_" + reduceId + ".checksum" From 71a2ef61780c5d3c4744a263ae2ffdb3e53e0729 Mon Sep 17 00:00:00 2001 From: "yi.wu" Date: Tue, 13 Jul 2021 12:07:13 +0800 Subject: [PATCH 37/47] handle error of checksum file delete --- .../spark/shuffle/IndexShuffleBlockResolver.scala | 15 +++++++++++++-- 1 file changed, 13 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala b/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala index 0e642b6257b7..0a574b9b904a 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala @@ -420,8 +420,19 @@ private[spark] class IndexShuffleBlockResolver( logError(s"Failed to delete temporary index file at ${indexTmp.getAbsolutePath}") } checksumTmpOpt.foreach { checksumTmp => - if (checksumTmp.exists() && !checksumTmp.delete()) { - logError(s"Failed to delete temporary checksum file at ${checksumTmp.getAbsolutePath}") + if (checksumTmp.exists()) { + try { + if (!checksumTmp.delete()) { + logError(s"Failed to delete temporary checksum file " + + s"at ${checksumTmp.getAbsolutePath}") + } + } catch { + case e: Exception => + // Unlike index deletion, we won't propagate the error for the checksum file since + // checksum is only a best-effort. + logError(s"Failed to delete temporary checksum file " + + s"at ${checksumTmp.getAbsolutePath}", e) + } } } } From 63df5cda4afe5e7c018f5fb635a5034fec9909ea Mon Sep 17 00:00:00 2001 From: "yi.wu" Date: Tue, 13 Jul 2021 12:07:57 +0800 Subject: [PATCH 38/47] update comment --- .../org/apache/spark/shuffle/IndexShuffleBlockResolver.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala b/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala index 0a574b9b904a..9ec59afaed2a 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala @@ -408,7 +408,7 @@ private[spark] class IndexShuffleBlockResolver( } if (!checksumTmp.renameTo(checksumFile)) { // It's not worthwhile to fail here after index file and data file are already - // successfully stored due to checksum is only used for the corner error case. + // successfully stored since checksum is only a best-effort for the corner error case. logWarning("fail to rename file " + checksumTmp + " to " + checksumFile) } } From 11adda4a92fc379debd7a381b91125ff8685f54e Mon Sep 17 00:00:00 2001 From: "yi.wu" Date: Tue, 13 Jul 2021 16:19:09 +0800 Subject: [PATCH 39/47] update writeMetadataFileAndCommit --- .../shuffle/IndexShuffleBlockResolver.scala | 104 +++++++++++------- 1 file changed, 63 insertions(+), 41 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala b/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala index 9ec59afaed2a..9c50569c7892 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala @@ -348,15 +348,21 @@ private[spark] class IndexShuffleBlockResolver( // the following check and rename are atomic. this.synchronized { val existingLengths = checkIndexAndDataFile(indexFile, dataFile, lengths.length) - // `existingChecksums` could be null either because we're the first task attempt - // reaches here or shuffle checksum is disabled. - val existingChecksums = getChecksums(checksumFileOpt.get, checksums.length) - if (existingLengths != null && (!checksumEnabled || existingChecksums != null)) { + if (existingLengths != null) { // Another attempt for the same task has already written our map outputs successfully, // so just use the existing partition lengths and delete our temporary map outputs. System.arraycopy(existingLengths, 0, lengths, 0, lengths.length) if (checksumEnabled) { - System.arraycopy(existingChecksums, 0, checksums, 0, lengths.length) + val existingChecksums = getChecksums(checksumFileOpt.get, checksums.length) + if (existingChecksums != null) { + System.arraycopy(existingChecksums, 0, checksums, 0, lengths.length) + } else { + // It's possible that the previous task attempt succeeded writing the + // index file and data file but failed to write the checksum file. In + // this case, the current task attempt could write the missing checksum + // file by itself. + writeMetadataFile(checksums, checksumTmpOpt.get, checksumFileOpt.get, false) + } } if (dataTmp != null && dataTmp.exists()) { dataTmp.delete() @@ -364,52 +370,27 @@ private[spark] class IndexShuffleBlockResolver( } else { // This is the first successful attempt in writing the map outputs for this task, // so override any existing index and data files with the ones we wrote. - val out = new DataOutputStream(new BufferedOutputStream(new FileOutputStream(indexTmp))) - Utils.tryWithSafeFinally { - // We take in lengths of each block, need to convert it to offsets. - var offset = 0L - out.writeLong(offset) - for (length <- lengths) { - offset += length - out.writeLong(offset) - } - } { - out.close() - } - if (indexFile.exists()) { - indexFile.delete() - } + val offsets = lengths.scanLeft(0L)(_ + _) + writeMetadataFile(offsets, indexTmp, indexFile, true) + if (dataFile.exists()) { dataFile.delete() } - if (!indexTmp.renameTo(indexFile)) { - throw new IOException("fail to rename file " + indexTmp + " to " + indexFile) - } if (dataTmp != null && dataTmp.exists() && !dataTmp.renameTo(dataFile)) { throw new IOException("fail to rename file " + dataTmp + " to " + dataFile) } // write the checksum file checksumTmpOpt.zip(checksumFileOpt).foreach { case (checksumTmp, checksumFile) => - val out = new DataOutputStream( - new BufferedOutputStream( - new FileOutputStream(checksumTmp) - ) - ) - Utils.tryWithSafeFinally { - checksums.foreach(out.writeLong) - } { - out.close() - } - - if (checksumFile.exists()) { - checksumFile.delete() - } - if (!checksumTmp.renameTo(checksumFile)) { - // It's not worthwhile to fail here after index file and data file are already - // successfully stored since checksum is only a best-effort for the corner error case. - logWarning("fail to rename file " + checksumTmp + " to " + checksumFile) + try { + writeMetadataFile(checksums, checksumTmp, checksumFile, false) + } catch { + case e: Exception => + // It's not worthwhile to fail here after index file and data file are + // already successfully stored since checksum is only a best-effort for + // the corner error case. + logError("Failed to write checksum file", e) } } } @@ -438,6 +419,47 @@ private[spark] class IndexShuffleBlockResolver( } } + /** + * Write the metadata file (index or checksum). Metadata values will be firstly write into + * the tmp file and the tmp file will be renamed to the target file at the end to avoid dirty + * writes. + * @param metaValues The metadata values + * @param tmpFile The temp file + * @param targetFile The target file + * @param propagateError Whether to propagate the error for file operation. Unlike index file, + * checksum is only a best-effort so we won't fail the whole task due to + * the error from checksum. + */ + private def writeMetadataFile( + metaValues: Array[Long], + tmpFile: File, + targetFile: File, + propagateError: Boolean): Unit = { + val out = new DataOutputStream( + new BufferedOutputStream( + new FileOutputStream(tmpFile) + ) + ) + Utils.tryWithSafeFinally { + metaValues.foreach(out.writeLong) + } { + out.close() + } + + if (targetFile.exists()) { + targetFile.delete() + } + + if (!tmpFile.renameTo(targetFile)) { + val errorMsg = s"fail to rename file $tmpFile to $targetFile" + if (propagateError) { + throw new IOException(errorMsg) + } else { + logWarning(errorMsg) + } + } + } + /** * This is only used for reading local merged block data. In such cases, all chunks in the * merged shuffle file need to be identified at once, so the ShuffleBlockFetcherIterator From 4da61d916b9c3502b53e5028e2895d8759fa8f68 Mon Sep 17 00:00:00 2001 From: "yi.wu" Date: Tue, 13 Jul 2021 23:35:17 +0800 Subject: [PATCH 40/47] fix --- .../apache/spark/shuffle/sort/ShuffleExternalSorterSuite.scala | 2 -- 1 file changed, 2 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/shuffle/sort/ShuffleExternalSorterSuite.scala b/core/src/test/scala/org/apache/spark/shuffle/sort/ShuffleExternalSorterSuite.scala index 8ae912de6ae1..b33708d24e7c 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/sort/ShuffleExternalSorterSuite.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/sort/ShuffleExternalSorterSuite.scala @@ -74,8 +74,6 @@ class ShuffleExternalSorterSuite extends SparkFunSuite with LocalSparkContext wi sc.env.blockManager, taskContext, 100, // initialSize - This will require ShuffleInMemorySorter to acquire at least 800 bytes - 0, - 0, 1, // numPartitions conf, new ShuffleWriteMetrics) From 8d54e38c1df2418aca77ccd97cf05939aa6ed19f Mon Sep 17 00:00:00 2001 From: "yi.wu" Date: Wed, 14 Jul 2021 10:35:07 +0800 Subject: [PATCH 41/47] fix compile --- .../org/apache/spark/shuffle/ShuffleChecksumTestHelper.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/test/scala/org/apache/spark/shuffle/ShuffleChecksumTestHelper.scala b/core/src/test/scala/org/apache/spark/shuffle/ShuffleChecksumTestHelper.scala index 005499c422a5..6637ccb8e6a2 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/ShuffleChecksumTestHelper.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/ShuffleChecksumTestHelper.scala @@ -18,7 +18,7 @@ package org.apache.spark.shuffle import java.io.{DataInputStream, File, FileInputStream} -import java.util.zip.{Adler32, CheckedInputStream} +import java.util.zip.CheckedInputStream import org.apache.spark.network.util.LimitedInputStream import org.apache.spark.shuffle.checksum.ShuffleChecksumHelper From dfbe4b6f6eaf5c52c6f054c65ebed293a6f99ee8 Mon Sep 17 00:00:00 2001 From: "yi.wu" Date: Wed, 14 Jul 2021 12:50:39 +0800 Subject: [PATCH 42/47] add license --- .../shuffle/checksum/ShuffleChecksumHelper.java | 17 +++++++++++++++++ 1 file changed, 17 insertions(+) diff --git a/core/src/main/java/org/apache/spark/shuffle/checksum/ShuffleChecksumHelper.java b/core/src/main/java/org/apache/spark/shuffle/checksum/ShuffleChecksumHelper.java index 91e7b060d052..c27586020ac7 100644 --- a/core/src/main/java/org/apache/spark/shuffle/checksum/ShuffleChecksumHelper.java +++ b/core/src/main/java/org/apache/spark/shuffle/checksum/ShuffleChecksumHelper.java @@ -1,3 +1,20 @@ +/* + * 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.checksum; import java.util.zip.Adler32; From 230648ce0dcc1762d9690aecfd63568721f82e4e Mon Sep 17 00:00:00 2001 From: "yi.wu" Date: Wed, 14 Jul 2021 12:55:58 +0800 Subject: [PATCH 43/47] fix mimia --- project/MimaExcludes.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index 01b7bed4f280..04694724ca38 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -102,7 +102,8 @@ object MimaExcludes { ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.shuffle.sort.io.LocalDiskSingleSpillMapOutputWriter.transferMapSpillFile"), ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.shuffle.api.ShuffleMapOutputWriter.commitAllPartitions"), ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.shuffle.api.SingleSpillShuffleMapOutputWriter.transferMapSpillFile"), - ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.shuffle.api.SingleSpillShuffleMapOutputWriter.transferMapSpillFile") + ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.shuffle.api.SingleSpillShuffleMapOutputWriter.transferMapSpillFile"), + ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.shuffle.api.ShuffleMapOutputWriter.commitAllPartitions") ) def excludes(version: String) = version match { From 68af8c4110ed8355db48a16eb6fe7e9fb0dd5f4b Mon Sep 17 00:00:00 2001 From: "yi.wu" Date: Thu, 15 Jul 2021 10:21:55 +0800 Subject: [PATCH 44/47] fix java lint --- .../spark/shuffle/sort/BypassMergeSortShuffleWriter.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java b/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java index b8822d7e1325..322224053df0 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java @@ -125,7 +125,8 @@ final class BypassMergeSortShuffleWriter extends ShuffleWriter { this.writeMetrics = writeMetrics; this.serializer = dep.serializer(); this.shuffleExecutorComponents = shuffleExecutorComponents; - this.partitionChecksums = ShuffleChecksumHelper.createPartitionChecksumsIfEnabled(numPartitions, conf); + this.partitionChecksums = + ShuffleChecksumHelper.createPartitionChecksumsIfEnabled(numPartitions, conf); } @Override From caaf76db3e462651f902fb43999953ba0292df15 Mon Sep 17 00:00:00 2001 From: "yi.wu" Date: Thu, 15 Jul 2021 10:56:57 +0800 Subject: [PATCH 45/47] fix doc --- .../apache/spark/shuffle/api/ShuffleMapOutputWriter.java | 2 +- .../apache/spark/shuffle/api/ShufflePartitionWriter.java | 6 +++--- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/core/src/main/java/org/apache/spark/shuffle/api/ShuffleMapOutputWriter.java b/core/src/main/java/org/apache/spark/shuffle/api/ShuffleMapOutputWriter.java index daa60028e7ad..2237ec052cac 100644 --- a/core/src/main/java/org/apache/spark/shuffle/api/ShuffleMapOutputWriter.java +++ b/core/src/main/java/org/apache/spark/shuffle/api/ShuffleMapOutputWriter.java @@ -60,7 +60,7 @@ public interface ShuffleMapOutputWriter { * {@link #abort(Throwable)} method will be invoked before propagating the exception. *

* Shuffle extensions which care about the cause of shuffle data corruption should store - * the {@param checksums} properly. When corruption happens, Spark would provide the checksum + * the checksums properly. When corruption happens, Spark would provide the checksum * of the fetched partition to the shuffle extension to help diagnose the cause of corruption. *

* This can also close any resources and clean up temporary state if necessary. diff --git a/core/src/main/java/org/apache/spark/shuffle/api/ShufflePartitionWriter.java b/core/src/main/java/org/apache/spark/shuffle/api/ShufflePartitionWriter.java index 928875156a70..143cc6c871e5 100644 --- a/core/src/main/java/org/apache/spark/shuffle/api/ShufflePartitionWriter.java +++ b/core/src/main/java/org/apache/spark/shuffle/api/ShufflePartitionWriter.java @@ -49,7 +49,7 @@ public interface ShufflePartitionWriter { * by the parent {@link ShuffleMapOutputWriter}. If one does so, ensure that * {@link OutputStream#close()} does not close the resource, since it will be reused across * partition writes. The underlying resources should be cleaned up in - * {@link ShuffleMapOutputWriter#commitAllPartitions()} and + * {@link ShuffleMapOutputWriter#commitAllPartitions(long[])} and * {@link ShuffleMapOutputWriter#abort(Throwable)}. */ OutputStream openStream() throws IOException; @@ -68,7 +68,7 @@ public interface ShufflePartitionWriter { * by the parent {@link ShuffleMapOutputWriter}. If one does so, ensure that * {@link WritableByteChannelWrapper#close()} does not close the resource, since the channel * will be reused across partition writes. The underlying resources should be cleaned up in - * {@link ShuffleMapOutputWriter#commitAllPartitions()} and + * {@link ShuffleMapOutputWriter#commitAllPartitions(long[])} and * {@link ShuffleMapOutputWriter#abort(Throwable)}. *

* This method is primarily for advanced optimizations where bytes can be copied from the input @@ -79,7 +79,7 @@ public interface ShufflePartitionWriter { * Note that the returned {@link WritableByteChannelWrapper} itself is closed, but not the * underlying channel that is returned by {@link WritableByteChannelWrapper#channel()}. Ensure * that the underlying channel is cleaned up in {@link WritableByteChannelWrapper#close()}, - * {@link ShuffleMapOutputWriter#commitAllPartitions()}, or + * {@link ShuffleMapOutputWriter#commitAllPartitions(long[])}, or * {@link ShuffleMapOutputWriter#abort(Throwable)}. */ default Optional openChannelWrapper() throws IOException { From 8fc719384ac98b03d1a404c215824f140f17ca1a Mon Sep 17 00:00:00 2001 From: "yi.wu" Date: Thu, 15 Jul 2021 15:22:36 +0800 Subject: [PATCH 46/47] fix test --- .../shuffle/checksum/ShuffleChecksumHelper.java | 2 +- .../shuffle/sort/UnsafeShuffleWriterSuite.java | 14 ++++++++++---- .../spark/shuffle/ShuffleChecksumTestHelper.scala | 4 ++-- .../sort/BypassMergeSortShuffleWriterSuite.scala | 6 ++++-- .../io/LocalDiskShuffleMapOutputWriterSuite.scala | 2 +- 5 files changed, 18 insertions(+), 10 deletions(-) diff --git a/core/src/main/java/org/apache/spark/shuffle/checksum/ShuffleChecksumHelper.java b/core/src/main/java/org/apache/spark/shuffle/checksum/ShuffleChecksumHelper.java index c27586020ac7..a368836d2bb1 100644 --- a/core/src/main/java/org/apache/spark/shuffle/checksum/ShuffleChecksumHelper.java +++ b/core/src/main/java/org/apache/spark/shuffle/checksum/ShuffleChecksumHelper.java @@ -90,7 +90,7 @@ public static String shuffleChecksumAlgorithm(SparkConf conf) { public static Checksum getChecksumByFileExtension(String fileName) throws SparkException { int index = fileName.lastIndexOf("."); String algorithm = fileName.substring(index + 1); - return getChecksumByAlgorithm(1, algorithm)[1]; + return getChecksumByAlgorithm(1, algorithm)[0]; } public static String getChecksumFileName(ShuffleChecksumBlockId blockId, SparkConf conf) { diff --git a/core/src/test/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriterSuite.java b/core/src/test/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriterSuite.java index b2bd9a78834e..e5b71d9e7519 100644 --- a/core/src/test/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriterSuite.java +++ b/core/src/test/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriterSuite.java @@ -24,6 +24,7 @@ import org.apache.spark.*; import org.apache.spark.shuffle.ShuffleChecksumTestHelper; +import org.apache.spark.shuffle.checksum.ShuffleChecksumHelper; import org.mockito.stubbing.Answer; import scala.*; import scala.collection.Iterator; @@ -298,10 +299,13 @@ public void writeWithoutSpilling() throws Exception { @Test public void writeChecksumFileWithoutSpill() throws Exception { IndexShuffleBlockResolver blockResolver = new IndexShuffleBlockResolver(conf, blockManager); - File checksumFile = new File(tempDir, "checksum"); + ShuffleChecksumBlockId checksumBlockId = + new ShuffleChecksumBlockId(0, 0, IndexShuffleBlockResolver.NOOP_REDUCE_ID()); + File checksumFile = new File(tempDir, + ShuffleChecksumHelper.getChecksumFileName(checksumBlockId, conf)); File dataFile = new File(tempDir, "data"); File indexFile = new File(tempDir, "index"); - when(diskBlockManager.getFile(new ShuffleChecksumBlockId(shuffleDep.shuffleId(), 0, 0))) + when(diskBlockManager.getFile(checksumFile.getName())) .thenReturn(checksumFile); when(diskBlockManager.getFile(new ShuffleDataBlockId(shuffleDep.shuffleId(), 0, 0))) .thenReturn(dataFile); @@ -324,10 +328,12 @@ public void writeChecksumFileWithoutSpill() throws Exception { @Test public void writeChecksumFileWithSpill() throws Exception { IndexShuffleBlockResolver blockResolver = new IndexShuffleBlockResolver(conf, blockManager); - File checksumFile = new File(tempDir, "checksum"); + ShuffleChecksumBlockId checksumBlockId = + new ShuffleChecksumBlockId(0, 0, IndexShuffleBlockResolver.NOOP_REDUCE_ID()); + File checksumFile = new File(tempDir, ShuffleChecksumHelper.getChecksumFileName(checksumBlockId, conf)); File dataFile = new File(tempDir, "data"); File indexFile = new File(tempDir, "index"); - when(diskBlockManager.getFile((BlockId) any())).thenReturn(checksumFile); + when(diskBlockManager.getFile(eq(checksumFile.getName()))).thenReturn(checksumFile); when(diskBlockManager.getFile(new ShuffleDataBlockId(shuffleDep.shuffleId(), 0, 0))) .thenReturn(dataFile); when(diskBlockManager.getFile(new ShuffleIndexBlockId(shuffleDep.shuffleId(), 0, 0))) diff --git a/core/src/test/scala/org/apache/spark/shuffle/ShuffleChecksumTestHelper.scala b/core/src/test/scala/org/apache/spark/shuffle/ShuffleChecksumTestHelper.scala index 6637ccb8e6a2..a8f2c4088c42 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/ShuffleChecksumTestHelper.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/ShuffleChecksumTestHelper.scala @@ -55,9 +55,9 @@ trait ShuffleChecksumTestHelper { val curOffset = indexIn.readLong val limit = (curOffset - prevOffset).toInt val bytes = new Array[Byte](limit) - val checksum = ShuffleChecksumHelper.getChecksumByFileExtension(index.getName) + val checksumCal = ShuffleChecksumHelper.getChecksumByFileExtension(checksum.getName) checkedIn = new CheckedInputStream( - new LimitedInputStream(dataIn, curOffset - prevOffset), checksum) + new LimitedInputStream(dataIn, curOffset - prevOffset), checksumCal) checkedIn.read(bytes, 0, limit) prevOffset = curOffset // checksum must be consistent at both write and read sides diff --git a/core/src/test/scala/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriterSuite.scala b/core/src/test/scala/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriterSuite.scala index 87d0994dc1f5..39eef9749eac 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriterSuite.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriterSuite.scala @@ -35,6 +35,7 @@ import org.apache.spark.memory.{TaskMemoryManager, TestMemoryManager} import org.apache.spark.serializer.{JavaSerializer, SerializerInstance, SerializerManager} import org.apache.spark.shuffle.{IndexShuffleBlockResolver, ShuffleChecksumTestHelper} import org.apache.spark.shuffle.api.ShuffleExecutorComponents +import org.apache.spark.shuffle.checksum.ShuffleChecksumHelper import org.apache.spark.shuffle.sort.io.LocalDiskShuffleExecutorComponents import org.apache.spark.storage._ import org.apache.spark.util.Utils @@ -247,11 +248,12 @@ class BypassMergeSortShuffleWriterSuite val checksumBlockId = ShuffleChecksumBlockId(shuffleId, mapId, 0) val dataBlockId = ShuffleDataBlockId(shuffleId, mapId, 0) val indexBlockId = ShuffleIndexBlockId(shuffleId, mapId, 0) - val checksumFile = new File(tempDir, checksumBlockId.name) + val checksumFile = new File(tempDir, + ShuffleChecksumHelper.getChecksumFileName(checksumBlockId, conf)) val dataFile = new File(tempDir, dataBlockId.name) val indexFile = new File(tempDir, indexBlockId.name) reset(diskBlockManager) - when(diskBlockManager.getFile(checksumBlockId)).thenAnswer(_ => checksumFile) + when(diskBlockManager.getFile(checksumFile.getName)).thenAnswer(_ => checksumFile) when(diskBlockManager.getFile(dataBlockId)).thenAnswer(_ => dataFile) when(diskBlockManager.getFile(indexBlockId)).thenAnswer(_ => indexFile) when(diskBlockManager.createTempShuffleBlock()) diff --git a/core/src/test/scala/org/apache/spark/shuffle/sort/io/LocalDiskShuffleMapOutputWriterSuite.scala b/core/src/test/scala/org/apache/spark/shuffle/sort/io/LocalDiskShuffleMapOutputWriterSuite.scala index 23f011f9fb9a..35d9b4ab1f76 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/sort/io/LocalDiskShuffleMapOutputWriterSuite.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/sort/io/LocalDiskShuffleMapOutputWriterSuite.scala @@ -78,7 +78,7 @@ class LocalDiskShuffleMapOutputWriterSuite extends SparkFunSuite with BeforeAndA anyInt, anyLong, any(classOf[Array[Long]]), any(classOf[Array[Long]]), any(classOf[File]))) .thenAnswer { invocationOnMock => partitionSizesInMergedFile = invocationOnMock.getArguments()(2).asInstanceOf[Array[Long]] - val tmp: File = invocationOnMock.getArguments()(3).asInstanceOf[File] + val tmp: File = invocationOnMock.getArguments()(4).asInstanceOf[File] if (tmp != null) { mergedOutputFile.delete() tmp.renameTo(mergedOutputFile) From 4bdde58a3791691f275968b117eeff260ef3016f Mon Sep 17 00:00:00 2001 From: "yi.wu" Date: Fri, 16 Jul 2021 14:49:39 +0800 Subject: [PATCH 47/47] fix java lint --- .../apache/spark/shuffle/sort/UnsafeShuffleWriterSuite.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/core/src/test/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriterSuite.java b/core/src/test/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriterSuite.java index e5b71d9e7519..cca3eb5b2cbc 100644 --- a/core/src/test/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriterSuite.java +++ b/core/src/test/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriterSuite.java @@ -330,7 +330,8 @@ public void writeChecksumFileWithSpill() throws Exception { IndexShuffleBlockResolver blockResolver = new IndexShuffleBlockResolver(conf, blockManager); ShuffleChecksumBlockId checksumBlockId = new ShuffleChecksumBlockId(0, 0, IndexShuffleBlockResolver.NOOP_REDUCE_ID()); - File checksumFile = new File(tempDir, ShuffleChecksumHelper.getChecksumFileName(checksumBlockId, conf)); + File checksumFile = + new File(tempDir, ShuffleChecksumHelper.getChecksumFileName(checksumBlockId, conf)); File dataFile = new File(tempDir, "data"); File indexFile = new File(tempDir, "index"); when(diskBlockManager.getFile(eq(checksumFile.getName()))).thenReturn(checksumFile);