From 149ea3ed9fb7019d52d8fc01c5bb0e5e12b58a1d Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Wed, 21 Oct 2015 14:26:26 -0500 Subject: [PATCH 01/33] ShuffleWriters write to temp file, then go through ShuffleOutputCoordinator to atomically move w/ "first one wins" --- .../shuffle/sort/UnsafeShuffleWriter.java | 38 +++++-- .../spark/scheduler/ShuffleMapTask.scala | 10 +- .../spark/scheduler/TaskSetManager.scala | 10 +- .../shuffle/FileShuffleBlockResolver.scala | 12 +- .../shuffle/IndexShuffleBlockResolver.scala | 9 +- .../shuffle/ShuffleOutputCoordinator.scala | 44 ++++++++ .../apache/spark/shuffle/ShuffleWriter.scala | 4 +- .../shuffle/hash/HashShuffleWriter.scala | 11 +- .../shuffle/sort/SortShuffleWriter.scala | 22 +++- .../spark/storage/DiskBlockObjectWriter.scala | 2 +- .../scala/org/apache/spark/ShuffleSuite.scala | 106 +++++++++++++++++- 11 files changed, 232 insertions(+), 36 deletions(-) create mode 100644 core/src/main/scala/org/apache/spark/shuffle/ShuffleOutputCoordinator.scala 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 e8f050cb2dab1..beba0bbf6f085 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 @@ -20,11 +20,18 @@ import javax.annotation.Nullable; import java.io.*; import java.nio.channels.FileChannel; +import java.util.Arrays; import java.util.Iterator; +import java.util.List; +import org.apache.spark.shuffle.IndexShuffleBlockResolver$; +import org.apache.spark.storage.ShuffleIndexBlockId; import scala.Option; import scala.Product2; +import scala.Tuple2; import scala.collection.JavaConverters; +import scala.collection.Seq; +import scala.collection.Seq$; import scala.collection.immutable.Map; import scala.reflect.ClassTag; import scala.reflect.ClassTag$; @@ -164,7 +171,7 @@ public void write(Iterator> records) throws IOException { } @Override - public void write(scala.collection.Iterator> records) throws IOException { + public Seq> write(scala.collection.Iterator> records) throws IOException { // Keep track of success so we know if we encountered an exception // We do this rather than a standard try/catch/re-throw to handle // generic throwables. @@ -173,8 +180,9 @@ public void write(scala.collection.Iterator> records) throws IOEx while (records.hasNext()) { insertRecordIntoSorter(records.next()); } - closeAndWriteOutput(); + Seq> result = closeAndWriteOutput(); success = true; + return result; } finally { if (sorter != null) { try { @@ -209,7 +217,7 @@ private void open() throws IOException { } @VisibleForTesting - void closeAndWriteOutput() throws IOException { + Seq> closeAndWriteOutput() throws IOException { assert(sorter != null); updatePeakMemoryUsed(); serBuffer = null; @@ -217,8 +225,11 @@ void closeAndWriteOutput() throws IOException { final SpillInfo[] spills = sorter.closeAndGetSpills(); sorter = null; final long[] partitionLengths; + final File tmpDataFile; try { - partitionLengths = mergeSpills(spills); + Tuple2 t = mergeSpills(spills); + partitionLengths = t._2(); + tmpDataFile = t._1(); } finally { for (SpillInfo spill : spills) { if (spill.file.exists() && ! spill.file.delete()) { @@ -226,8 +237,17 @@ void closeAndWriteOutput() throws IOException { } } } - shuffleBlockResolver.writeIndexFile(shuffleId, mapId, partitionLengths); + File tmpIndexFile = shuffleBlockResolver.writeIndexFile(shuffleId, mapId, partitionLengths); mapStatus = MapStatus$.MODULE$.apply(blockManager.shuffleServerId(), partitionLengths); + File dataFile = shuffleBlockResolver.getDataFile(shuffleId, mapId); + File indexFile = blockManager.diskBlockManager().getFile( + new ShuffleIndexBlockId(shuffleId, mapId, IndexShuffleBlockResolver$.MODULE$.NOOP_REDUCE_ID()) + ); + + return JavaConverters.asScalaBufferConverter(Arrays.asList( + new Tuple2<>(tmpIndexFile, indexFile), + new Tuple2<>(tmpDataFile, dataFile) + )).asScala(); } @VisibleForTesting @@ -259,8 +279,12 @@ void forceSorterToSpill() throws IOException { * * @return the partition lengths in the merged file. */ - private long[] mergeSpills(SpillInfo[] spills) throws IOException { - final File outputFile = shuffleBlockResolver.getDataFile(shuffleId, mapId); + private Tuple2 mergeSpills(SpillInfo[] spills) throws IOException { + final File outputFile = blockManager.diskBlockManager().createTempShuffleBlock()._2(); + return new Tuple2<>(outputFile, mergeSpills(spills, outputFile)); + } + + private long[] mergeSpills(SpillInfo[] spills, File outputFile) throws IOException { final boolean compressionEnabled = sparkConf.getBoolean("spark.shuffle.compress", true); final CompressionCodec compressionCodec = CompressionCodec$.MODULE$.createCodec(sparkConf); final boolean fastMergeEnabled = diff --git a/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala b/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala index f478f9982afef..2611210d4cbd9 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala @@ -24,7 +24,7 @@ import scala.language.existentials import org.apache.spark._ import org.apache.spark.broadcast.Broadcast import org.apache.spark.rdd.RDD -import org.apache.spark.shuffle.ShuffleWriter +import org.apache.spark.shuffle.{ShuffleOutputCoordinator, ShuffleWriter} /** * A ShuffleMapTask divides the elements of an RDD into multiple buckets (based on a partitioner @@ -70,8 +70,12 @@ private[spark] class ShuffleMapTask( try { val manager = SparkEnv.get.shuffleManager writer = manager.getWriter[Any, Any](dep.shuffleHandle, partitionId, context) - writer.write(rdd.iterator(partition, context).asInstanceOf[Iterator[_ <: Product2[Any, Any]]]) - writer.stop(success = true).get + val tmpToDestFiles = writer.write( + rdd.iterator(partition, context).asInstanceOf[Iterator[_ <: Product2[Any, Any]]]) + val mapStatus = writer.stop(success = true).get + // SPARK-8029 make sure only one task on this executor writes the final shuffle files + ShuffleOutputCoordinator.moveIfDestMissing(tmpToDestFiles) + mapStatus } catch { case e: Exception => try { diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala index 987800d3d1f1e..8026cbf1ae967 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala @@ -615,6 +615,7 @@ private[spark] class TaskSetManager( val index = info.index info.markSuccessful() removeRunningTask(tid) + val task = tasks(index) // This method is called by "TaskSchedulerImpl.handleSuccessfulTask" which holds the // "TaskSchedulerImpl" lock until exiting. To avoid the SPARK-7655 issue, we should not // "deserialize" the value when holding a lock to avoid blocking other threads. So we call @@ -622,11 +623,14 @@ private[spark] class TaskSetManager( // Note: "result.value()" only deserializes the value when it's called at the first time, so // here "result.value()" just returns the value and won't block other threads. sched.dagScheduler.taskEnded( - tasks(index), Success, result.value(), result.accumUpdates, info, result.metrics) + task, Success, result.value(), result.accumUpdates, info, result.metrics) if (!successful(index)) { tasksSuccessful += 1 - logInfo("Finished task %s in stage %s (TID %d) in %d ms on %s (%d/%d)".format( - info.id, taskSet.id, info.taskId, info.duration, info.host, tasksSuccessful, numTasks)) + // include the partition here b/c on a stage retry, the partition is *not* necessarily + // the same as info.id + logInfo(s"Finished task ${info.id} in stage ${taskSet.id} (TID ${info.taskId}}, " + + s"partition ${task.partitionId}) in ${info.duration} ms on executor ${info.executorId} " + + s"(${info.host}) ($tasksSuccessful/$numTasks)") // Mark successful and stop if all the tasks have succeeded. successful(index) = true if (tasksSuccessful == numTasks) { diff --git a/core/src/main/scala/org/apache/spark/shuffle/FileShuffleBlockResolver.scala b/core/src/main/scala/org/apache/spark/shuffle/FileShuffleBlockResolver.scala index cd253a78c2b19..90dc07a84e59e 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/FileShuffleBlockResolver.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/FileShuffleBlockResolver.scala @@ -17,6 +17,7 @@ package org.apache.spark.shuffle +import java.io.File import java.util.concurrent.ConcurrentLinkedQueue import scala.collection.JavaConverters._ @@ -31,7 +32,7 @@ import org.apache.spark.util.{MetadataCleaner, MetadataCleanerType, TimeStampedH /** A group of writers for a ShuffleMapTask, one writer per reducer. */ private[spark] trait ShuffleWriterGroup { - val writers: Array[DiskBlockObjectWriter] + val writers: Array[(DiskBlockObjectWriter, File)] /** @param success Indicates all writes were successful. If false, no blocks will be recorded. */ def releaseWriters(success: Boolean) @@ -80,10 +81,11 @@ private[spark] class FileShuffleBlockResolver(conf: SparkConf) val openStartTime = System.nanoTime val serializerInstance = serializer.newInstance() - val writers: Array[DiskBlockObjectWriter] = { - Array.tabulate[DiskBlockObjectWriter](numReducers) { bucketId => + val writers: Array[(DiskBlockObjectWriter, File)] = { + Array.tabulate[(DiskBlockObjectWriter, File)](numReducers) { bucketId => val blockId = ShuffleBlockId(shuffleId, mapId, bucketId) val blockFile = blockManager.diskBlockManager.getFile(blockId) + val (_, tmpBlockFile) = blockManager.diskBlockManager.createTempLocalBlock() // Because of previous failures, the shuffle file may already exist on this machine. // If so, remove it. if (blockFile.exists) { @@ -93,8 +95,8 @@ private[spark] class FileShuffleBlockResolver(conf: SparkConf) logWarning(s"Failed to remove existing shuffle file $blockFile") } } - blockManager.getDiskWriter(blockId, blockFile, serializerInstance, bufferSize, - writeMetrics) + blockManager.getDiskWriter(blockId, tmpBlockFile, serializerInstance, bufferSize, + writeMetrics) -> blockFile } } // Creating the file to write to and creating a disk writer both involve interacting with 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 5e4c2b5d0a5c4..5910d2057542c 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala @@ -77,11 +77,11 @@ private[spark] class IndexShuffleBlockResolver(conf: SparkConf) extends ShuffleB /** * 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. + * begins and ends. Writes to a temp file, and returns that file. * */ - def writeIndexFile(shuffleId: Int, mapId: Int, lengths: Array[Long]): Unit = { - val indexFile = getIndexFile(shuffleId, mapId) - val out = new DataOutputStream(new BufferedOutputStream(new FileOutputStream(indexFile))) + def writeIndexFile(shuffleId: Int, mapId: Int, lengths: Array[Long]): File = { + val (_, tmpIndexFile) = blockManager.diskBlockManager.createTempShuffleBlock() + val out = new DataOutputStream(new BufferedOutputStream(new FileOutputStream(tmpIndexFile))) Utils.tryWithSafeFinally { // We take in lengths of each block, need to convert it to offsets. var offset = 0L @@ -90,6 +90,7 @@ private[spark] class IndexShuffleBlockResolver(conf: SparkConf) extends ShuffleB offset += length out.writeLong(offset) } + tmpIndexFile } { out.close() } diff --git a/core/src/main/scala/org/apache/spark/shuffle/ShuffleOutputCoordinator.scala b/core/src/main/scala/org/apache/spark/shuffle/ShuffleOutputCoordinator.scala new file mode 100644 index 0000000000000..a6afc520cc14d --- /dev/null +++ b/core/src/main/scala/org/apache/spark/shuffle/ShuffleOutputCoordinator.scala @@ -0,0 +1,44 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.shuffle + +import java.io.File + +import org.apache.spark.Logging + +/** + * Ensures that on each executor, there are no conflicting writes to the same shuffle files. It + * implements "first write wins", by atomically moving all shuffle files into their final location, + * only if the files did not already exist. + */ +object ShuffleOutputCoordinator extends Logging { + + def moveIfDestMissing(tmpToDest: Seq[(File, File)]): Boolean = synchronized { + val prevExists = tmpToDest.forall(_._2.exists) + if (!prevExists) { + tmpToDest.foreach { case (tmp, dest) => + !tmp.renameTo(dest) + } + true + } else { + logInfo(s"shuffle output already exists, not overwriting. Another task must have created" + + s" this shuffle outputs") + tmpToDest.foreach{ case (tmp, _) => tmp.delete()} + false + } + } +} diff --git a/core/src/main/scala/org/apache/spark/shuffle/ShuffleWriter.scala b/core/src/main/scala/org/apache/spark/shuffle/ShuffleWriter.scala index 4cc4ef5f1886e..7d3cb1a0079a9 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/ShuffleWriter.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/ShuffleWriter.scala @@ -17,7 +17,7 @@ package org.apache.spark.shuffle -import java.io.IOException +import java.io.{File, IOException} import org.apache.spark.scheduler.MapStatus @@ -27,7 +27,7 @@ import org.apache.spark.scheduler.MapStatus private[spark] abstract class ShuffleWriter[K, V] { /** Write a sequence of records to this task's output */ @throws[IOException] - def write(records: Iterator[Product2[K, V]]): Unit + def write(records: Iterator[Product2[K, V]]): Seq[(File, File)] /** Close this writer, passing along whether the map completed */ def stop(success: Boolean): Option[MapStatus] diff --git a/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleWriter.scala b/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleWriter.scala index 41df70c602c30..76d61a04ca310 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleWriter.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleWriter.scala @@ -17,6 +17,8 @@ package org.apache.spark.shuffle.hash +import java.io.File + import org.apache.spark._ import org.apache.spark.executor.ShuffleWriteMetrics import org.apache.spark.scheduler.MapStatus @@ -49,7 +51,7 @@ private[spark] class HashShuffleWriter[K, V]( writeMetrics) /** Write a bunch of records to this task's output */ - override def write(records: Iterator[Product2[K, V]]): Unit = { + override def write(records: Iterator[Product2[K, V]]): Seq[(File, File)] = { val iter = if (dep.aggregator.isDefined) { if (dep.mapSideCombine) { dep.aggregator.get.combineValuesByKey(records, context) @@ -63,8 +65,9 @@ private[spark] class HashShuffleWriter[K, V]( for (elem <- iter) { val bucketId = dep.partitioner.getPartition(elem._1) - shuffle.writers(bucketId).write(elem._1, elem._2) + shuffle.writers(bucketId)._1.write(elem._1, elem._2) } + shuffle.writers.map { case (writer, destFile) => writer.file -> destFile} } /** Close this writer, passing along whether the map completed */ @@ -102,7 +105,7 @@ private[spark] class HashShuffleWriter[K, V]( private def commitWritesAndBuildStatus(): MapStatus = { // Commit the writes. Get the size of each bucket block (total block size). - val sizes: Array[Long] = shuffle.writers.map { writer: DiskBlockObjectWriter => + val sizes: Array[Long] = shuffle.writers.map { case (writer: DiskBlockObjectWriter, _) => writer.commitAndClose() writer.fileSegment().length } @@ -112,7 +115,7 @@ private[spark] class HashShuffleWriter[K, V]( private def revertWrites(): Unit = { if (shuffle != null && shuffle.writers != null) { for (writer <- shuffle.writers) { - writer.revertPartialWritesAndClose() + writer._1.revertPartialWritesAndClose() } } } 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 bbd9c1ab53cd8..9746f20a124eb 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 @@ -17,11 +17,14 @@ package org.apache.spark.shuffle.sort +import java.io.File + import org.apache.spark._ import org.apache.spark.executor.ShuffleWriteMetrics import org.apache.spark.scheduler.MapStatus -import org.apache.spark.shuffle.{IndexShuffleBlockResolver, ShuffleWriter, BaseShuffleHandle} -import org.apache.spark.storage.ShuffleBlockId +import org.apache.spark.serializer.Serializer +import org.apache.spark.shuffle.{ShuffleOutputCoordinator, IndexShuffleBlockResolver, ShuffleWriter, BaseShuffleHandle} +import org.apache.spark.storage.{ShuffleIndexBlockId, ShuffleBlockId} import org.apache.spark.util.collection.ExternalSorter private[spark] class SortShuffleWriter[K, V, C]( @@ -48,7 +51,7 @@ private[spark] class SortShuffleWriter[K, V, C]( context.taskMetrics.shuffleWriteMetrics = Some(writeMetrics) /** Write a bunch of records to this task's output */ - override def write(records: Iterator[Product2[K, V]]): Unit = { + override def write(records: Iterator[Product2[K, V]]): Seq[(File, File)] = { sorter = if (dep.mapSideCombine) { require(dep.aggregator.isDefined, "Map-side combine without Aggregator specified!") new ExternalSorter[K, V, C]( @@ -65,12 +68,19 @@ private[spark] class SortShuffleWriter[K, V, C]( // Don't bother including the time to open the merged output file in the shuffle write time, // because it just opens a single file, so is typically too fast to measure accurately // (see SPARK-3570). - val outputFile = shuffleBlockResolver.getDataFile(dep.shuffleId, mapId) + val (_, tmpDataFile) = blockManager.diskBlockManager.createTempShuffleBlock() val blockId = ShuffleBlockId(dep.shuffleId, mapId, IndexShuffleBlockResolver.NOOP_REDUCE_ID) - val partitionLengths = sorter.writePartitionedFile(blockId, context, outputFile) - shuffleBlockResolver.writeIndexFile(dep.shuffleId, mapId, partitionLengths) + val partitionLengths = sorter.writePartitionedFile(blockId, context, tmpDataFile) + val tmpIndexFile = shuffleBlockResolver.writeIndexFile(dep.shuffleId, mapId, partitionLengths) + val dataFile = shuffleBlockResolver.getDataFile(dep.shuffleId, mapId) + val indexFile = blockManager.diskBlockManager.getFile( + ShuffleIndexBlockId(handle.shuffleId, mapId, IndexShuffleBlockResolver.NOOP_REDUCE_ID)) mapStatus = MapStatus(blockManager.shuffleServerId, partitionLengths) + Seq( + tmpDataFile -> dataFile, + tmpIndexFile -> indexFile + ) } /** Close this writer, passing along whether the map completed */ 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 80d426fadc65e..4469bb22f3e10 100644 --- a/core/src/main/scala/org/apache/spark/storage/DiskBlockObjectWriter.scala +++ b/core/src/main/scala/org/apache/spark/storage/DiskBlockObjectWriter.scala @@ -34,7 +34,7 @@ import org.apache.spark.util.Utils * reopened again. */ private[spark] class DiskBlockObjectWriter( - file: File, + val file: File, serializerInstance: SerializerInstance, bufferSize: Int, compressStream: OutputStream => OutputStream, diff --git a/core/src/test/scala/org/apache/spark/ShuffleSuite.scala b/core/src/test/scala/org/apache/spark/ShuffleSuite.scala index 4a0877d86f2c6..4ba88035fe433 100644 --- a/core/src/test/scala/org/apache/spark/ShuffleSuite.scala +++ b/core/src/test/scala/org/apache/spark/ShuffleSuite.scala @@ -17,13 +17,17 @@ package org.apache.spark +import java.util.concurrent.{Executors, ExecutorService, Callable, CyclicBarrier} + import org.scalatest.Matchers import org.apache.spark.ShuffleSuite.NonJavaSerializableClass import org.apache.spark.rdd.{CoGroupedRDD, OrderedRDDFunctions, RDD, ShuffledRDD, SubtractedRDD} -import org.apache.spark.scheduler.{SparkListener, SparkListenerTaskEnd} +import org.apache.spark.scheduler.{MapStatus, MyRDD, SparkListener, SparkListenerTaskEnd} import org.apache.spark.serializer.KryoSerializer +import org.apache.spark.shuffle.{ShuffleWriter, ShuffleOutputCoordinator} import org.apache.spark.storage.{ShuffleDataBlockId, ShuffleBlockId} +import org.apache.spark.unsafe.memory.TaskMemoryManager import org.apache.spark.util.MutablePair abstract class ShuffleSuite extends SparkFunSuite with Matchers with LocalSparkContext { @@ -317,6 +321,106 @@ abstract class ShuffleSuite extends SparkFunSuite with Matchers with LocalSparkC assert(metrics.bytesWritten === metrics.byresRead) assert(metrics.bytesWritten > 0) } + + test("multiple attempts for one task") { + sc = new SparkContext("local", "test", conf) + val mapTrackerMaster = sc.env.mapOutputTracker.asInstanceOf[MapOutputTrackerMaster] + val manager = sc.env.shuffleManager + val taskMemoryManager = new TaskMemoryManager(sc.env.executorMemoryManager) + val metricsSystem = sc.env.metricsSystem + val shuffleMapRdd = new MyRDD(sc, 1, Nil) + val shuffleDep = new ShuffleDependency(shuffleMapRdd, new HashPartitioner(1)) + val shuffleHandle = manager.registerShuffle(0, 1, shuffleDep) + + // first attempt -- its successful + val writer1 = manager.getWriter[Int, Int](shuffleHandle, 0, + new TaskContextImpl(0, 0, 0L, 0, taskMemoryManager, metricsSystem, + InternalAccumulator.create(sc))) + val data1 = (1 to 10).map { x => x -> x} + + // second attempt -- also successful. We'll write out different data, + // just to simulate the fact that the records may get written differently + // depending on what gets spilled, what gets combined, etc. + val writer2 = manager.getWriter[Int, Int](shuffleHandle, 0, + new TaskContextImpl(0, 0, 1L, 0, taskMemoryManager, metricsSystem, + InternalAccumulator.create(sc))) + val data2 = (11 to 20).map { x => x -> x} + + // interleave writes of both attempts -- we want to test that both attempts can occur + // simultaneously, and everything is still OK + + def writeAndClose( + writer: ShuffleWriter[Int, Int])( + iter: Iterator[(Int, Int)]): Option[MapStatus] = { + val files = writer.write(iter) + val output = writer.stop(true) + ShuffleOutputCoordinator.moveIfDestMissing(files) + output + } + val interleaver = new InterleaveIterators( + data1, writeAndClose(writer1), data2, writeAndClose(writer2)) + + val (mapOutput1, mapOutput2) = interleaver.run() + + + // check that we can read the map output and it has the right data (can be either from + // either task, but must be consistent) + assert(mapOutput1.isDefined) + assert(mapOutput2.isDefined) + assert(mapOutput1.get.location === mapOutput2.get.location) + // register one of the map outputs -- doesn't matter which one + mapOutput1.foreach { mapStatus => mapTrackerMaster.registerMapOutputs(0, Array(mapStatus))} + + val reader = manager.getReader[Int, Int](shuffleHandle, 0, 1, + new TaskContextImpl(1, 0, 2L, 0, taskMemoryManager, metricsSystem, + InternalAccumulator.create(sc))) + val readData = reader.read().toIndexedSeq + assert(readData === data1.toIndexedSeq || readData === data2.toIndexedSeq) + } + +} + +/** + * Utility to help tests make sure that we can process two different iterators simultaneously + * in different threads. This makes sure that in your test, you don't completely process data1 with + * f1 before processing data2 with f2 (or vice versa). It adds a barrier so that the functions only + * process one element, before pausing to wait for the other function to "catch up". + */ +class InterleaveIterators[T, R]( + data1: Seq[T], + f1: Iterator[T] => R, + data2: Seq[T], + f2: Iterator[T] => R) { + + require(data1.size == data2.size) + + val barrier = new CyclicBarrier(2) + class BarrierIterator[E](id: Int, sub: Iterator[E]) extends Iterator[E] { + def hasNext: Boolean = sub.hasNext + + def next: E = { + barrier.await() + sub.next() + } + } + + val c1 = new Callable[R] { + override def call(): R = f1(new BarrierIterator(1, data1.iterator)) + } + val c2 = new Callable[R] { + override def call(): R = f2(new BarrierIterator(2, data2.iterator)) + } + + val e: ExecutorService = Executors.newFixedThreadPool(2) + + def run(): (R, R) = { + val future1 = e.submit(c1) + val future2 = e.submit(c2) + val r1 = future1.get() + val r2 = future2.get() + e.shutdown() + (r1, r2) + } } object ShuffleSuite { From cf8118eb33e04928ee27cb084aac6d166e480aaf Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Wed, 21 Oct 2015 22:17:17 -0500 Subject: [PATCH 02/33] assorted cleanup --- .../spark/scheduler/ShuffleMapTask.scala | 2 +- .../shuffle/ShuffleOutputCoordinator.scala | 19 ++++++++++++++----- .../apache/spark/shuffle/ShuffleWriter.scala | 7 ++++++- .../scala/org/apache/spark/ShuffleSuite.scala | 4 ++-- 4 files changed, 23 insertions(+), 9 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala b/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala index 2611210d4cbd9..4c1c8f2608eee 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala @@ -74,7 +74,7 @@ private[spark] class ShuffleMapTask( rdd.iterator(partition, context).asInstanceOf[Iterator[_ <: Product2[Any, Any]]]) val mapStatus = writer.stop(success = true).get // SPARK-8029 make sure only one task on this executor writes the final shuffle files - ShuffleOutputCoordinator.moveIfDestMissing(tmpToDestFiles) + ShuffleOutputCoordinator.commitOutputs(dep.shuffleId, partitionId, tmpToDestFiles) mapStatus } catch { case e: Exception => diff --git a/core/src/main/scala/org/apache/spark/shuffle/ShuffleOutputCoordinator.scala b/core/src/main/scala/org/apache/spark/shuffle/ShuffleOutputCoordinator.scala index a6afc520cc14d..a0a39c46c05f3 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/ShuffleOutputCoordinator.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/ShuffleOutputCoordinator.scala @@ -23,20 +23,29 @@ import org.apache.spark.Logging /** * Ensures that on each executor, there are no conflicting writes to the same shuffle files. It * implements "first write wins", by atomically moving all shuffle files into their final location, - * only if the files did not already exist. + * only if the files did not already exist. See SPARK-8029 */ object ShuffleOutputCoordinator extends Logging { - def moveIfDestMissing(tmpToDest: Seq[(File, File)]): Boolean = synchronized { + /** + * if any of the destination files do not exist, then move all of the temporary files to their + * destinations. If all destination files exist, then simply delete all temporary files + * + * @param tmpToDest pairs of (temporary, destination) file pairs + * @return + */ + def commitOutputs( + shuffleId: Int, + partitionId: Int, tmpToDest: Seq[(File, File)]): Boolean = synchronized { val prevExists = tmpToDest.forall(_._2.exists) if (!prevExists) { tmpToDest.foreach { case (tmp, dest) => - !tmp.renameTo(dest) + tmp.renameTo(dest) } true } else { - logInfo(s"shuffle output already exists, not overwriting. Another task must have created" + - s" this shuffle outputs") + logInfo(s"shuffle output for shuffle $shuffleId, partition $partitionId already exists, not " + + s"overwriting. Another task must have created this shuffle output.") tmpToDest.foreach{ case (tmp, _) => tmp.delete()} false } diff --git a/core/src/main/scala/org/apache/spark/shuffle/ShuffleWriter.scala b/core/src/main/scala/org/apache/spark/shuffle/ShuffleWriter.scala index 7d3cb1a0079a9..56af1590b088c 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/ShuffleWriter.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/ShuffleWriter.scala @@ -25,7 +25,12 @@ import org.apache.spark.scheduler.MapStatus * Obtained inside a map task to write out records to the shuffle system. */ private[spark] abstract class ShuffleWriter[K, V] { - /** Write a sequence of records to this task's output */ + /** + * Write a sequence of records to this task's output. This should write all data + * to temporary files, but return (temporaryFile, destinationFile) pairs for each + * file written. The temporary files will get moved to their destination or deleted + * by the [[ShuffleOutputCoordinator]] + */ @throws[IOException] def write(records: Iterator[Product2[K, V]]): Seq[(File, File)] diff --git a/core/src/test/scala/org/apache/spark/ShuffleSuite.scala b/core/src/test/scala/org/apache/spark/ShuffleSuite.scala index 4ba88035fe433..832936fa76197 100644 --- a/core/src/test/scala/org/apache/spark/ShuffleSuite.scala +++ b/core/src/test/scala/org/apache/spark/ShuffleSuite.scala @@ -322,7 +322,7 @@ abstract class ShuffleSuite extends SparkFunSuite with Matchers with LocalSparkC assert(metrics.bytesWritten > 0) } - test("multiple attempts for one task") { + test("multiple simultaneous attempts for one task (SPARK-8029)") { sc = new SparkContext("local", "test", conf) val mapTrackerMaster = sc.env.mapOutputTracker.asInstanceOf[MapOutputTrackerMaster] val manager = sc.env.shuffleManager @@ -354,7 +354,7 @@ abstract class ShuffleSuite extends SparkFunSuite with Matchers with LocalSparkC iter: Iterator[(Int, Int)]): Option[MapStatus] = { val files = writer.write(iter) val output = writer.stop(true) - ShuffleOutputCoordinator.moveIfDestMissing(files) + ShuffleOutputCoordinator.commitOutputs(0, 0, files) output } val interleaver = new InterleaveIterators( From ea1ae0711ff1692bf97d7f69134a3dec572b6a1f Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Wed, 21 Oct 2015 22:19:07 -0500 Subject: [PATCH 03/33] style --- .../org/apache/spark/shuffle/ShuffleOutputCoordinator.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/shuffle/ShuffleOutputCoordinator.scala b/core/src/main/scala/org/apache/spark/shuffle/ShuffleOutputCoordinator.scala index a0a39c46c05f3..c54ff41d39041 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/ShuffleOutputCoordinator.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/ShuffleOutputCoordinator.scala @@ -44,8 +44,8 @@ object ShuffleOutputCoordinator extends Logging { } true } else { - logInfo(s"shuffle output for shuffle $shuffleId, partition $partitionId already exists, not " + - s"overwriting. Another task must have created this shuffle output.") + logInfo(s"shuffle output for shuffle $shuffleId, partition $partitionId already exists, " + + s"not overwriting. Another task must have created this shuffle output.") tmpToDest.foreach{ case (tmp, _) => tmp.delete()} false } From 9356c6734f33dbf0a92befa031665b0b923a46da Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Thu, 22 Oct 2015 09:06:45 -0500 Subject: [PATCH 04/33] fix compilation in StoragePerfTester --- .../main/scala/org/apache/spark/tools/StoragePerfTester.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tools/src/main/scala/org/apache/spark/tools/StoragePerfTester.scala b/tools/src/main/scala/org/apache/spark/tools/StoragePerfTester.scala index 0dc2861253f17..50a68d4116f83 100644 --- a/tools/src/main/scala/org/apache/spark/tools/StoragePerfTester.scala +++ b/tools/src/main/scala/org/apache/spark/tools/StoragePerfTester.scala @@ -64,9 +64,9 @@ object StoragePerfTester { new KryoSerializer(sc.conf), new ShuffleWriteMetrics()) val writers = shuffle.writers for (i <- 1 to recordsPerMap) { - writers(i % numOutputSplits).write(writeKey, writeValue) + writers(i % numOutputSplits)._1.write(writeKey, writeValue) } - writers.map { w => + writers.map { case (w, _) => w.commitAndClose() total.addAndGet(w.fileSegment().length) } From 2b42eb5685667240555d33bcbcf8bc708091a4b0 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Thu, 22 Oct 2015 10:42:31 -0500 Subject: [PATCH 05/33] mima --- project/MimaExcludes.scala | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index b5e661d3ecfa8..2253f2936c1f9 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -108,6 +108,10 @@ object MimaExcludes { ) ++ Seq( ProblemFilters.exclude[MissingMethodProblem]( "org.apache.spark.SparkContext.preferredNodeLocationData_=") + ) ++ Seq( + // SPARK-8029 -- this is a private[spark] method but that has to be public in java + ProblemFilters.exclude[IncompatibleResultTypeProblem]( + "org.apache.spark.shuffle.unsafe.UnsafeShuffleWriter.write") ) case v if v.startsWith("1.5") => Seq( From 32d4b3b69651263922b317db54bef992d67a3a3c Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Thu, 22 Oct 2015 14:43:18 -0500 Subject: [PATCH 06/33] update UnsafeShuffleWriterSuite --- .../shuffle/sort/UnsafeShuffleWriter.java | 4 +- .../sort/UnsafeShuffleWriterSuite.java | 60 +++++++++++++------ 2 files changed, 45 insertions(+), 19 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 beba0bbf6f085..0235a43336a25 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 @@ -166,8 +166,8 @@ public long getPeakMemoryUsedBytes() { * This convenience method should only be called in test code. */ @VisibleForTesting - public void write(Iterator> records) throws IOException { - write(JavaConverters.asScalaIteratorConverter(records).asScala()); + public Seq> write(Iterator> records) throws IOException { + return write(JavaConverters.asScalaIteratorConverter(records).asScala()); } @Override 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 29d9823b1f71b..8ba9cd6dd5b7d 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,6 +23,7 @@ import scala.*; import scala.collection.Iterator; +import scala.collection.Seq; import scala.runtime.AbstractFunction1; import com.google.common.collect.Iterators; @@ -48,6 +49,7 @@ import org.apache.spark.io.LZ4CompressionCodec; import org.apache.spark.io.LZFCompressionCodec; import org.apache.spark.io.SnappyCompressionCodec; +import org.apache.spark.shuffle.ShuffleOutputCoordinator; import org.apache.spark.executor.ShuffleWriteMetrics; import org.apache.spark.executor.TaskMetrics; import org.apache.spark.network.util.LimitedInputStream; @@ -71,7 +73,7 @@ public class UnsafeShuffleWriterSuite { File mergedOutputFile; File tempDir; long[] partitionSizesInMergedFile; - final LinkedList spillFilesCreated = new LinkedList(); + final LinkedList tmpShuffleFilesCreated = new LinkedList(); SparkConf conf; final Serializer serializer = new KryoSerializer(new SparkConf()); TaskMetrics taskMetrics; @@ -109,8 +111,10 @@ public void setUp() throws IOException { MockitoAnnotations.initMocks(this); tempDir = Utils.createTempDir("test", "test"); mergedOutputFile = File.createTempFile("mergedoutput", "", tempDir); + // the ShuffleOutputCoordinator requires that this file does not exist + mergedOutputFile.delete(); partitionSizesInMergedFile = null; - spillFilesCreated.clear(); + tmpShuffleFilesCreated.clear(); conf = new SparkConf().set("spark.buffer.pageSize", "128m"); taskMetrics = new TaskMetrics(); @@ -169,11 +173,11 @@ public OutputStream answer(InvocationOnMock invocation) throws Throwable { ); when(shuffleBlockResolver.getDataFile(anyInt(), anyInt())).thenReturn(mergedOutputFile); - doAnswer(new Answer() { + doAnswer(new Answer() { @Override - public Void answer(InvocationOnMock invocationOnMock) throws Throwable { + public File answer(InvocationOnMock invocationOnMock) throws Throwable { partitionSizesInMergedFile = (long[]) invocationOnMock.getArguments()[2]; - return null; + return diskBlockManager.createTempShuffleBlock()._2(); } }).when(shuffleBlockResolver).writeIndexFile(anyInt(), anyInt(), any(long[].class)); @@ -184,11 +188,23 @@ public Tuple2 answer( InvocationOnMock invocationOnMock) throws Throwable { TempShuffleBlockId blockId = new TempShuffleBlockId(UUID.randomUUID()); File file = File.createTempFile("spillFile", ".spill", tempDir); - spillFilesCreated.add(file); + tmpShuffleFilesCreated.add(file); return Tuple2$.MODULE$.apply(blockId, file); } }); + when(diskBlockManager.getFile(any(BlockId.class))).thenAnswer( + new Answer() { + @Override + public File answer(InvocationOnMock invocationOnMock) throws Throwable { + File f = File.createTempFile("shuffleFile",".index", tempDir); + // the ShuffleOutputCoordinator requires that this file does not exist + f.delete(); + return f; + } + } + ); + when(taskContext.taskMetrics()).thenReturn(taskMetrics); when(taskContext.internalMetricsToAccumulators()).thenReturn(null); @@ -212,7 +228,7 @@ private UnsafeShuffleWriter createWriter( } private void assertSpillFilesWereCleanedUp() { - for (File spillFile : spillFilesCreated) { + for (File spillFile : tmpShuffleFilesCreated) { assertFalse("Spill file " + spillFile.getPath() + " was not cleaned up", spillFile.exists()); } @@ -274,8 +290,9 @@ class BadRecords extends scala.collection.AbstractIterator writer = createWriter(true); - writer.write(Iterators.>emptyIterator()); + Seq> files = writer.write(Iterators.>emptyIterator()); final Option mapStatus = writer.stop(true); + ShuffleOutputCoordinator.commitOutputs(0, 0, files); assertTrue(mapStatus.isDefined()); assertTrue(mergedOutputFile.exists()); assertArrayEquals(new long[NUM_PARTITITONS], partitionSizesInMergedFile); @@ -294,8 +311,9 @@ public void writeWithoutSpilling() throws Exception { dataToWrite.add(new Tuple2(i, i)); } final UnsafeShuffleWriter writer = createWriter(true); - writer.write(dataToWrite.iterator()); + Seq> files = writer.write(dataToWrite.iterator()); final Option mapStatus = writer.stop(true); + ShuffleOutputCoordinator.commitOutputs(0, 0, files); assertTrue(mapStatus.isDefined()); assertTrue(mergedOutputFile.exists()); @@ -339,11 +357,13 @@ private void testMergingSpills( writer.forceSorterToSpill(); writer.insertRecordIntoSorter(dataToWrite.get(4)); writer.insertRecordIntoSorter(dataToWrite.get(5)); - writer.closeAndWriteOutput(); + Seq> files = writer.closeAndWriteOutput(); final Option mapStatus = writer.stop(true); + ShuffleOutputCoordinator.commitOutputs(0, 0, files); assertTrue(mapStatus.isDefined()); assertTrue(mergedOutputFile.exists()); - assertEquals(2, spillFilesCreated.size()); + // this includes the tmp index & data files, before the output is committed + assertEquals(4, tmpShuffleFilesCreated.size()); long sumOfPartitionSizes = 0; for (long size: partitionSizesInMergedFile) { @@ -416,10 +436,12 @@ public void writeEnoughDataToTriggerSpill() throws Exception { for (int i = 0; i < 128 + 1; i++) { dataToWrite.add(new Tuple2(i, bigByteArray)); } - writer.write(dataToWrite.iterator()); + Seq> files = writer.write(dataToWrite.iterator()); verify(shuffleMemoryManager, times(5)).tryToAcquire(anyLong()); - assertEquals(2, spillFilesCreated.size()); + // this includes the tmp index & data files, before the output is committed + assertEquals(4, tmpShuffleFilesCreated.size()); writer.stop(true); + ShuffleOutputCoordinator.commitOutputs(0, 0, files); readRecordsFromFile(); assertSpillFilesWereCleanedUp(); ShuffleWriteMetrics shuffleWriteMetrics = taskMetrics.shuffleWriteMetrics().get(); @@ -442,10 +464,12 @@ public void writeEnoughRecordsToTriggerSortBufferExpansionAndSpill() throws Exce for (int i = 0; i < UnsafeShuffleWriter.INITIAL_SORT_BUFFER_SIZE; i++) { dataToWrite.add(new Tuple2(i, i)); } - writer.write(dataToWrite.iterator()); + Seq> files = writer.write(dataToWrite.iterator()); verify(shuffleMemoryManager, times(5)).tryToAcquire(anyLong()); - assertEquals(2, spillFilesCreated.size()); + // this includes the tmp index & data files, before the output is committed + assertEquals(4, tmpShuffleFilesCreated.size()); writer.stop(true); + ShuffleOutputCoordinator.commitOutputs(0, 0, files); readRecordsFromFile(); assertSpillFilesWereCleanedUp(); ShuffleWriteMetrics shuffleWriteMetrics = taskMetrics.shuffleWriteMetrics().get(); @@ -464,8 +488,9 @@ public void writeRecordsThatAreBiggerThanDiskWriteBufferSize() throws Exception final byte[] bytes = new byte[(int) (ShuffleExternalSorter.DISK_WRITE_BUFFER_SIZE * 2.5)]; new Random(42).nextBytes(bytes); dataToWrite.add(new Tuple2(1, ByteBuffer.wrap(bytes))); - writer.write(dataToWrite.iterator()); + Seq> files = writer.write(dataToWrite.iterator()); writer.stop(true); + ShuffleOutputCoordinator.commitOutputs(0, 0, files); assertEquals( HashMultiset.create(dataToWrite), HashMultiset.create(readRecordsFromFile())); @@ -485,8 +510,9 @@ public void writeRecordsThatAreBiggerThanMaxRecordSize() throws Exception { final byte[] exceedsMaxRecordSize = new byte[writer.maxRecordSizeBytes() + 1]; new Random(42).nextBytes(exceedsMaxRecordSize); dataToWrite.add(new Tuple2(3, ByteBuffer.wrap(exceedsMaxRecordSize))); - writer.write(dataToWrite.iterator()); + Seq> files = writer.write(dataToWrite.iterator()); writer.stop(true); + ShuffleOutputCoordinator.commitOutputs(0, 0, files); assertEquals( HashMultiset.create(dataToWrite), HashMultiset.create(readRecordsFromFile())); From 550e1983ee170928e816642101b5171b517cbd4c Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Thu, 22 Oct 2015 14:51:36 -0500 Subject: [PATCH 07/33] fix imports --- .../org/apache/spark/shuffle/sort/SortShuffleWriter.scala | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) 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 9746f20a124eb..6146338a18938 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 @@ -22,9 +22,8 @@ import java.io.File import org.apache.spark._ import org.apache.spark.executor.ShuffleWriteMetrics import org.apache.spark.scheduler.MapStatus -import org.apache.spark.serializer.Serializer -import org.apache.spark.shuffle.{ShuffleOutputCoordinator, IndexShuffleBlockResolver, ShuffleWriter, BaseShuffleHandle} -import org.apache.spark.storage.{ShuffleIndexBlockId, ShuffleBlockId} +import org.apache.spark.shuffle.{BaseShuffleHandle, IndexShuffleBlockResolver, ShuffleWriter} +import org.apache.spark.storage.{ShuffleBlockId, ShuffleIndexBlockId} import org.apache.spark.util.collection.ExternalSorter private[spark] class SortShuffleWriter[K, V, C]( From 4ff98bf47440e5634377ffde445523b2c32254ca Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Thu, 22 Oct 2015 22:13:08 -0500 Subject: [PATCH 08/33] should work now, but needs cleanup --- .../sort/BypassMergeSortShuffleWriter.java | 35 +++++++++++++++---- .../shuffle/sort/UnsafeShuffleWriter.java | 6 ++-- .../shuffle/ShuffleOutputCoordinator.scala | 30 +++++++++++----- .../BypassMergeSortShuffleWriterSuite.scala | 32 ++++++++++++++--- 4 files changed, 80 insertions(+), 23 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 ee82d679935c0..703e01e3127df 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 @@ -21,6 +21,7 @@ import java.io.FileInputStream; import java.io.FileOutputStream; import java.io.IOException; +import java.util.Arrays; import javax.annotation.Nullable; import scala.None$; @@ -41,12 +42,15 @@ import org.apache.spark.executor.ShuffleWriteMetrics; import org.apache.spark.scheduler.MapStatus; import org.apache.spark.scheduler.MapStatus$; +import org.apache.spark.shuffle.IndexShuffleBlockResolver$; import org.apache.spark.serializer.Serializer; import org.apache.spark.serializer.SerializerInstance; import org.apache.spark.shuffle.IndexShuffleBlockResolver; import org.apache.spark.shuffle.ShuffleWriter; import org.apache.spark.storage.*; import org.apache.spark.util.Utils; +import scala.collection.JavaConverters; +import scala.collection.Seq; /** * This class implements sort-based shuffle's hash-style shuffle fallback path. This write path @@ -121,13 +125,26 @@ public BypassMergeSortShuffleWriter( } @Override - public void write(Iterator> records) throws IOException { + public Seq> write(Iterator> records) throws IOException { assert (partitionWriters == null); + File indexFile = blockManager.diskBlockManager().getFile(new ShuffleIndexBlockId( + shuffleId, mapId, IndexShuffleBlockResolver$.MODULE$.NOOP_REDUCE_ID()) + ); + File dataFile = shuffleBlockResolver.getDataFile(shuffleId, mapId); if (!records.hasNext()) { partitionLengths = new long[numPartitions]; - shuffleBlockResolver.writeIndexFile(shuffleId, mapId, partitionLengths); + File tmpIndexFile = shuffleBlockResolver.writeIndexFile(shuffleId, mapId, partitionLengths); + if (tmpIndexFile == indexFile) { + throw new RuntimeException("oops, index files equal"); + } mapStatus = MapStatus$.MODULE$.apply(blockManager.shuffleServerId(), partitionLengths); - return; + // create an empty data file + File tmpDataFile = blockManager.diskBlockManager().createTempShuffleBlock()._2(); + tmpDataFile.createNewFile(); + return JavaConverters.asScalaBufferConverter(Arrays.asList( + new Tuple2<>(tmpIndexFile, indexFile), + new Tuple2<>(tmpDataFile, dataFile) + )).asScala(); } final SerializerInstance serInstance = serializer.newInstance(); final long openStartTime = System.nanoTime(); @@ -155,10 +172,16 @@ public void write(Iterator> records) throws IOException { writer.commitAndClose(); } - partitionLengths = - writePartitionedFile(shuffleBlockResolver.getDataFile(shuffleId, mapId)); - shuffleBlockResolver.writeIndexFile(shuffleId, mapId, partitionLengths); + File tmpDataFile = blockManager.diskBlockManager().createTempShuffleBlock()._2(); + + partitionLengths = writePartitionedFile(tmpDataFile); + File tmpIndexFile = shuffleBlockResolver.writeIndexFile(shuffleId, mapId, partitionLengths); mapStatus = MapStatus$.MODULE$.apply(blockManager.shuffleServerId(), partitionLengths); + return JavaConverters.asScalaBufferConverter(Arrays.asList( + new Tuple2<>(tmpIndexFile, indexFile), + new Tuple2<>(tmpDataFile, dataFile) + )).asScala(); + } @VisibleForTesting 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 0235a43336a25..6f82cc81e13d5 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 @@ -22,16 +22,12 @@ import java.nio.channels.FileChannel; import java.util.Arrays; import java.util.Iterator; -import java.util.List; -import org.apache.spark.shuffle.IndexShuffleBlockResolver$; -import org.apache.spark.storage.ShuffleIndexBlockId; import scala.Option; import scala.Product2; import scala.Tuple2; import scala.collection.JavaConverters; import scala.collection.Seq; -import scala.collection.Seq$; import scala.collection.immutable.Map; import scala.reflect.ClassTag; import scala.reflect.ClassTag$; @@ -56,9 +52,11 @@ import org.apache.spark.serializer.Serializer; import org.apache.spark.serializer.SerializerInstance; import org.apache.spark.shuffle.IndexShuffleBlockResolver; +import org.apache.spark.shuffle.IndexShuffleBlockResolver$; import org.apache.spark.shuffle.ShuffleMemoryManager; import org.apache.spark.shuffle.ShuffleWriter; import org.apache.spark.storage.BlockManager; +import org.apache.spark.storage.ShuffleIndexBlockId; import org.apache.spark.storage.TimeTrackingOutputStream; import org.apache.spark.unsafe.Platform; import org.apache.spark.unsafe.memory.TaskMemoryManager; diff --git a/core/src/main/scala/org/apache/spark/shuffle/ShuffleOutputCoordinator.scala b/core/src/main/scala/org/apache/spark/shuffle/ShuffleOutputCoordinator.scala index c54ff41d39041..e8ea651d4bf06 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/ShuffleOutputCoordinator.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/ShuffleOutputCoordinator.scala @@ -28,8 +28,8 @@ import org.apache.spark.Logging object ShuffleOutputCoordinator extends Logging { /** - * if any of the destination files do not exist, then move all of the temporary files to their - * destinations. If all destination files exist, then simply delete all temporary files + * if all of the destination files do not exist, then move all of the temporary files to their + * destinations. If any destination files exist, then simply delete all temporary files * * @param tmpToDest pairs of (temporary, destination) file pairs * @return @@ -37,15 +37,29 @@ object ShuffleOutputCoordinator extends Logging { def commitOutputs( shuffleId: Int, partitionId: Int, tmpToDest: Seq[(File, File)]): Boolean = synchronized { - val prevExists = tmpToDest.forall(_._2.exists) - if (!prevExists) { - tmpToDest.foreach { case (tmp, dest) => - tmp.renameTo(dest) + logInfo(s"renaming: $tmpToDest") + val someDestAlreadyExists = tmpToDest.exists(_._2.exists) + if (!someDestAlreadyExists) { + // if any of the renames fail, delete all the dest files. otherwise, future + // attempts have no hope of succeeding + val renamesSucceeded = tmpToDest.map { case (tmp, dest) => + logInfo(s"trying to rename: $tmp -> $dest. ${tmp.exists()}; ${dest.exists()}") + val r = tmp.renameTo(dest) + if (!r) { + logInfo(s"failed to rename $tmp to $dest. ${tmp.exists()}; ${dest.exists()}") + } + r + }.forall{identity} + if (!renamesSucceeded) { + tmpToDest.foreach { case (tmp, dest) => if (dest.exists()) dest.delete() } + false + } else { + true } - true } else { logInfo(s"shuffle output for shuffle $shuffleId, partition $partitionId already exists, " + - s"not overwriting. Another task must have created this shuffle output.") + s"not overwriting. Another task must have created this shuffle output:" + + tmpToDest.map{_._2}.filter{_.exists()}) tmpToDest.foreach{ case (tmp, _) => tmp.delete()} false } 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 b92a302806f76..b0ccbdd773c2e 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.scalatest.BeforeAndAfterEach import org.apache.spark._ import org.apache.spark.executor.{TaskMetrics, ShuffleWriteMetrics} -import org.apache.spark.shuffle.IndexShuffleBlockResolver +import org.apache.spark.shuffle.{ShuffleOutputCoordinator, IndexShuffleBlockResolver$, IndexShuffleBlockResolver} import org.apache.spark.serializer.{JavaSerializer, SerializerInstance} import org.apache.spark.storage._ import org.apache.spark.util.Utils @@ -49,6 +49,7 @@ class BypassMergeSortShuffleWriterSuite extends SparkFunSuite with BeforeAndAfte private var taskMetrics: TaskMetrics = _ private var tempDir: File = _ private var outputFile: File = _ + private var indexFile: File = _ private val conf: SparkConf = new SparkConf(loadDefaults = false) private val temporaryFilesCreated: mutable.Buffer[File] = new ArrayBuffer[File]() private val blockIdToFileMap: mutable.Map[BlockId, File] = new mutable.HashMap[BlockId, File] @@ -57,6 +58,10 @@ class BypassMergeSortShuffleWriterSuite extends SparkFunSuite with BeforeAndAfte override def beforeEach(): Unit = { tempDir = Utils.createTempDir() outputFile = File.createTempFile("shuffle", null, tempDir) + indexFile = File.createTempFile("shuffle", ".index", tempDir) + // ShuffleOutputCoordinator requires these files to not exist yet + outputFile.delete() + indexFile.delete() taskMetrics = new TaskMetrics MockitoAnnotations.initMocks(this) shuffleHandle = new BypassMergeSortShuffleHandle[Int, Int]( @@ -68,6 +73,14 @@ class BypassMergeSortShuffleWriterSuite extends SparkFunSuite with BeforeAndAfte when(dependency.serializer).thenReturn(Some(new JavaSerializer(conf))) when(taskContext.taskMetrics()).thenReturn(taskMetrics) when(blockResolver.getDataFile(0, 0)).thenReturn(outputFile) + // the index file will be empty, but that is fine for these tests + when(blockResolver.writeIndexFile(anyInt(), anyInt(), any())).thenAnswer(new Answer[File] { + override def answer(invocationOnMock: InvocationOnMock): File = { + val f = diskBlockManager.createTempShuffleBlock()._2 + f.createNewFile() + f + } + }) when(blockManager.diskBlockManager).thenReturn(diskBlockManager) when(blockManager.getDiskWriter( any[BlockId], @@ -93,6 +106,7 @@ class BypassMergeSortShuffleWriterSuite extends SparkFunSuite with BeforeAndAfte override def answer(invocation: InvocationOnMock): (TempShuffleBlockId, File) = { val blockId = new TempShuffleBlockId(UUID.randomUUID) val file = File.createTempFile(blockId.toString, null, tempDir) + file.delete() blockIdToFileMap.put(blockId, file) temporaryFilesCreated.append(file) (blockId, file) @@ -101,7 +115,12 @@ class BypassMergeSortShuffleWriterSuite extends SparkFunSuite with BeforeAndAfte when(diskBlockManager.getFile(any[BlockId])).thenAnswer( new Answer[File] { override def answer(invocation: InvocationOnMock): File = { - blockIdToFileMap.get(invocation.getArguments.head.asInstanceOf[BlockId]).get + val blk = invocation.getArguments.head.asInstanceOf[BlockId] + if (blk == new ShuffleIndexBlockId(0, 0, IndexShuffleBlockResolver.NOOP_REDUCE_ID)) { + indexFile + } else { + blockIdToFileMap.get(invocation.getArguments.head.asInstanceOf[BlockId]).get + } } }) } @@ -121,12 +140,14 @@ class BypassMergeSortShuffleWriterSuite extends SparkFunSuite with BeforeAndAfte taskContext, conf ) - writer.write(Iterator.empty) + val files = writer.write(Iterator.empty) writer.stop( /* success = */ true) + assert(files.map{_._2}.contains(outputFile)) + assert(ShuffleOutputCoordinator.commitOutputs(0, 0, files)) assert(writer.getPartitionLengths.sum === 0) assert(outputFile.exists()) assert(outputFile.length() === 0) - assert(temporaryFilesCreated.isEmpty) + assert(temporaryFilesCreated.size === 2) val shuffleWriteMetrics = taskContext.taskMetrics().shuffleWriteMetrics.get assert(shuffleWriteMetrics.shuffleBytesWritten === 0) assert(shuffleWriteMetrics.shuffleRecordsWritten === 0) @@ -145,8 +166,9 @@ class BypassMergeSortShuffleWriterSuite extends SparkFunSuite with BeforeAndAfte taskContext, conf ) - writer.write(records) + val files = writer.write(records) writer.stop( /* success = */ true) + ShuffleOutputCoordinator.commitOutputs(0, 0, files) assert(temporaryFilesCreated.nonEmpty) assert(writer.getPartitionLengths.sum === outputFile.length()) assert(temporaryFilesCreated.count(_.exists()) === 0) // check that temporary files were deleted From 4a19702bf1de0af003c2cbc58bf2ec61c79d17b9 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Fri, 23 Oct 2015 11:20:19 -0500 Subject: [PATCH 09/33] only consider tmp files that exist; only consider the dest pre-existing if *all* files exist --- .../shuffle/ShuffleOutputCoordinator.scala | 22 ++++++++++++------- 1 file changed, 14 insertions(+), 8 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/shuffle/ShuffleOutputCoordinator.scala b/core/src/main/scala/org/apache/spark/shuffle/ShuffleOutputCoordinator.scala index e8ea651d4bf06..bfb51b17211dc 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/ShuffleOutputCoordinator.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/ShuffleOutputCoordinator.scala @@ -38,12 +38,19 @@ object ShuffleOutputCoordinator extends Logging { shuffleId: Int, partitionId: Int, tmpToDest: Seq[(File, File)]): Boolean = synchronized { logInfo(s"renaming: $tmpToDest") - val someDestAlreadyExists = tmpToDest.exists(_._2.exists) - if (!someDestAlreadyExists) { + + // HashShuffleWriter might not write any records to some of its files -- that's OK, we only + // move the files that do exist + val toMove = tmpToDest.filter{_._1.exists()} + + val destAlreadyExists = toMove.forall(_._2.exists) + if (!destAlreadyExists) { // if any of the renames fail, delete all the dest files. otherwise, future // attempts have no hope of succeeding - val renamesSucceeded = tmpToDest.map { case (tmp, dest) => - logInfo(s"trying to rename: $tmp -> $dest. ${tmp.exists()}; ${dest.exists()}") + val renamesSucceeded = toMove.map { case (tmp, dest) => + if (dest.exists()) { + dest.delete() + } val r = tmp.renameTo(dest) if (!r) { logInfo(s"failed to rename $tmp to $dest. ${tmp.exists()}; ${dest.exists()}") @@ -51,16 +58,15 @@ object ShuffleOutputCoordinator extends Logging { r }.forall{identity} if (!renamesSucceeded) { - tmpToDest.foreach { case (tmp, dest) => if (dest.exists()) dest.delete() } + toMove.foreach { case (tmp, dest) => if (dest.exists()) dest.delete() } false } else { true } } else { logInfo(s"shuffle output for shuffle $shuffleId, partition $partitionId already exists, " + - s"not overwriting. Another task must have created this shuffle output:" + - tmpToDest.map{_._2}.filter{_.exists()}) - tmpToDest.foreach{ case (tmp, _) => tmp.delete()} + s"not overwriting. Another task must have created this shuffle output.") + toMove.foreach{ case (tmp, _) => tmp.delete()} false } } From 89063dd3654066e076e0e4f13250d25c414e88c3 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Fri, 23 Oct 2015 11:31:08 -0500 Subject: [PATCH 10/33] cleanup --- .../shuffle/sort/BypassMergeSortShuffleWriter.java | 13 +++---------- .../sort/BypassMergeSortShuffleWriterSuite.scala | 12 +++++++----- 2 files changed, 10 insertions(+), 15 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 703e01e3127df..0e5945912a408 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 @@ -127,23 +127,16 @@ public BypassMergeSortShuffleWriter( @Override public Seq> write(Iterator> records) throws IOException { assert (partitionWriters == null); - File indexFile = blockManager.diskBlockManager().getFile(new ShuffleIndexBlockId( + final File indexFile = blockManager.diskBlockManager().getFile(new ShuffleIndexBlockId( shuffleId, mapId, IndexShuffleBlockResolver$.MODULE$.NOOP_REDUCE_ID()) ); - File dataFile = shuffleBlockResolver.getDataFile(shuffleId, mapId); + final File dataFile = shuffleBlockResolver.getDataFile(shuffleId, mapId); if (!records.hasNext()) { partitionLengths = new long[numPartitions]; File tmpIndexFile = shuffleBlockResolver.writeIndexFile(shuffleId, mapId, partitionLengths); - if (tmpIndexFile == indexFile) { - throw new RuntimeException("oops, index files equal"); - } mapStatus = MapStatus$.MODULE$.apply(blockManager.shuffleServerId(), partitionLengths); - // create an empty data file - File tmpDataFile = blockManager.diskBlockManager().createTempShuffleBlock()._2(); - tmpDataFile.createNewFile(); return JavaConverters.asScalaBufferConverter(Arrays.asList( - new Tuple2<>(tmpIndexFile, indexFile), - new Tuple2<>(tmpDataFile, dataFile) + new Tuple2<>(tmpIndexFile, indexFile) )).asScala(); } final SerializerInstance serInstance = serializer.newInstance(); 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 b0ccbdd773c2e..2349b2f074230 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.scalatest.BeforeAndAfterEach import org.apache.spark._ import org.apache.spark.executor.{TaskMetrics, ShuffleWriteMetrics} -import org.apache.spark.shuffle.{ShuffleOutputCoordinator, IndexShuffleBlockResolver$, IndexShuffleBlockResolver} +import org.apache.spark.shuffle.{ShuffleOutputCoordinator, IndexShuffleBlockResolver} import org.apache.spark.serializer.{JavaSerializer, SerializerInstance} import org.apache.spark.storage._ import org.apache.spark.util.Utils @@ -142,12 +142,14 @@ class BypassMergeSortShuffleWriterSuite extends SparkFunSuite with BeforeAndAfte ) val files = writer.write(Iterator.empty) writer.stop( /* success = */ true) - assert(files.map{_._2}.contains(outputFile)) assert(ShuffleOutputCoordinator.commitOutputs(0, 0, files)) assert(writer.getPartitionLengths.sum === 0) - assert(outputFile.exists()) - assert(outputFile.length() === 0) - assert(temporaryFilesCreated.size === 2) + if (outputFile.exists()) { + assert(outputFile.length() === 0) + assert(temporaryFilesCreated.size === 2) + } else { + assert(temporaryFilesCreated.size === 1) + } val shuffleWriteMetrics = taskContext.taskMetrics().shuffleWriteMetrics.get assert(shuffleWriteMetrics.shuffleBytesWritten === 0) assert(shuffleWriteMetrics.shuffleRecordsWritten === 0) From 4145651724eb99fb440cc8509df154d8f8095b47 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Fri, 23 Oct 2015 12:19:59 -0500 Subject: [PATCH 11/33] ShuffleOutputCoordinatorSuite --- .../ShuffleOutputCoordinatorSuite.scala | 126 ++++++++++++++++++ 1 file changed, 126 insertions(+) create mode 100644 core/src/test/scala/org/apache/spark/shuffle/ShuffleOutputCoordinatorSuite.scala diff --git a/core/src/test/scala/org/apache/spark/shuffle/ShuffleOutputCoordinatorSuite.scala b/core/src/test/scala/org/apache/spark/shuffle/ShuffleOutputCoordinatorSuite.scala new file mode 100644 index 0000000000000..f473322fc8c56 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/shuffle/ShuffleOutputCoordinatorSuite.scala @@ -0,0 +1,126 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.shuffle + +import java.io.{FileInputStream, FileOutputStream, File} + +import org.scalatest.BeforeAndAfterEach + +import org.apache.spark.SparkFunSuite +import org.apache.spark.util.Utils + +class ShuffleOutputCoordinatorSuite extends SparkFunSuite with BeforeAndAfterEach { + + var tempDir: File = _ + + override def beforeEach(): Unit = { + tempDir = Utils.createTempDir() + } + + def writeFile(filename: String, data: Int): File = { + val f = new File(tempDir, filename) + val out = new FileOutputStream(f) + out.write(data) + out.close() + f + } + + def verifyFiles(successfulAttempt: Int): Unit = { + (0 until 3).foreach { idx => + val exp = successfulAttempt* 3 + idx + val file = new File(tempDir, s"d$idx") + withClue(s"checking dest file $file") { + assert(file.length === 1) + val in = new FileInputStream(file) + assert(in.read() === exp) + in.close() + + } + } + } + + override def afterEach(): Unit = { + Utils.deleteRecursively(tempDir) + } + + + def generateAttempt(attempt: Int): Seq[(File, File)] = { + (0 until 3).map { idx => + val j = attempt * 3 + idx + writeFile(s"t$j", j) -> new File(tempDir, s"d$idx") + } + } + + test("move files if dest missing") { + val firstAttempt = generateAttempt(0) + assert(ShuffleOutputCoordinator.commitOutputs(0, 0, firstAttempt)) + verifyFiles(0) + firstAttempt.foreach{ case (t, d) => assert(!t.exists())} + + val secondAttempt = generateAttempt(1) + // second commit fails, and also deletes the tmp files + assert(!ShuffleOutputCoordinator.commitOutputs(0, 0, secondAttempt)) + verifyFiles(0) + // make sure we delete the temp files if the dest exists + secondAttempt.foreach{ case (t, d) => assert(!t.exists())} + } + + test("move files if dest partially missing") { + val firstAttempt = generateAttempt(0) + assert(ShuffleOutputCoordinator.commitOutputs(0, 0, firstAttempt)) + verifyFiles(0) + firstAttempt.foreach{ case (t, d) => assert(!t.exists())} + + val secondAttempt = generateAttempt(1) + firstAttempt(0)._2.delete() + // second commit now succeeds since one destination file is missing + assert(ShuffleOutputCoordinator.commitOutputs(0, 0, secondAttempt)) + verifyFiles(1) + secondAttempt.foreach{ case (t, d) => assert(!t.exists())} + } + + test("ignore missing tmp files") { + // HashShuffle doesn't necessarily even create 0 length files for all of its output, + // so just ignore tmp files that are missing + val firstAttempt = generateAttempt(0) ++ + Seq(new File(tempDir, "bogus") -> new File(tempDir, "blah")) + assert(ShuffleOutputCoordinator.commitOutputs(0, 0, firstAttempt)) + verifyFiles(0) + assert(!new File(tempDir, "blah").exists()) + firstAttempt.foreach{ case (t, d) => assert(!t.exists())} + + // if we try again, once more with the missing tmp file, commit fails even though dest + // is "partially missing" + // TODO figure out right semantics, esp wrt non-determinstic data + val secondAttempt = generateAttempt(1) ++ + Seq(new File(tempDir, "bogus") -> new File(tempDir, "blah")) + assert(!ShuffleOutputCoordinator.commitOutputs(0, 0, secondAttempt)) + verifyFiles(0) + assert(!new File(tempDir, "blah").exists()) + secondAttempt.foreach{ case (t, d) => assert(!t.exists())} + + // but now if we delete one of the real dest files, and try again, it goes through + val thirdAttempt = generateAttempt(2) ++ + Seq(new File(tempDir, "bogus") -> new File(tempDir, "blah")) + firstAttempt(0)._2.delete() + assert(ShuffleOutputCoordinator.commitOutputs(0, 0, thirdAttempt)) + verifyFiles(2) + assert(!new File(tempDir, "blah").exists()) + thirdAttempt.foreach{ case (t, d) => assert(!t.exists())} + } + +} From 2089e12d93ff00f0d18e0ec3cbf23322a6974831 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Fri, 23 Oct 2015 14:17:51 -0500 Subject: [PATCH 12/33] cleanup --- .../shuffle/ShuffleOutputCoordinator.scala | 33 ++++++++----------- 1 file changed, 13 insertions(+), 20 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/shuffle/ShuffleOutputCoordinator.scala b/core/src/main/scala/org/apache/spark/shuffle/ShuffleOutputCoordinator.scala index bfb51b17211dc..a1c390c4b9ee7 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/ShuffleOutputCoordinator.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/ShuffleOutputCoordinator.scala @@ -28,41 +28,34 @@ import org.apache.spark.Logging object ShuffleOutputCoordinator extends Logging { /** - * if all of the destination files do not exist, then move all of the temporary files to their - * destinations. If any destination files exist, then simply delete all temporary files + * if any of the destination files do not exist, then move all of the temporary files to their + * destinations. If all destination files exist, then delete all temporary files. * * @param tmpToDest pairs of (temporary, destination) file pairs * @return */ def commitOutputs( shuffleId: Int, - partitionId: Int, tmpToDest: Seq[(File, File)]): Boolean = synchronized { - logInfo(s"renaming: $tmpToDest") + partitionId: Int, + tmpToDest: Seq[(File, File)]): Boolean = synchronized { - // HashShuffleWriter might not write any records to some of its files -- that's OK, we only - // move the files that do exist + // There might not even be zero-length files for some of the temp files -- that's OK, + // we just ignore those val toMove = tmpToDest.filter{_._1.exists()} val destAlreadyExists = toMove.forall(_._2.exists) if (!destAlreadyExists) { - // if any of the renames fail, delete all the dest files. otherwise, future - // attempts have no hope of succeeding - val renamesSucceeded = toMove.map { case (tmp, dest) => + toMove.foreach { case (tmp, dest) => + // If *some* of the destination files exist, but not all of them, then its not clear + // what to do. There could be a task already reading from this dest file when we delete + // it -- but then again, something in that taskset would be doomed to fail in any case when + // it got to the missing files. Better to just put consistent output into place if (dest.exists()) { dest.delete() } - val r = tmp.renameTo(dest) - if (!r) { - logInfo(s"failed to rename $tmp to $dest. ${tmp.exists()}; ${dest.exists()}") - } - r - }.forall{identity} - if (!renamesSucceeded) { - toMove.foreach { case (tmp, dest) => if (dest.exists()) dest.delete() } - false - } else { - true + tmp.renameTo(dest) } + true } else { logInfo(s"shuffle output for shuffle $shuffleId, partition $partitionId already exists, " + s"not overwriting. Another task must have created this shuffle output.") From 4cd423e35bf0bdccbcb1966e81104926178bc5b7 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Mon, 26 Oct 2015 15:50:52 -0500 Subject: [PATCH 13/33] write the winning mapStatus to disk, so subsequent tasks can respond w/ the same status --- .../sort/BypassMergeSortShuffleWriter.java | 6 + .../shuffle/sort/UnsafeShuffleWriter.java | 7 ++ .../spark/scheduler/ShuffleMapTask.scala | 4 +- .../shuffle/FileShuffleBlockResolver.scala | 9 ++ .../shuffle/IndexShuffleBlockResolver.scala | 7 ++ .../shuffle/ShuffleOutputCoordinator.scala | 28 +++-- .../apache/spark/shuffle/ShuffleWriter.scala | 7 ++ .../shuffle/hash/HashShuffleWriter.scala | 6 +- .../shuffle/sort/SortShuffleWriter.scala | 7 +- .../org/apache/spark/storage/BlockId.scala | 10 ++ .../sort/UnsafeShuffleWriterSuite.java | 33 ++++-- .../scala/org/apache/spark/ShuffleSuite.scala | 26 ++++- .../org/apache/spark/SortShuffleSuite.scala | 4 +- .../ShuffleOutputCoordinatorSuite.scala | 108 +++++++++++------- .../BypassMergeSortShuffleWriterSuite.scala | 15 ++- 15 files changed, 195 insertions(+), 82 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 0e5945912a408..78bbccdb3d0bb 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 @@ -252,4 +252,10 @@ public Option stop(boolean success) { } } } + + @Override + public File mapStatusFile() { + return blockManager.diskBlockManager().getFile(new ShuffleMapStatusBlockId(shuffleId, mapId)); + } + } 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 314fc861ee141..198cc69a38058 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 @@ -56,6 +56,7 @@ import org.apache.spark.shuffle.ShuffleWriter; import org.apache.spark.storage.BlockManager; import org.apache.spark.storage.ShuffleIndexBlockId; +import org.apache.spark.storage.ShuffleMapStatusBlockId; import org.apache.spark.storage.TimeTrackingOutputStream; import org.apache.spark.unsafe.Platform; import org.apache.spark.memory.TaskMemoryManager; @@ -503,4 +504,10 @@ public Option stop(boolean success) { } } } + + @Override + public File mapStatusFile() { + return blockManager.diskBlockManager().getFile(new ShuffleMapStatusBlockId(shuffleId, mapId)); + } + } diff --git a/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala b/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala index 4c1c8f2608eee..553fc1df77cd3 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala @@ -74,8 +74,8 @@ private[spark] class ShuffleMapTask( rdd.iterator(partition, context).asInstanceOf[Iterator[_ <: Product2[Any, Any]]]) val mapStatus = writer.stop(success = true).get // SPARK-8029 make sure only one task on this executor writes the final shuffle files - ShuffleOutputCoordinator.commitOutputs(dep.shuffleId, partitionId, tmpToDestFiles) - mapStatus + ShuffleOutputCoordinator.commitOutputs(dep.shuffleId, partitionId, tmpToDestFiles, mapStatus, + writer.mapStatusFile, SparkEnv.get.serializer.newInstance())._2 } catch { case e: Exception => try { diff --git a/core/src/main/scala/org/apache/spark/shuffle/FileShuffleBlockResolver.scala b/core/src/main/scala/org/apache/spark/shuffle/FileShuffleBlockResolver.scala index 90dc07a84e59e..2805e5c1655b2 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/FileShuffleBlockResolver.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/FileShuffleBlockResolver.scala @@ -134,6 +134,15 @@ private[spark] class FileShuffleBlockResolver(conf: SparkConf) logWarning(s"Error deleting ${file.getPath()}") } } + for (mapId <- state.completedMapTasks.asScala) { + val mapStatusFile = + blockManager.diskBlockManager.getFile(ShuffleMapStatusBlockId(shuffleId, mapId)) + if (mapStatusFile.exists()) { + if (!mapStatusFile.delete()) { + logWarning(s"Error deleting MapStatus file ${mapStatusFile.getPath()}") + } + } + } logInfo("Deleted all files for shuffle " + shuffleId) true case None => 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 5910d2057542c..fb5b981f427c4 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala @@ -72,6 +72,13 @@ private[spark] class IndexShuffleBlockResolver(conf: SparkConf) extends ShuffleB logWarning(s"Error deleting index ${file.getPath()}") } } + + file = blockManager.diskBlockManager.getFile(ShuffleMapStatusBlockId(shuffleId, mapId)) + if (file.exists()) { + if (!file.delete()) { + logWarning(s"Error deleting MapStatus file ${file.getPath()}") + } + } } /** diff --git a/core/src/main/scala/org/apache/spark/shuffle/ShuffleOutputCoordinator.scala b/core/src/main/scala/org/apache/spark/shuffle/ShuffleOutputCoordinator.scala index a1c390c4b9ee7..a57107eb58179 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/ShuffleOutputCoordinator.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/ShuffleOutputCoordinator.scala @@ -16,9 +16,11 @@ */ package org.apache.spark.shuffle -import java.io.File +import java.io.{FileOutputStream, FileInputStream, File} import org.apache.spark.Logging +import org.apache.spark.scheduler.MapStatus +import org.apache.spark.serializer.{SerializerInstance, Serializer} /** * Ensures that on each executor, there are no conflicting writes to the same shuffle files. It @@ -37,14 +39,12 @@ object ShuffleOutputCoordinator extends Logging { def commitOutputs( shuffleId: Int, partitionId: Int, - tmpToDest: Seq[(File, File)]): Boolean = synchronized { - - // There might not even be zero-length files for some of the temp files -- that's OK, - // we just ignore those - val toMove = tmpToDest.filter{_._1.exists()} - - val destAlreadyExists = toMove.forall(_._2.exists) - if (!destAlreadyExists) { + tmpToDest: Seq[(File, File)], + mapStatus: MapStatus, + mapStatusFile: File, + serializer: SerializerInstance): (Boolean, MapStatus) = synchronized { + val toMove = tmpToDest.filter { _._1.exists()} + if (!mapStatusFile.exists()) { toMove.foreach { case (tmp, dest) => // If *some* of the destination files exist, but not all of them, then its not clear // what to do. There could be a task already reading from this dest file when we delete @@ -55,12 +55,18 @@ object ShuffleOutputCoordinator extends Logging { } tmp.renameTo(dest) } - true + val out = serializer.serializeStream(new FileOutputStream(mapStatusFile)) + out.writeObject(mapStatus) + out.close() + (true, mapStatus) } else { logInfo(s"shuffle output for shuffle $shuffleId, partition $partitionId already exists, " + s"not overwriting. Another task must have created this shuffle output.") toMove.foreach{ case (tmp, _) => tmp.delete()} - false + val in = serializer.deserializeStream(new FileInputStream(mapStatusFile)) + val readStatus = in.readObject[MapStatus] + in.close() + (false, readStatus) } } } diff --git a/core/src/main/scala/org/apache/spark/shuffle/ShuffleWriter.scala b/core/src/main/scala/org/apache/spark/shuffle/ShuffleWriter.scala index 56af1590b088c..7410d19516c3f 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/ShuffleWriter.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/ShuffleWriter.scala @@ -34,6 +34,13 @@ private[spark] abstract class ShuffleWriter[K, V] { @throws[IOException] def write(records: Iterator[Product2[K, V]]): Seq[(File, File)] + /** + * The canonical location of a file for storing the serialized output, to coordinate between + * multiple tasks that may write the same shuffle data + * @return + */ + def mapStatusFile: File + /** Close this writer, passing along whether the map completed */ def stop(success: Boolean): Option[MapStatus] } diff --git a/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleWriter.scala b/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleWriter.scala index 76d61a04ca310..793830563c73a 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleWriter.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleWriter.scala @@ -24,7 +24,7 @@ import org.apache.spark.executor.ShuffleWriteMetrics import org.apache.spark.scheduler.MapStatus import org.apache.spark.serializer.Serializer import org.apache.spark.shuffle._ -import org.apache.spark.storage.DiskBlockObjectWriter +import org.apache.spark.storage.{DiskBlockObjectWriter, ShuffleMapStatusBlockId} private[spark] class HashShuffleWriter[K, V]( shuffleBlockResolver: FileShuffleBlockResolver, @@ -103,6 +103,10 @@ private[spark] class HashShuffleWriter[K, V]( } } + override def mapStatusFile: File = { + blockManager.diskBlockManager.getFile(ShuffleMapStatusBlockId(handle.shuffleId, mapId)) + } + private def commitWritesAndBuildStatus(): MapStatus = { // Commit the writes. Get the size of each bucket block (total block size). val sizes: Array[Long] = shuffle.writers.map { case (writer: DiskBlockObjectWriter, _) => 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 de778cd717492..b7e7d9dae5c5c 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 @@ -23,7 +23,7 @@ import org.apache.spark._ import org.apache.spark.executor.ShuffleWriteMetrics import org.apache.spark.scheduler.MapStatus import org.apache.spark.shuffle.{BaseShuffleHandle, IndexShuffleBlockResolver, ShuffleWriter} -import org.apache.spark.storage.{ShuffleBlockId, ShuffleIndexBlockId} +import org.apache.spark.storage.{ShuffleBlockId, ShuffleIndexBlockId, ShuffleMapStatusBlockId} import org.apache.spark.util.collection.ExternalSorter private[spark] class SortShuffleWriter[K, V, C]( @@ -107,6 +107,11 @@ private[spark] class SortShuffleWriter[K, V, C]( } } } + + override def mapStatusFile: File = { + blockManager.diskBlockManager.getFile(ShuffleMapStatusBlockId(handle.shuffleId, mapId)) + } + } private[spark] object SortShuffleWriter { 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 524f6970992a5..23d05a405d21a 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockId.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockId.scala @@ -60,6 +60,16 @@ case class ShuffleBlockId(shuffleId: Int, mapId: Int, reduceId: Int) extends Blo override def name: String = "shuffle_" + shuffleId + "_" + mapId + "_" + reduceId } +/** + * Used to get the canonical filename for the [[org.apache.spark.shuffle.ShuffleOutputCoordinator]] + * to store the MapStatus between attempts. See ShuffleOutputCoordinator for more details. Note + * that this "block" is never shared between executors, its just used between tasks on one executor. + * Its just a convenient way to get a canonical file to store this data. + */ +case class ShuffleMapStatusBlockId(shuffleId: Int, mapId: Int) extends BlockId { + override def name: String = "shuffle_" + shuffleId + "_" + mapId + ".mapstatus" +} + @DeveloperApi case class ShuffleDataBlockId(shuffleId: Int, mapId: Int, reduceId: Int) extends BlockId { override def name: String = "shuffle_" + shuffleId + "_" + mapId + "_" + reduceId + ".data" 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 0c896b14bcf85..29723a173e204 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 @@ -66,6 +66,7 @@ public class UnsafeShuffleWriterSuite { TaskMemoryManager taskMemoryManager; final HashPartitioner hashPartitioner = new HashPartitioner(NUM_PARTITITONS); File mergedOutputFile; + File mapStatusFile; File tempDir; long[] partitionSizesInMergedFile; final LinkedList tmpShuffleFilesCreated = new LinkedList(); @@ -105,8 +106,10 @@ public void setUp() throws IOException { MockitoAnnotations.initMocks(this); tempDir = Utils.createTempDir("test", "test"); mergedOutputFile = File.createTempFile("mergedoutput", "", tempDir); + mapStatusFile = File.createTempFile("shuffle", ".mapstatus", tempDir); // the ShuffleOutputCoordinator requires that this file does not exist mergedOutputFile.delete(); + mapStatusFile.delete(); partitionSizesInMergedFile = null; tmpShuffleFilesCreated.clear(); conf = new SparkConf() @@ -165,6 +168,7 @@ public OutputStream answer(InvocationOnMock invocation) throws Throwable { } } ); + when(blockManager.shuffleServerId()).thenReturn(BlockManagerId$.MODULE$.apply("1", "a.b.c", 1)); when(shuffleBlockResolver.getDataFile(anyInt(), anyInt())).thenReturn(mergedOutputFile); doAnswer(new Answer() { @@ -285,8 +289,9 @@ public void writeEmptyIterator() throws Exception { final UnsafeShuffleWriter writer = createWriter(true); Seq> files = writer.write(Iterators.>emptyIterator()); final Option mapStatus = writer.stop(true); - ShuffleOutputCoordinator.commitOutputs(0, 0, files); assertTrue(mapStatus.isDefined()); + ShuffleOutputCoordinator.commitOutputs(0, 0, files, mapStatus.get(), mapStatusFile, + serializer.newInstance()); assertTrue(mergedOutputFile.exists()); assertArrayEquals(new long[NUM_PARTITITONS], partitionSizesInMergedFile); assertEquals(0, taskMetrics.shuffleWriteMetrics().get().shuffleRecordsWritten()); @@ -306,7 +311,8 @@ public void writeWithoutSpilling() throws Exception { final UnsafeShuffleWriter writer = createWriter(true); Seq> files = writer.write(dataToWrite.iterator()); final Option mapStatus = writer.stop(true); - ShuffleOutputCoordinator.commitOutputs(0, 0, files); + ShuffleOutputCoordinator.commitOutputs(0, 0, files, mapStatus.get(), mapStatusFile, + serializer.newInstance()); assertTrue(mapStatus.isDefined()); assertTrue(mergedOutputFile.exists()); @@ -352,7 +358,8 @@ private void testMergingSpills( writer.insertRecordIntoSorter(dataToWrite.get(5)); Seq> files = writer.closeAndWriteOutput(); final Option mapStatus = writer.stop(true); - ShuffleOutputCoordinator.commitOutputs(0, 0, files); + ShuffleOutputCoordinator.commitOutputs(0, 0, files, mapStatus.get(), mapStatusFile, + serializer.newInstance()); assertTrue(mapStatus.isDefined()); assertTrue(mergedOutputFile.exists()); // this includes the tmp index & data files, before the output is committed @@ -434,8 +441,9 @@ public void writeEnoughDataToTriggerSpill() throws Exception { verify(taskMemoryManager, times(5)).acquireExecutionMemory(anyLong()); // this includes the tmp index & data files, before the output is committed assertEquals(4, tmpShuffleFilesCreated.size()); - writer.stop(true); - ShuffleOutputCoordinator.commitOutputs(0, 0, files); + MapStatus mapStatus = writer.stop(true).get(); + ShuffleOutputCoordinator.commitOutputs(0, 0, files, mapStatus, mapStatusFile, + serializer.newInstance()); readRecordsFromFile(); assertSpillFilesWereCleanedUp(); ShuffleWriteMetrics shuffleWriteMetrics = taskMetrics.shuffleWriteMetrics().get(); @@ -463,8 +471,9 @@ public void writeEnoughRecordsToTriggerSortBufferExpansionAndSpill() throws Exce verify(taskMemoryManager, times(5)).acquireExecutionMemory(anyLong()); // this includes the tmp index & data files, before the output is committed assertEquals(4, tmpShuffleFilesCreated.size()); - writer.stop(true); - ShuffleOutputCoordinator.commitOutputs(0, 0, files); + MapStatus mapStatus = writer.stop(true).get(); + ShuffleOutputCoordinator.commitOutputs(0, 0, files, mapStatus, mapStatusFile, + serializer.newInstance()); readRecordsFromFile(); assertSpillFilesWereCleanedUp(); ShuffleWriteMetrics shuffleWriteMetrics = taskMetrics.shuffleWriteMetrics().get(); @@ -484,8 +493,9 @@ public void writeRecordsThatAreBiggerThanDiskWriteBufferSize() throws Exception new Random(42).nextBytes(bytes); dataToWrite.add(new Tuple2(1, ByteBuffer.wrap(bytes))); Seq> files = writer.write(dataToWrite.iterator()); - writer.stop(true); - ShuffleOutputCoordinator.commitOutputs(0, 0, files); + MapStatus mapStatus = writer.stop(true).get(); + ShuffleOutputCoordinator.commitOutputs(0, 0, files, mapStatus, mapStatusFile, + serializer.newInstance()); assertEquals( HashMultiset.create(dataToWrite), HashMultiset.create(readRecordsFromFile())); @@ -506,8 +516,9 @@ public void writeRecordsThatAreBiggerThanMaxRecordSize() throws Exception { new Random(42).nextBytes(exceedsMaxRecordSize); dataToWrite.add(new Tuple2(3, ByteBuffer.wrap(exceedsMaxRecordSize))); Seq> files = writer.write(dataToWrite.iterator()); - writer.stop(true); - ShuffleOutputCoordinator.commitOutputs(0, 0, files); + MapStatus mapStatus = writer.stop(true).get(); + ShuffleOutputCoordinator.commitOutputs(0, 0, files, mapStatus, mapStatusFile, + serializer.newInstance()); assertEquals( HashMultiset.create(dataToWrite), HashMultiset.create(readRecordsFromFile())); diff --git a/core/src/test/scala/org/apache/spark/ShuffleSuite.scala b/core/src/test/scala/org/apache/spark/ShuffleSuite.scala index d6ad9caee1d63..ca419f1fa0318 100644 --- a/core/src/test/scala/org/apache/spark/ShuffleSuite.scala +++ b/core/src/test/scala/org/apache/spark/ShuffleSuite.scala @@ -26,7 +26,7 @@ import org.apache.spark.rdd.{CoGroupedRDD, OrderedRDDFunctions, RDD, ShuffledRDD import org.apache.spark.scheduler.{MapStatus, MyRDD, SparkListener, SparkListenerTaskEnd} import org.apache.spark.serializer.KryoSerializer import org.apache.spark.shuffle.{ShuffleWriter, ShuffleOutputCoordinator} -import org.apache.spark.storage.{ShuffleDataBlockId, ShuffleBlockId} +import org.apache.spark.storage.{ShuffleMapStatusBlockId, ShuffleDataBlockId, ShuffleBlockId} import org.apache.spark.memory.TaskMemoryManager import org.apache.spark.util.MutablePair @@ -324,8 +324,10 @@ abstract class ShuffleSuite extends SparkFunSuite with Matchers with LocalSparkC test("multiple simultaneous attempts for one task (SPARK-8029)") { sc = new SparkContext("local", "test", conf) + val mapStatusFile = sc.env.blockManager.diskBlockManager.getFile(ShuffleMapStatusBlockId(0, 0)) val mapTrackerMaster = sc.env.mapOutputTracker.asInstanceOf[MapOutputTrackerMaster] val manager = sc.env.shuffleManager + val taskMemoryManager = new TaskMemoryManager(sc.env.memoryManager, 0L) val metricsSystem = sc.env.metricsSystem val shuffleMapRdd = new MyRDD(sc, 1, Nil) @@ -351,11 +353,12 @@ abstract class ShuffleSuite extends SparkFunSuite with Matchers with LocalSparkC def writeAndClose( writer: ShuffleWriter[Int, Int])( - iter: Iterator[(Int, Int)]): Option[MapStatus] = { + iter: Iterator[(Int, Int)]): Option[(Boolean, MapStatus)] = { val files = writer.write(iter) val output = writer.stop(true) - ShuffleOutputCoordinator.commitOutputs(0, 0, files) - output + output.map(ShuffleOutputCoordinator.commitOutputs(0, 0, files, _, mapStatusFile, + serializer = SparkEnv.get.serializer.newInstance()) + ) } val interleaver = new InterleaveIterators( data1, writeAndClose(writer1), data2, writeAndClose(writer2)) @@ -367,15 +370,26 @@ abstract class ShuffleSuite extends SparkFunSuite with Matchers with LocalSparkC // either task, but must be consistent) assert(mapOutput1.isDefined) assert(mapOutput2.isDefined) - assert(mapOutput1.get.location === mapOutput2.get.location) + // exactly one succeeded + assert(mapOutput1.get._1 ^ mapOutput2.get._1) + // The mapstatuses should be equivalent, but not the same object, since they will be + // deserialized independently. Unfortunately we can't check that they are the same since + // MapStatus doesn't override equals() + // register one of the map outputs -- doesn't matter which one - mapOutput1.foreach { mapStatus => mapTrackerMaster.registerMapOutputs(0, Array(mapStatus))} + mapOutput1.foreach { case (_, mapStatus) => + mapTrackerMaster.registerMapOutputs(0, Array(mapStatus)) + } val reader = manager.getReader[Int, Int](shuffleHandle, 0, 1, new TaskContextImpl(1, 0, 2L, 0, taskMemoryManager, metricsSystem, InternalAccumulator.create(sc))) val readData = reader.read().toIndexedSeq assert(readData === data1.toIndexedSeq || readData === data2.toIndexedSeq) + + assert(mapStatusFile.exists()) + manager.unregisterShuffle(0) + assert(!mapStatusFile.exists(), s"$mapStatusFile did not get deleted") } } diff --git a/core/src/test/scala/org/apache/spark/SortShuffleSuite.scala b/core/src/test/scala/org/apache/spark/SortShuffleSuite.scala index b8ab227517cc4..b54267dc95245 100644 --- a/core/src/test/scala/org/apache/spark/SortShuffleSuite.scala +++ b/core/src/test/scala/org/apache/spark/SortShuffleSuite.scala @@ -83,8 +83,8 @@ class SortShuffleSuite extends ShuffleSuite with BeforeAndAfterAll { shuffledRdd.count() // Ensure that the shuffle actually created files that will need to be cleaned up val filesCreatedByShuffle = getAllFiles -- filesBeforeShuffle - filesCreatedByShuffle.map(_.getName) should be - Set("shuffle_0_0_0.data", "shuffle_0_0_0.index") + filesCreatedByShuffle.map(_.getName) should be (Set( + "shuffle_0_0_0.data", "shuffle_0_0_0.index", "shuffle_0_0.mapstatus")) // Check that the cleanup actually removes the files sc.env.blockManager.master.removeShuffle(0, blocking = true) for (file <- filesCreatedByShuffle) { diff --git a/core/src/test/scala/org/apache/spark/shuffle/ShuffleOutputCoordinatorSuite.scala b/core/src/test/scala/org/apache/spark/shuffle/ShuffleOutputCoordinatorSuite.scala index f473322fc8c56..d8ee0fc67cd22 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/ShuffleOutputCoordinatorSuite.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/ShuffleOutputCoordinatorSuite.scala @@ -20,18 +20,30 @@ import java.io.{FileInputStream, FileOutputStream, File} import org.scalatest.BeforeAndAfterEach -import org.apache.spark.SparkFunSuite +import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.scheduler.MapStatus +import org.apache.spark.serializer.{JavaSerializer, SerializerInstance} +import org.apache.spark.storage.BlockManagerId import org.apache.spark.util.Utils class ShuffleOutputCoordinatorSuite extends SparkFunSuite with BeforeAndAfterEach { var tempDir: File = _ + var mapStatusFile: File = _ + // use the "port" as a way to distinguish mapstatuses, just for the test + def mapStatus(id: Int) = MapStatus(BlockManagerId("1", "a.b.c", id), Array(0L, 1L)) + def ser: SerializerInstance = new JavaSerializer(new SparkConf()).newInstance() override def beforeEach(): Unit = { tempDir = Utils.createTempDir() + mapStatusFile = File.createTempFile("shuffle", ".mapstatus", tempDir) } - def writeFile(filename: String, data: Int): File = { + override def afterEach(): Unit = { + Utils.deleteRecursively(tempDir) + } + + private def writeFile(filename: String, data: Int): File = { val f = new File(tempDir, filename) val out = new FileOutputStream(f) out.write(data) @@ -39,7 +51,7 @@ class ShuffleOutputCoordinatorSuite extends SparkFunSuite with BeforeAndAfterEac f } - def verifyFiles(successfulAttempt: Int): Unit = { + private def verifyFiles(successfulAttempt: Int): Unit = { (0 until 3).foreach { idx => val exp = successfulAttempt* 3 + idx val file = new File(tempDir, s"d$idx") @@ -53,74 +65,82 @@ class ShuffleOutputCoordinatorSuite extends SparkFunSuite with BeforeAndAfterEac } } - override def afterEach(): Unit = { - Utils.deleteRecursively(tempDir) - } - - - def generateAttempt(attempt: Int): Seq[(File, File)] = { + private def generateAttempt(attempt: Int): Seq[(File, File)] = { (0 until 3).map { idx => val j = attempt * 3 + idx writeFile(s"t$j", j) -> new File(tempDir, s"d$idx") } } + private def commit(files: Seq[(File, File)], id: Int): (Boolean, MapStatus) = { + ShuffleOutputCoordinator.commitOutputs(0, 0, files, mapStatus(id), mapStatusFile, ser) + } + test("move files if dest missing") { val firstAttempt = generateAttempt(0) - assert(ShuffleOutputCoordinator.commitOutputs(0, 0, firstAttempt)) + val firstCommit = commit(firstAttempt, 1) + assert(firstCommit._1) + assert(firstCommit._2.location.port === 1) verifyFiles(0) firstAttempt.foreach{ case (t, d) => assert(!t.exists())} val secondAttempt = generateAttempt(1) // second commit fails, and also deletes the tmp files - assert(!ShuffleOutputCoordinator.commitOutputs(0, 0, secondAttempt)) + val secondCommit = commit(secondAttempt, 2) + assert(!secondCommit._1) + // still the mapstatus from the first commit + assert(firstCommit._2.location.port === 1) verifyFiles(0) // make sure we delete the temp files if the dest exists secondAttempt.foreach{ case (t, d) => assert(!t.exists())} } - test("move files if dest partially missing") { + test("move files if just map status file missing") { val firstAttempt = generateAttempt(0) - assert(ShuffleOutputCoordinator.commitOutputs(0, 0, firstAttempt)) + val firstCommit = commit(firstAttempt, 1) + assert(firstCommit._1) + assert(firstCommit._2.location.port === 1) verifyFiles(0) firstAttempt.foreach{ case (t, d) => assert(!t.exists())} val secondAttempt = generateAttempt(1) - firstAttempt(0)._2.delete() + firstAttempt(0)._2.delete() // TODO should be mapStatusFile.delete() // second commit now succeeds since one destination file is missing - assert(ShuffleOutputCoordinator.commitOutputs(0, 0, secondAttempt)) + val secondCommit = commit(secondAttempt, 2) + assert(secondCommit._1) + assert(secondCommit._2.location.port === 2) verifyFiles(1) secondAttempt.foreach{ case (t, d) => assert(!t.exists())} } - test("ignore missing tmp files") { - // HashShuffle doesn't necessarily even create 0 length files for all of its output, - // so just ignore tmp files that are missing - val firstAttempt = generateAttempt(0) ++ - Seq(new File(tempDir, "bogus") -> new File(tempDir, "blah")) - assert(ShuffleOutputCoordinator.commitOutputs(0, 0, firstAttempt)) - verifyFiles(0) - assert(!new File(tempDir, "blah").exists()) - firstAttempt.foreach{ case (t, d) => assert(!t.exists())} - - // if we try again, once more with the missing tmp file, commit fails even though dest - // is "partially missing" - // TODO figure out right semantics, esp wrt non-determinstic data - val secondAttempt = generateAttempt(1) ++ - Seq(new File(tempDir, "bogus") -> new File(tempDir, "blah")) - assert(!ShuffleOutputCoordinator.commitOutputs(0, 0, secondAttempt)) - verifyFiles(0) - assert(!new File(tempDir, "blah").exists()) - secondAttempt.foreach{ case (t, d) => assert(!t.exists())} - - // but now if we delete one of the real dest files, and try again, it goes through - val thirdAttempt = generateAttempt(2) ++ - Seq(new File(tempDir, "bogus") -> new File(tempDir, "blah")) - firstAttempt(0)._2.delete() - assert(ShuffleOutputCoordinator.commitOutputs(0, 0, thirdAttempt)) - verifyFiles(2) - assert(!new File(tempDir, "blah").exists()) - thirdAttempt.foreach{ case (t, d) => assert(!t.exists())} - } +// test("ignore missing tmp files") { +// // HashShuffle doesn't necessarily even create 0 length files for all of its output, +// // so just ignore tmp files that are missing +// val firstAttempt = generateAttempt(0) ++ +// Seq(new File(tempDir, "bogus") -> new File(tempDir, "blah")) +// assert(ShuffleOutputCoordinator.commitOutputs(0, 0, firstAttempt)) +// verifyFiles(0) +// assert(!new File(tempDir, "blah").exists()) +// firstAttempt.foreach{ case (t, d) => assert(!t.exists())} +// +// // if we try again, once more with the missing tmp file, commit fails even though dest +// // is "partially missing" +// // TODO figure out right semantics, esp wrt non-determinstic data +// val secondAttempt = generateAttempt(1) ++ +// Seq(new File(tempDir, "bogus") -> new File(tempDir, "blah")) +// assert(!ShuffleOutputCoordinator.commitOutputs(0, 0, secondAttempt)) +// verifyFiles(0) +// assert(!new File(tempDir, "blah").exists()) +// secondAttempt.foreach{ case (t, d) => assert(!t.exists())} +// +// // but now if we delete one of the real dest files, and try again, it goes through +// val thirdAttempt = generateAttempt(2) ++ +// Seq(new File(tempDir, "bogus") -> new File(tempDir, "blah")) +// firstAttempt(0)._2.delete() +// assert(ShuffleOutputCoordinator.commitOutputs(0, 0, thirdAttempt)) +// verifyFiles(2) +// assert(!new File(tempDir, "blah").exists()) +// thirdAttempt.foreach{ case (t, d) => assert(!t.exists())} +// } } 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 2349b2f074230..52090dfc25e11 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 @@ -50,6 +50,7 @@ class BypassMergeSortShuffleWriterSuite extends SparkFunSuite with BeforeAndAfte private var tempDir: File = _ private var outputFile: File = _ private var indexFile: File = _ + private var mapStatusFile: File = _ private val conf: SparkConf = new SparkConf(loadDefaults = false) private val temporaryFilesCreated: mutable.Buffer[File] = new ArrayBuffer[File]() private val blockIdToFileMap: mutable.Map[BlockId, File] = new mutable.HashMap[BlockId, File] @@ -59,9 +60,11 @@ class BypassMergeSortShuffleWriterSuite extends SparkFunSuite with BeforeAndAfte tempDir = Utils.createTempDir() outputFile = File.createTempFile("shuffle", null, tempDir) indexFile = File.createTempFile("shuffle", ".index", tempDir) + mapStatusFile = File.createTempFile("shuffle", ".mapstatus", tempDir) // ShuffleOutputCoordinator requires these files to not exist yet outputFile.delete() indexFile.delete() + mapStatusFile.delete() taskMetrics = new TaskMetrics MockitoAnnotations.initMocks(this) shuffleHandle = new BypassMergeSortShuffleHandle[Int, Int]( @@ -101,6 +104,8 @@ class BypassMergeSortShuffleWriterSuite extends SparkFunSuite with BeforeAndAfte ) } }) + when(blockManager.shuffleServerId).thenReturn(BlockManagerId.apply("1", "a.b.c", 1)) + when(diskBlockManager.createTempShuffleBlock()).thenAnswer( new Answer[(TempShuffleBlockId, File)] { override def answer(invocation: InvocationOnMock): (TempShuffleBlockId, File) = { @@ -141,8 +146,9 @@ class BypassMergeSortShuffleWriterSuite extends SparkFunSuite with BeforeAndAfte conf ) val files = writer.write(Iterator.empty) - writer.stop( /* success = */ true) - assert(ShuffleOutputCoordinator.commitOutputs(0, 0, files)) + val mapStatus = writer.stop( /* success = */ true).get + val ser = new JavaSerializer(conf).newInstance() + assert(ShuffleOutputCoordinator.commitOutputs(0, 0, files, mapStatus, mapStatusFile, ser)._1) assert(writer.getPartitionLengths.sum === 0) if (outputFile.exists()) { assert(outputFile.length() === 0) @@ -169,8 +175,9 @@ class BypassMergeSortShuffleWriterSuite extends SparkFunSuite with BeforeAndAfte conf ) val files = writer.write(records) - writer.stop( /* success = */ true) - ShuffleOutputCoordinator.commitOutputs(0, 0, files) + val mapStatus = writer.stop( /* success = */ true).get + val ser = new JavaSerializer(conf).newInstance() + ShuffleOutputCoordinator.commitOutputs(0, 0, files, mapStatus, mapStatusFile, ser) assert(temporaryFilesCreated.nonEmpty) assert(writer.getPartitionLengths.sum === outputFile.length()) assert(temporaryFilesCreated.count(_.exists()) === 0) // check that temporary files were deleted From dc4b7f69924ceb7247c31747f90c13410ebfb40a Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Mon, 26 Oct 2015 15:54:49 -0500 Subject: [PATCH 14/33] fix imports --- .../spark/shuffle/sort/BypassMergeSortShuffleWriter.java | 4 ++-- .../apache/spark/shuffle/sort/UnsafeShuffleWriter.java | 2 +- .../apache/spark/shuffle/ShuffleOutputCoordinator.scala | 2 +- .../spark/shuffle/sort/UnsafeShuffleWriterSuite.java | 6 +++--- core/src/test/scala/org/apache/spark/ShuffleSuite.scala | 8 ++++---- .../test/scala/org/apache/spark/SortShuffleSuite.scala | 2 +- 6 files changed, 12 insertions(+), 12 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 78bbccdb3d0bb..b3c4d82448c88 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 @@ -29,6 +29,8 @@ import scala.Product2; import scala.Tuple2; import scala.collection.Iterator; +import scala.collection.JavaConverters; +import scala.collection.Seq; import com.google.common.annotations.VisibleForTesting; import com.google.common.io.Closeables; @@ -49,8 +51,6 @@ import org.apache.spark.shuffle.ShuffleWriter; import org.apache.spark.storage.*; import org.apache.spark.util.Utils; -import scala.collection.JavaConverters; -import scala.collection.Seq; /** * This class implements sort-based shuffle's hash-style shuffle fallback path. This write path 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 198cc69a38058..78240fab055d2 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 @@ -45,6 +45,7 @@ import org.apache.spark.io.CompressionCodec; import org.apache.spark.io.CompressionCodec$; import org.apache.spark.io.LZFCompressionCodec; +import org.apache.spark.memory.TaskMemoryManager; import org.apache.spark.network.util.LimitedInputStream; import org.apache.spark.scheduler.MapStatus; import org.apache.spark.scheduler.MapStatus$; @@ -59,7 +60,6 @@ import org.apache.spark.storage.ShuffleMapStatusBlockId; import org.apache.spark.storage.TimeTrackingOutputStream; import org.apache.spark.unsafe.Platform; -import org.apache.spark.memory.TaskMemoryManager; @Private public class UnsafeShuffleWriter extends ShuffleWriter { diff --git a/core/src/main/scala/org/apache/spark/shuffle/ShuffleOutputCoordinator.scala b/core/src/main/scala/org/apache/spark/shuffle/ShuffleOutputCoordinator.scala index a57107eb58179..5e007dc1f7747 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/ShuffleOutputCoordinator.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/ShuffleOutputCoordinator.scala @@ -20,7 +20,7 @@ import java.io.{FileOutputStream, FileInputStream, File} import org.apache.spark.Logging import org.apache.spark.scheduler.MapStatus -import org.apache.spark.serializer.{SerializerInstance, Serializer} +import org.apache.spark.serializer.SerializerInstance /** * Ensures that on each executor, there are no conflicting writes to the same shuffle files. It 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 29723a173e204..45a328b5540e2 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 @@ -48,16 +48,16 @@ import org.apache.spark.io.LZ4CompressionCodec; import org.apache.spark.io.LZFCompressionCodec; import org.apache.spark.io.SnappyCompressionCodec; -import org.apache.spark.shuffle.ShuffleOutputCoordinator; import org.apache.spark.executor.ShuffleWriteMetrics; import org.apache.spark.executor.TaskMetrics; +import org.apache.spark.memory.GrantEverythingMemoryManager; +import org.apache.spark.memory.TaskMemoryManager; import org.apache.spark.network.util.LimitedInputStream; import org.apache.spark.serializer.*; import org.apache.spark.scheduler.MapStatus; import org.apache.spark.shuffle.IndexShuffleBlockResolver; +import org.apache.spark.shuffle.ShuffleOutputCoordinator; import org.apache.spark.storage.*; -import org.apache.spark.memory.GrantEverythingMemoryManager; -import org.apache.spark.memory.TaskMemoryManager; import org.apache.spark.util.Utils; public class UnsafeShuffleWriterSuite { diff --git a/core/src/test/scala/org/apache/spark/ShuffleSuite.scala b/core/src/test/scala/org/apache/spark/ShuffleSuite.scala index ca419f1fa0318..a3b763728fee4 100644 --- a/core/src/test/scala/org/apache/spark/ShuffleSuite.scala +++ b/core/src/test/scala/org/apache/spark/ShuffleSuite.scala @@ -17,17 +17,17 @@ package org.apache.spark -import java.util.concurrent.{Executors, ExecutorService, Callable, CyclicBarrier} +import java.util.concurrent.{Callable, CyclicBarrier, ExecutorService, Executors} import org.scalatest.Matchers import org.apache.spark.ShuffleSuite.NonJavaSerializableClass +import org.apache.spark.memory.TaskMemoryManager import org.apache.spark.rdd.{CoGroupedRDD, OrderedRDDFunctions, RDD, ShuffledRDD, SubtractedRDD} import org.apache.spark.scheduler.{MapStatus, MyRDD, SparkListener, SparkListenerTaskEnd} import org.apache.spark.serializer.KryoSerializer -import org.apache.spark.shuffle.{ShuffleWriter, ShuffleOutputCoordinator} -import org.apache.spark.storage.{ShuffleMapStatusBlockId, ShuffleDataBlockId, ShuffleBlockId} -import org.apache.spark.memory.TaskMemoryManager +import org.apache.spark.shuffle.{ShuffleOutputCoordinator, ShuffleWriter} +import org.apache.spark.storage.{ShuffleBlockId, ShuffleDataBlockId, ShuffleMapStatusBlockId} import org.apache.spark.util.MutablePair abstract class ShuffleSuite extends SparkFunSuite with Matchers with LocalSparkContext { diff --git a/core/src/test/scala/org/apache/spark/SortShuffleSuite.scala b/core/src/test/scala/org/apache/spark/SortShuffleSuite.scala index b54267dc95245..67f9c006929a0 100644 --- a/core/src/test/scala/org/apache/spark/SortShuffleSuite.scala +++ b/core/src/test/scala/org/apache/spark/SortShuffleSuite.scala @@ -26,8 +26,8 @@ import org.apache.commons.io.filefilter.TrueFileFilter import org.scalatest.BeforeAndAfterAll import org.apache.spark.rdd.ShuffledRDD -import org.apache.spark.shuffle.sort.SortShuffleManager import org.apache.spark.serializer.{JavaSerializer, KryoSerializer} +import org.apache.spark.shuffle.sort.SortShuffleManager import org.apache.spark.util.Utils class SortShuffleSuite extends ShuffleSuite with BeforeAndAfterAll { From b7a0981416cc43d6d1dfb69acfb96a575796950a Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Mon, 26 Oct 2015 16:56:24 -0500 Subject: [PATCH 15/33] fixes --- core/src/main/scala/org/apache/spark/storage/BlockId.scala | 3 +++ .../apache/spark/shuffle/ShuffleOutputCoordinatorSuite.scala | 3 ++- 2 files changed, 5 insertions(+), 1 deletion(-) 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 23d05a405d21a..1eaf43f23cefa 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockId.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockId.scala @@ -116,6 +116,7 @@ object BlockId { val SHUFFLE = "shuffle_([0-9]+)_([0-9]+)_([0-9]+)".r val SHUFFLE_DATA = "shuffle_([0-9]+)_([0-9]+)_([0-9]+).data".r val SHUFFLE_INDEX = "shuffle_([0-9]+)_([0-9]+)_([0-9]+).index".r + val SHUFFLE_MAPSTATUS = "shuffle_([0-9]+)_([0-9]+).mapstatus".r val BROADCAST = "broadcast_([0-9]+)([_A-Za-z0-9]*)".r val TASKRESULT = "taskresult_([0-9]+)".r val STREAM = "input-([0-9]+)-([0-9]+)".r @@ -131,6 +132,8 @@ object BlockId { ShuffleDataBlockId(shuffleId.toInt, mapId.toInt, reduceId.toInt) case SHUFFLE_INDEX(shuffleId, mapId, reduceId) => ShuffleIndexBlockId(shuffleId.toInt, mapId.toInt, reduceId.toInt) + case SHUFFLE_MAPSTATUS(shuffleId, mapId) => + ShuffleMapStatusBlockId(shuffleId.toInt, mapId.toInt) case BROADCAST(broadcastId, field) => BroadcastBlockId(broadcastId.toLong, field.stripPrefix("_")) case TASKRESULT(taskId) => diff --git a/core/src/test/scala/org/apache/spark/shuffle/ShuffleOutputCoordinatorSuite.scala b/core/src/test/scala/org/apache/spark/shuffle/ShuffleOutputCoordinatorSuite.scala index d8ee0fc67cd22..34580d8d51019 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/ShuffleOutputCoordinatorSuite.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/ShuffleOutputCoordinatorSuite.scala @@ -37,6 +37,7 @@ class ShuffleOutputCoordinatorSuite extends SparkFunSuite with BeforeAndAfterEac override def beforeEach(): Unit = { tempDir = Utils.createTempDir() mapStatusFile = File.createTempFile("shuffle", ".mapstatus", tempDir) + mapStatusFile.delete() } override def afterEach(): Unit = { @@ -104,7 +105,7 @@ class ShuffleOutputCoordinatorSuite extends SparkFunSuite with BeforeAndAfterEac firstAttempt.foreach{ case (t, d) => assert(!t.exists())} val secondAttempt = generateAttempt(1) - firstAttempt(0)._2.delete() // TODO should be mapStatusFile.delete() + mapStatusFile.delete() // second commit now succeeds since one destination file is missing val secondCommit = commit(secondAttempt, 2) assert(secondCommit._1) From 830a097bc55323d7caffe09939193dfa872dd647 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Mon, 26 Oct 2015 20:59:41 -0500 Subject: [PATCH 16/33] shuffle writers must write always write all tmp files --- .../sort/BypassMergeSortShuffleWriter.java | 7 +++- .../shuffle/ShuffleOutputCoordinator.scala | 30 +++++++++++----- .../spark/storage/DiskBlockObjectWriter.scala | 5 +++ .../ShuffleOutputCoordinatorSuite.scala | 35 ++++--------------- .../BypassMergeSortShuffleWriterSuite.scala | 9 ++--- 5 files changed, 41 insertions(+), 45 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 b3c4d82448c88..fb7042b14b1f1 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,8 +135,13 @@ public Seq> write(Iterator> records) throws IO partitionLengths = new long[numPartitions]; File tmpIndexFile = shuffleBlockResolver.writeIndexFile(shuffleId, mapId, partitionLengths); mapStatus = MapStatus$.MODULE$.apply(blockManager.shuffleServerId(), partitionLengths); + // create empty data file so we always commit same set of shuffle output files, even if + // data is non-deterministic + File tmpDataFile = blockManager.diskBlockManager().createTempShuffleBlock()._2(); + tmpDataFile.createNewFile(); return JavaConverters.asScalaBufferConverter(Arrays.asList( - new Tuple2<>(tmpIndexFile, indexFile) + new Tuple2<>(tmpIndexFile, indexFile), + new Tuple2<>(tmpDataFile, dataFile) )).asScala(); } final SerializerInstance serInstance = serializer.newInstance(); diff --git a/core/src/main/scala/org/apache/spark/shuffle/ShuffleOutputCoordinator.scala b/core/src/main/scala/org/apache/spark/shuffle/ShuffleOutputCoordinator.scala index 5e007dc1f7747..ee828c65a129c 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/ShuffleOutputCoordinator.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/ShuffleOutputCoordinator.scala @@ -27,14 +27,24 @@ import org.apache.spark.serializer.SerializerInstance * implements "first write wins", by atomically moving all shuffle files into their final location, * only if the files did not already exist. See SPARK-8029 */ -object ShuffleOutputCoordinator extends Logging { +private[spark] object ShuffleOutputCoordinator extends Logging { /** - * if any of the destination files do not exist, then move all of the temporary files to their - * destinations. If all destination files exist, then delete all temporary files. + * If any of the destination files do not exist, then move all of the temporary files to their + * destinations, and return (true, the given MapStatus). If all destination files exist, then + * delete all temporary files, and return (false, the MapStatus from previously committed shuffle + * output). + + * @param shuffleId + * @param partitionId + * @param tmpToDest Seq of (temporary, destination) file pairs + * @param mapStatus the [[MapStatus]] for the output already written to the the temporary files + * @param mapStatusFile canonical location to store the mapStatus, for future attempts. See + * [[org.apache.spark.storage.ShuffleMapStatusBlockId]] + * @param serializer to serialize the mapStatus to disk + * @return pair of (true iff the set of temporary files was moved to the destination, the + * MapStatus of the winn * - * @param tmpToDest pairs of (temporary, destination) file pairs - * @return */ def commitOutputs( shuffleId: Int, @@ -43,9 +53,11 @@ object ShuffleOutputCoordinator extends Logging { mapStatus: MapStatus, mapStatusFile: File, serializer: SerializerInstance): (Boolean, MapStatus) = synchronized { - val toMove = tmpToDest.filter { _._1.exists()} - if (!mapStatusFile.exists()) { - toMove.foreach { case (tmp, dest) => + tmpToDest.foreach { case (tmp, _) => require(tmp.exists(), s"Cannot commit non-existent " + + s"shuffle output $tmp -- must be at least a zero-length file.")} + val destAlreadyExists = tmpToDest.forall{_._2.exists()} && mapStatusFile.exists() + if (!destAlreadyExists) { + tmpToDest.foreach { case (tmp, dest) => // If *some* of the destination files exist, but not all of them, then its not clear // what to do. There could be a task already reading from this dest file when we delete // it -- but then again, something in that taskset would be doomed to fail in any case when @@ -62,7 +74,7 @@ object ShuffleOutputCoordinator extends Logging { } else { logInfo(s"shuffle output for shuffle $shuffleId, partition $partitionId already exists, " + s"not overwriting. Another task must have created this shuffle output.") - toMove.foreach{ case (tmp, _) => tmp.delete()} + tmpToDest.foreach{ case (tmp, _) => tmp.delete()} val in = serializer.deserializeStream(new FileInputStream(mapStatusFile)) val readStatus = in.readObject[MapStatus] in.close() 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 4469bb22f3e10..f85d7d1e4adbc 100644 --- a/core/src/main/scala/org/apache/spark/storage/DiskBlockObjectWriter.scala +++ b/core/src/main/scala/org/apache/spark/storage/DiskBlockObjectWriter.scala @@ -133,6 +133,11 @@ private[spark] class DiskBlockObjectWriter( // In certain compression codecs, more bytes are written after close() is called writeMetrics.incShuffleBytesWritten(finalPosition - reportedPosition) } else { + if (!file.exists()) { + // SPARK-8029 -- we need to write a zero-length file so we can commit the same set of files + // on all attempts (even if the data is non-deterministic) + file.createNewFile() + } finalPosition = file.length() } commitAndCloseHasBeenCalled = true diff --git a/core/src/test/scala/org/apache/spark/shuffle/ShuffleOutputCoordinatorSuite.scala b/core/src/test/scala/org/apache/spark/shuffle/ShuffleOutputCoordinatorSuite.scala index 34580d8d51019..f7e62d2d56457 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/ShuffleOutputCoordinatorSuite.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/ShuffleOutputCoordinatorSuite.scala @@ -114,34 +114,11 @@ class ShuffleOutputCoordinatorSuite extends SparkFunSuite with BeforeAndAfterEac secondAttempt.foreach{ case (t, d) => assert(!t.exists())} } -// test("ignore missing tmp files") { -// // HashShuffle doesn't necessarily even create 0 length files for all of its output, -// // so just ignore tmp files that are missing -// val firstAttempt = generateAttempt(0) ++ -// Seq(new File(tempDir, "bogus") -> new File(tempDir, "blah")) -// assert(ShuffleOutputCoordinator.commitOutputs(0, 0, firstAttempt)) -// verifyFiles(0) -// assert(!new File(tempDir, "blah").exists()) -// firstAttempt.foreach{ case (t, d) => assert(!t.exists())} -// -// // if we try again, once more with the missing tmp file, commit fails even though dest -// // is "partially missing" -// // TODO figure out right semantics, esp wrt non-determinstic data -// val secondAttempt = generateAttempt(1) ++ -// Seq(new File(tempDir, "bogus") -> new File(tempDir, "blah")) -// assert(!ShuffleOutputCoordinator.commitOutputs(0, 0, secondAttempt)) -// verifyFiles(0) -// assert(!new File(tempDir, "blah").exists()) -// secondAttempt.foreach{ case (t, d) => assert(!t.exists())} -// -// // but now if we delete one of the real dest files, and try again, it goes through -// val thirdAttempt = generateAttempt(2) ++ -// Seq(new File(tempDir, "bogus") -> new File(tempDir, "blah")) -// firstAttempt(0)._2.delete() -// assert(ShuffleOutputCoordinator.commitOutputs(0, 0, thirdAttempt)) -// verifyFiles(2) -// assert(!new File(tempDir, "blah").exists()) -// thirdAttempt.foreach{ case (t, d) => assert(!t.exists())} -// } + test("no missing tmp files") { + val firstAttempt = generateAttempt(0) ++ + Seq(new File(tempDir, "bogus") -> new File(tempDir, "blah")) + val ex = intercept[IllegalArgumentException] {commit(firstAttempt, 1)} + assert(ex.getMessage.contains("Cannot commit non-existent shuffle output")) + } } 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 52090dfc25e11..c0a97f0e1d721 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 @@ -150,12 +150,9 @@ class BypassMergeSortShuffleWriterSuite extends SparkFunSuite with BeforeAndAfte val ser = new JavaSerializer(conf).newInstance() assert(ShuffleOutputCoordinator.commitOutputs(0, 0, files, mapStatus, mapStatusFile, ser)._1) assert(writer.getPartitionLengths.sum === 0) - if (outputFile.exists()) { - assert(outputFile.length() === 0) - assert(temporaryFilesCreated.size === 2) - } else { - assert(temporaryFilesCreated.size === 1) - } + assert(outputFile.exists()) + assert(outputFile.length() === 0) + assert(temporaryFilesCreated.size === 2) val shuffleWriteMetrics = taskContext.taskMetrics().shuffleWriteMetrics.get assert(shuffleWriteMetrics.shuffleBytesWritten === 0) assert(shuffleWriteMetrics.shuffleRecordsWritten === 0) From 5d11eca843ab19fc4d1b83d50a677bd6f2b6f0d8 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Mon, 26 Oct 2015 21:33:13 -0500 Subject: [PATCH 17/33] more fixes for zero-sized blocks --- .../util/collection/ExternalSorter.scala | 7 ++++- .../scala/org/apache/spark/ShuffleSuite.scala | 31 +++++-------------- 2 files changed, 13 insertions(+), 25 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 a44e72b7c16d3..3046b191be309 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 @@ -639,7 +639,6 @@ private[spark] class ExternalSorter[K, V, C]( * called by the SortShuffleWriter. * * @param blockId block ID to write to. The index file will be blockId.name + ".index". - * @param context a TaskContext for a running Spark task, for us to update shuffle metrics. * @return array of lengths, in bytes, of each partition of the file (used by map output tracker) */ def writePartitionedFile( @@ -680,6 +679,12 @@ private[spark] class ExternalSorter[K, V, C]( } } + // SPARK-8029 the ShuffleOutputCoordinator requires all shuffle output files to always exist, + // even if they are zero-length + if (!outputFile.exists()) { + outputFile.createNewFile() + } + context.taskMetrics().incMemoryBytesSpilled(memoryBytesSpilled) context.taskMetrics().incDiskBytesSpilled(diskBytesSpilled) context.internalMetricsToAccumulators( diff --git a/core/src/test/scala/org/apache/spark/ShuffleSuite.scala b/core/src/test/scala/org/apache/spark/ShuffleSuite.scala index a3b763728fee4..d8a6aca4eeb08 100644 --- a/core/src/test/scala/org/apache/spark/ShuffleSuite.scala +++ b/core/src/test/scala/org/apache/spark/ShuffleSuite.scala @@ -25,7 +25,7 @@ import org.apache.spark.ShuffleSuite.NonJavaSerializableClass import org.apache.spark.memory.TaskMemoryManager import org.apache.spark.rdd.{CoGroupedRDD, OrderedRDDFunctions, RDD, ShuffledRDD, SubtractedRDD} import org.apache.spark.scheduler.{MapStatus, MyRDD, SparkListener, SparkListenerTaskEnd} -import org.apache.spark.serializer.KryoSerializer +import org.apache.spark.serializer.{KryoSerializer, Serializer} import org.apache.spark.shuffle.{ShuffleOutputCoordinator, ShuffleWriter} import org.apache.spark.storage.{ShuffleBlockId, ShuffleDataBlockId, ShuffleMapStatusBlockId} import org.apache.spark.util.MutablePair @@ -92,33 +92,16 @@ abstract class ShuffleSuite extends SparkFunSuite with Matchers with LocalSparkC } test("zero sized blocks") { - // Use a local cluster with 2 processes to make sure there are both local and remote blocks - sc = new SparkContext("local-cluster[2,1,1024]", "test", conf) - - // 201 partitions (greater than "spark.shuffle.sort.bypassMergeThreshold") from 4 keys - val NUM_BLOCKS = 201 - val a = sc.parallelize(1 to 4, NUM_BLOCKS) - val b = a.map(x => (x, x*2)) - // NOTE: The default Java serializer doesn't create zero-sized blocks. // So, use Kryo - val c = new ShuffledRDD[Int, Int, Int](b, new HashPartitioner(NUM_BLOCKS)) - .setSerializer(new KryoSerializer(conf)) - - val shuffleId = c.dependencies.head.asInstanceOf[ShuffleDependency[_, _, _]].shuffleId - assert(c.count === 4) - - val blockSizes = (0 until NUM_BLOCKS).flatMap { id => - val statuses = SparkEnv.get.mapOutputTracker.getMapSizesByExecutorId(shuffleId, id) - statuses.flatMap(_._2.map(_._2)) - } - val nonEmptyBlocks = blockSizes.filter(x => x > 0) - - // We should have at most 4 non-zero sized partitions - assert(nonEmptyBlocks.size <= 4) + testZeroSizedBlocks(Some(new KryoSerializer(conf))) } test("zero sized blocks without kryo") { + testZeroSizedBlocks(None) + } + + def testZeroSizedBlocks(serOpt: Option[Serializer]): Unit = { // Use a local cluster with 2 processes to make sure there are both local and remote blocks sc = new SparkContext("local-cluster[2,1,1024]", "test", conf) @@ -127,8 +110,8 @@ abstract class ShuffleSuite extends SparkFunSuite with Matchers with LocalSparkC val a = sc.parallelize(1 to 4, NUM_BLOCKS) val b = a.map(x => (x, x*2)) - // NOTE: The default Java serializer should create zero-sized blocks val c = new ShuffledRDD[Int, Int, Int](b, new HashPartitioner(NUM_BLOCKS)) + serOpt.foreach(c.setSerializer(_)) val shuffleId = c.dependencies.head.asInstanceOf[ShuffleDependency[_, _, _]].shuffleId assert(c.count === 4) From 3f5af9c25fddbdb8e31c77eddc7861d27286c845 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Mon, 26 Oct 2015 21:59:41 -0500 Subject: [PATCH 18/33] dont make ShuffleWriter return mapStatusFile --- .../sort/BypassMergeSortShuffleWriter.java | 5 ----- .../shuffle/sort/UnsafeShuffleWriter.java | 5 ----- .../spark/scheduler/ShuffleMapTask.scala | 2 +- .../shuffle/ShuffleOutputCoordinator.scala | 21 +++++++++++++++---- .../apache/spark/shuffle/ShuffleWriter.scala | 7 ------- .../shuffle/hash/HashShuffleWriter.scala | 4 ---- .../shuffle/sort/SortShuffleWriter.scala | 4 ---- .../scala/org/apache/spark/ShuffleSuite.scala | 4 +--- .../ShuffleOutputCoordinatorSuite.scala | 2 +- 9 files changed, 20 insertions(+), 34 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 fb7042b14b1f1..562e4568ab6e5 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 @@ -258,9 +258,4 @@ public Option stop(boolean success) { } } - @Override - public File mapStatusFile() { - return blockManager.diskBlockManager().getFile(new ShuffleMapStatusBlockId(shuffleId, mapId)); - } - } 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 78240fab055d2..ab105487cc888 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 @@ -505,9 +505,4 @@ public Option stop(boolean success) { } } - @Override - public File mapStatusFile() { - return blockManager.diskBlockManager().getFile(new ShuffleMapStatusBlockId(shuffleId, mapId)); - } - } diff --git a/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala b/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala index 553fc1df77cd3..b1b4fc34dabf6 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala @@ -75,7 +75,7 @@ private[spark] class ShuffleMapTask( val mapStatus = writer.stop(success = true).get // SPARK-8029 make sure only one task on this executor writes the final shuffle files ShuffleOutputCoordinator.commitOutputs(dep.shuffleId, partitionId, tmpToDestFiles, mapStatus, - writer.mapStatusFile, SparkEnv.get.serializer.newInstance())._2 + SparkEnv.get)._2 } catch { case e: Exception => try { diff --git a/core/src/main/scala/org/apache/spark/shuffle/ShuffleOutputCoordinator.scala b/core/src/main/scala/org/apache/spark/shuffle/ShuffleOutputCoordinator.scala index ee828c65a129c..b3b221389ec38 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/ShuffleOutputCoordinator.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/ShuffleOutputCoordinator.scala @@ -18,7 +18,10 @@ package org.apache.spark.shuffle import java.io.{FileOutputStream, FileInputStream, File} -import org.apache.spark.Logging +import com.google.common.annotations.VisibleForTesting + +import org.apache.spark.storage.ShuffleMapStatusBlockId +import org.apache.spark.{SparkEnv, Logging} import org.apache.spark.scheduler.MapStatus import org.apache.spark.serializer.SerializerInstance @@ -39,13 +42,23 @@ private[spark] object ShuffleOutputCoordinator extends Logging { * @param partitionId * @param tmpToDest Seq of (temporary, destination) file pairs * @param mapStatus the [[MapStatus]] for the output already written to the the temporary files - * @param mapStatusFile canonical location to store the mapStatus, for future attempts. See - * [[org.apache.spark.storage.ShuffleMapStatusBlockId]] - * @param serializer to serialize the mapStatus to disk * @return pair of (true iff the set of temporary files was moved to the destination, the * MapStatus of the winn * */ + def commitOutputs( + shuffleId: Int, + partitionId: Int, + tmpToDest: Seq[(File, File)], + mapStatus: MapStatus, + sparkEnv: SparkEnv): (Boolean, MapStatus) = synchronized { + val mapStatusFile = sparkEnv.blockManager.diskBlockManager.getFile( + ShuffleMapStatusBlockId(shuffleId, partitionId)) + val ser = sparkEnv.serializer.newInstance() + commitOutputs(shuffleId, partitionId, tmpToDest, mapStatus, mapStatusFile, ser) + } + + @VisibleForTesting def commitOutputs( shuffleId: Int, partitionId: Int, diff --git a/core/src/main/scala/org/apache/spark/shuffle/ShuffleWriter.scala b/core/src/main/scala/org/apache/spark/shuffle/ShuffleWriter.scala index 7410d19516c3f..56af1590b088c 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/ShuffleWriter.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/ShuffleWriter.scala @@ -34,13 +34,6 @@ private[spark] abstract class ShuffleWriter[K, V] { @throws[IOException] def write(records: Iterator[Product2[K, V]]): Seq[(File, File)] - /** - * The canonical location of a file for storing the serialized output, to coordinate between - * multiple tasks that may write the same shuffle data - * @return - */ - def mapStatusFile: File - /** Close this writer, passing along whether the map completed */ def stop(success: Boolean): Option[MapStatus] } diff --git a/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleWriter.scala b/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleWriter.scala index 793830563c73a..42050b4116584 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleWriter.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleWriter.scala @@ -103,10 +103,6 @@ private[spark] class HashShuffleWriter[K, V]( } } - override def mapStatusFile: File = { - blockManager.diskBlockManager.getFile(ShuffleMapStatusBlockId(handle.shuffleId, mapId)) - } - private def commitWritesAndBuildStatus(): MapStatus = { // Commit the writes. Get the size of each bucket block (total block size). val sizes: Array[Long] = shuffle.writers.map { case (writer: DiskBlockObjectWriter, _) => 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 b7e7d9dae5c5c..1ff553d5dc603 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 @@ -108,10 +108,6 @@ private[spark] class SortShuffleWriter[K, V, C]( } } - override def mapStatusFile: File = { - blockManager.diskBlockManager.getFile(ShuffleMapStatusBlockId(handle.shuffleId, mapId)) - } - } private[spark] object SortShuffleWriter { diff --git a/core/src/test/scala/org/apache/spark/ShuffleSuite.scala b/core/src/test/scala/org/apache/spark/ShuffleSuite.scala index d8a6aca4eeb08..5b8c9b76a3bd6 100644 --- a/core/src/test/scala/org/apache/spark/ShuffleSuite.scala +++ b/core/src/test/scala/org/apache/spark/ShuffleSuite.scala @@ -339,9 +339,7 @@ abstract class ShuffleSuite extends SparkFunSuite with Matchers with LocalSparkC iter: Iterator[(Int, Int)]): Option[(Boolean, MapStatus)] = { val files = writer.write(iter) val output = writer.stop(true) - output.map(ShuffleOutputCoordinator.commitOutputs(0, 0, files, _, mapStatusFile, - serializer = SparkEnv.get.serializer.newInstance()) - ) + output.map(ShuffleOutputCoordinator.commitOutputs(0, 0, files, _, SparkEnv.get)) } val interleaver = new InterleaveIterators( data1, writeAndClose(writer1), data2, writeAndClose(writer2)) diff --git a/core/src/test/scala/org/apache/spark/shuffle/ShuffleOutputCoordinatorSuite.scala b/core/src/test/scala/org/apache/spark/shuffle/ShuffleOutputCoordinatorSuite.scala index f7e62d2d56457..bbf94fd9275cf 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/ShuffleOutputCoordinatorSuite.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/ShuffleOutputCoordinatorSuite.scala @@ -16,7 +16,7 @@ */ package org.apache.spark.shuffle -import java.io.{FileInputStream, FileOutputStream, File} +import java.io.{File, FileInputStream, FileOutputStream} import org.scalatest.BeforeAndAfterEach From 4b7c71a938d69be93baecb8ce320a2151b7a4658 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Mon, 26 Oct 2015 22:29:16 -0500 Subject: [PATCH 19/33] rather than requiring all tmp files to exist, just write a zero-length dest file either way --- .../shuffle/ShuffleOutputCoordinator.scala | 24 ++++++++++++------- .../spark/storage/DiskBlockObjectWriter.scala | 5 ---- .../util/collection/ExternalSorter.scala | 6 ----- .../ShuffleOutputCoordinatorSuite.scala | 24 ++++++++++++++----- 4 files changed, 34 insertions(+), 25 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/shuffle/ShuffleOutputCoordinator.scala b/core/src/main/scala/org/apache/spark/shuffle/ShuffleOutputCoordinator.scala index b3b221389ec38..a195316409866 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/ShuffleOutputCoordinator.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/ShuffleOutputCoordinator.scala @@ -37,13 +37,16 @@ private[spark] object ShuffleOutputCoordinator extends Logging { * destinations, and return (true, the given MapStatus). If all destination files exist, then * delete all temporary files, and return (false, the MapStatus from previously committed shuffle * output). - - * @param shuffleId - * @param partitionId + * + * Note that this will write to all destination files. If the tmp file is missing, then a + * zero-length destination file will be created. This is so the ShuffleOutputCoordinator can work + * even when there is a non-determinstic data, where the output exists in one attempt, but is + * empty in another attempt. + * * @param tmpToDest Seq of (temporary, destination) file pairs * @param mapStatus the [[MapStatus]] for the output already written to the the temporary files - * @return pair of (true iff the set of temporary files was moved to the destination, the - * MapStatus of the winn + * @return pair of: (1) true iff the set of temporary files was moved to the destination and (2) + * the MapStatus of the committed attempt. * */ def commitOutputs( @@ -66,8 +69,6 @@ private[spark] object ShuffleOutputCoordinator extends Logging { mapStatus: MapStatus, mapStatusFile: File, serializer: SerializerInstance): (Boolean, MapStatus) = synchronized { - tmpToDest.foreach { case (tmp, _) => require(tmp.exists(), s"Cannot commit non-existent " + - s"shuffle output $tmp -- must be at least a zero-length file.")} val destAlreadyExists = tmpToDest.forall{_._2.exists()} && mapStatusFile.exists() if (!destAlreadyExists) { tmpToDest.foreach { case (tmp, dest) => @@ -78,7 +79,14 @@ private[spark] object ShuffleOutputCoordinator extends Logging { if (dest.exists()) { dest.delete() } - tmp.renameTo(dest) + if (tmp.exists()) { + tmp.renameTo(dest) + } else { + // we always create the destination files, so this works correctly even when the + // input data is non-deterministic (potentially empty in one iteration, and non-empty + // in another) + dest.createNewFile() + } } val out = serializer.serializeStream(new FileOutputStream(mapStatusFile)) out.writeObject(mapStatus) 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 f85d7d1e4adbc..4469bb22f3e10 100644 --- a/core/src/main/scala/org/apache/spark/storage/DiskBlockObjectWriter.scala +++ b/core/src/main/scala/org/apache/spark/storage/DiskBlockObjectWriter.scala @@ -133,11 +133,6 @@ private[spark] class DiskBlockObjectWriter( // In certain compression codecs, more bytes are written after close() is called writeMetrics.incShuffleBytesWritten(finalPosition - reportedPosition) } else { - if (!file.exists()) { - // SPARK-8029 -- we need to write a zero-length file so we can commit the same set of files - // on all attempts (even if the data is non-deterministic) - file.createNewFile() - } finalPosition = file.length() } commitAndCloseHasBeenCalled = true 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 3046b191be309..ea85c63e0be14 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 @@ -679,12 +679,6 @@ private[spark] class ExternalSorter[K, V, C]( } } - // SPARK-8029 the ShuffleOutputCoordinator requires all shuffle output files to always exist, - // even if they are zero-length - if (!outputFile.exists()) { - outputFile.createNewFile() - } - context.taskMetrics().incMemoryBytesSpilled(memoryBytesSpilled) context.taskMetrics().incDiskBytesSpilled(diskBytesSpilled) context.internalMetricsToAccumulators( diff --git a/core/src/test/scala/org/apache/spark/shuffle/ShuffleOutputCoordinatorSuite.scala b/core/src/test/scala/org/apache/spark/shuffle/ShuffleOutputCoordinatorSuite.scala index bbf94fd9275cf..d806ea6a250e9 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/ShuffleOutputCoordinatorSuite.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/ShuffleOutputCoordinatorSuite.scala @@ -31,7 +31,7 @@ class ShuffleOutputCoordinatorSuite extends SparkFunSuite with BeforeAndAfterEac var tempDir: File = _ var mapStatusFile: File = _ // use the "port" as a way to distinguish mapstatuses, just for the test - def mapStatus(id: Int) = MapStatus(BlockManagerId("1", "a.b.c", id), Array(0L, 1L)) + def mapStatus(id: Int): MapStatus = MapStatus(BlockManagerId("1", "a.b.c", id), Array(0L, 1L)) def ser: SerializerInstance = new JavaSerializer(new SparkConf()).newInstance() override def beforeEach(): Unit = { @@ -114,11 +114,23 @@ class ShuffleOutputCoordinatorSuite extends SparkFunSuite with BeforeAndAfterEac secondAttempt.foreach{ case (t, d) => assert(!t.exists())} } - test("no missing tmp files") { + test("missing tmp files become zero-length destination files") { + val extraDestFile = new File(tempDir, "blah") val firstAttempt = generateAttempt(0) ++ - Seq(new File(tempDir, "bogus") -> new File(tempDir, "blah")) - val ex = intercept[IllegalArgumentException] {commit(firstAttempt, 1)} - assert(ex.getMessage.contains("Cannot commit non-existent shuffle output")) + Seq(new File(tempDir, "bogus") -> extraDestFile) + assert(commit(firstAttempt, 1)._1) + verifyFiles(0) + assert(extraDestFile.exists()) + assert(extraDestFile.length() === 0) + + // if we attempt the move again and *only* the missing tmp file is missing, we still + // do the move + extraDestFile.delete() + val secondAttempt = generateAttempt(1) ++ + Seq(new File(tempDir, "flippy") -> extraDestFile) + assert(commit(secondAttempt, 2)._1) + verifyFiles(1) + assert(extraDestFile.exists()) + assert(extraDestFile.length() === 0) } - } From eabf97838517d5354685f48a4e0abc5fec329df3 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Tue, 27 Oct 2015 08:40:40 -0500 Subject: [PATCH 20/33] update test case --- .../spark/util/collection/ExternalSorterSuite.scala | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) 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 d7b2d07a40052..d9661b842f7bc 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 @@ -481,15 +481,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 + // After the shuffle, there should be only 3 files on disk: the output of task 1, its index, + // and mapstatus file. All other files (map 2's output and intermediate merge files) should // have been deleted. - assert(diskBlockManager.getAllFiles().length === 2) + 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 their + // indices, and mapstatus files. All intermediate merge files should have been deleted. + assert(diskBlockManager.getAllFiles().length === 6) } } } From 5bbeec3f8243f9caa94096e03c88d060ad5fcf13 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Tue, 27 Oct 2015 09:08:14 -0500 Subject: [PATCH 21/33] minor cleanup --- .../sort/BypassMergeSortShuffleWriter.java | 9 ++++----- .../shuffle/sort/UnsafeShuffleWriter.java | 20 ++++++++----------- 2 files changed, 12 insertions(+), 17 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 562e4568ab6e5..b85007eebd0c2 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 @@ -133,11 +133,11 @@ public Seq> write(Iterator> records) throws IO final File dataFile = shuffleBlockResolver.getDataFile(shuffleId, mapId); if (!records.hasNext()) { partitionLengths = new long[numPartitions]; - File tmpIndexFile = shuffleBlockResolver.writeIndexFile(shuffleId, mapId, partitionLengths); + final File tmpIndexFile = shuffleBlockResolver.writeIndexFile(shuffleId, mapId, partitionLengths); mapStatus = MapStatus$.MODULE$.apply(blockManager.shuffleServerId(), partitionLengths); // create empty data file so we always commit same set of shuffle output files, even if // data is non-deterministic - File tmpDataFile = blockManager.diskBlockManager().createTempShuffleBlock()._2(); + final File tmpDataFile = blockManager.diskBlockManager().createTempShuffleBlock()._2(); tmpDataFile.createNewFile(); return JavaConverters.asScalaBufferConverter(Arrays.asList( new Tuple2<>(tmpIndexFile, indexFile), @@ -170,10 +170,10 @@ public Seq> write(Iterator> records) throws IO writer.commitAndClose(); } - File tmpDataFile = blockManager.diskBlockManager().createTempShuffleBlock()._2(); + final File tmpDataFile = blockManager.diskBlockManager().createTempShuffleBlock()._2(); partitionLengths = writePartitionedFile(tmpDataFile); - File tmpIndexFile = shuffleBlockResolver.writeIndexFile(shuffleId, mapId, partitionLengths); + final File tmpIndexFile = shuffleBlockResolver.writeIndexFile(shuffleId, mapId, partitionLengths); mapStatus = MapStatus$.MODULE$.apply(blockManager.shuffleServerId(), partitionLengths); return JavaConverters.asScalaBufferConverter(Arrays.asList( new Tuple2<>(tmpIndexFile, indexFile), @@ -257,5 +257,4 @@ public Option stop(boolean success) { } } } - } 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 ab105487cc888..48efd0156f8a1 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 @@ -175,7 +175,7 @@ public Seq> write(scala.collection.Iterator> r while (records.hasNext()) { insertRecordIntoSorter(records.next()); } - Seq> result = closeAndWriteOutput(); + final Seq> result = closeAndWriteOutput(); success = true; return result; } finally { @@ -221,7 +221,7 @@ Seq> closeAndWriteOutput() throws IOException { final long[] partitionLengths; final File tmpDataFile; try { - Tuple2 t = mergeSpills(spills); + final Tuple2 t = mergeSpills(spills); partitionLengths = t._2(); tmpDataFile = t._1(); } finally { @@ -231,10 +231,10 @@ Seq> closeAndWriteOutput() throws IOException { } } } - File tmpIndexFile = shuffleBlockResolver.writeIndexFile(shuffleId, mapId, partitionLengths); + final File tmpIndexFile = shuffleBlockResolver.writeIndexFile(shuffleId, mapId, partitionLengths); mapStatus = MapStatus$.MODULE$.apply(blockManager.shuffleServerId(), partitionLengths); - File dataFile = shuffleBlockResolver.getDataFile(shuffleId, mapId); - File indexFile = blockManager.diskBlockManager().getFile( + final File dataFile = shuffleBlockResolver.getDataFile(shuffleId, mapId); + final File indexFile = blockManager.diskBlockManager().getFile( new ShuffleIndexBlockId(shuffleId, mapId, IndexShuffleBlockResolver$.MODULE$.NOOP_REDUCE_ID()) ); @@ -275,10 +275,6 @@ void forceSorterToSpill() throws IOException { */ private Tuple2 mergeSpills(SpillInfo[] spills) throws IOException { final File outputFile = blockManager.diskBlockManager().createTempShuffleBlock()._2(); - return new Tuple2<>(outputFile, mergeSpills(spills, outputFile)); - } - - private long[] mergeSpills(SpillInfo[] spills, File outputFile) throws IOException { final boolean compressionEnabled = sparkConf.getBoolean("spark.shuffle.compress", true); final CompressionCodec compressionCodec = CompressionCodec$.MODULE$.createCodec(sparkConf); final boolean fastMergeEnabled = @@ -288,12 +284,12 @@ private long[] mergeSpills(SpillInfo[] spills, File outputFile) throws IOExcepti try { if (spills.length == 0) { new FileOutputStream(outputFile).close(); // Create an empty file - return new long[partitioner.numPartitions()]; + return new Tuple2<>(outputFile, new long[partitioner.numPartitions()]); } else if (spills.length == 1) { // Here, we don't need to perform any metrics updates because the bytes written to this // output file would have already been counted as shuffle bytes written. Files.move(spills[0].file, outputFile); - return spills[0].partitionLengths; + return new Tuple2<>(outputFile, spills[0].partitionLengths); } else { final long[] partitionLengths; // There are multiple spills to merge, so none of these spill files' lengths were counted @@ -328,7 +324,7 @@ private long[] mergeSpills(SpillInfo[] spills, File outputFile) throws IOExcepti // SpillInfo's bytes. writeMetrics.decShuffleBytesWritten(spills[spills.length - 1].file.length()); writeMetrics.incShuffleBytesWritten(outputFile.length()); - return partitionLengths; + return new Tuple2<>(outputFile, partitionLengths); } } catch (IOException e) { if (outputFile.exists() && !outputFile.delete()) { From e141d82b86e48215a5cebf5e10a7cd733d75ea78 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Tue, 27 Oct 2015 10:41:56 -0500 Subject: [PATCH 22/33] test that shuffle output files are always the same --- .../scala/org/apache/spark/ShuffleSuite.scala | 45 ++++++++++++++++++- 1 file changed, 44 insertions(+), 1 deletion(-) diff --git a/core/src/test/scala/org/apache/spark/ShuffleSuite.scala b/core/src/test/scala/org/apache/spark/ShuffleSuite.scala index 5b8c9b76a3bd6..aeedbe4c565df 100644 --- a/core/src/test/scala/org/apache/spark/ShuffleSuite.scala +++ b/core/src/test/scala/org/apache/spark/ShuffleSuite.scala @@ -17,8 +17,13 @@ package org.apache.spark +import java.io.File import java.util.concurrent.{Callable, CyclicBarrier, ExecutorService, Executors} +import scala.collection.JavaConverters._ + +import org.apache.commons.io.FileUtils +import org.apache.commons.io.filefilter.TrueFileFilter import org.scalatest.Matchers import org.apache.spark.ShuffleSuite.NonJavaSerializableClass @@ -28,7 +33,7 @@ import org.apache.spark.scheduler.{MapStatus, MyRDD, SparkListener, SparkListene import org.apache.spark.serializer.{KryoSerializer, Serializer} import org.apache.spark.shuffle.{ShuffleOutputCoordinator, ShuffleWriter} import org.apache.spark.storage.{ShuffleBlockId, ShuffleDataBlockId, ShuffleMapStatusBlockId} -import org.apache.spark.util.MutablePair +import org.apache.spark.util.{MutablePair, Utils} abstract class ShuffleSuite extends SparkFunSuite with Matchers with LocalSparkContext { @@ -305,6 +310,44 @@ abstract class ShuffleSuite extends SparkFunSuite with Matchers with LocalSparkC assert(metrics.bytesWritten > 0) } + /** + * ShuffleOutputCoordinator requires that a given shuffle always generate the same set of files on + * all attempts, even if the data is non-deterministic. We test the extreme case where the data + * is completely missing in one case + */ + test("same set of shuffle files regardless of data") { + + def shuffleAndGetShuffleFiles(data: Seq[Int]): Map[String, Long] = { + var tempDir: File = null + try { + tempDir = Utils.createTempDir() + conf.set("spark.local.dir", tempDir.getAbsolutePath) + sc = new SparkContext("local", "test", conf) + val rdd = sc.parallelize(data, 10).map(x => (x, x)) + val shuffledRdd = new ShuffledRDD[Int, Int, Int](rdd, new HashPartitioner(4)) + shuffledRdd.count() + val shuffleFiles = + FileUtils.listFiles(tempDir, TrueFileFilter.INSTANCE, TrueFileFilter.INSTANCE) + sc.stop() + shuffleFiles.asScala.map { file: File => + file.getName -> file.length() + }.toMap + } finally { + conf.remove("spark.local.dir") + Utils.deleteRecursively(tempDir) + } + } + + val shuffleFilesWithData = shuffleAndGetShuffleFiles(0 until 100) + val shuffleFilesNoData = shuffleAndGetShuffleFiles(Seq[Int]()) + assert(shuffleFilesNoData.keySet === shuffleFilesWithData.keySet) + // make sure our test is doing what it is supposed to -- at least some of the + // "no data" files are empty + assert(shuffleFilesNoData.filter{ case (name, size) => + size == 0L && shuffleFilesWithData(name) != 0L + }.nonEmpty) + } + test("multiple simultaneous attempts for one task (SPARK-8029)") { sc = new SparkContext("local", "test", conf) val mapStatusFile = sc.env.blockManager.diskBlockManager.getFile(ShuffleMapStatusBlockId(0, 0)) From 4df7955db9c46b1549b3c0f4e238f5be7970c337 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Tue, 27 Oct 2015 12:19:22 -0500 Subject: [PATCH 23/33] fix compression settings of tmp files; minor cleanup --- .../shuffle/sort/BypassMergeSortShuffleWriter.java | 4 +--- .../spark/shuffle/sort/UnsafeShuffleWriter.java | 5 +---- .../spark/shuffle/FileShuffleBlockResolver.scala | 2 +- .../spark/shuffle/IndexShuffleBlockResolver.scala | 2 +- .../spark/shuffle/sort/SortShuffleWriter.scala | 5 ++--- .../scala/org/apache/spark/storage/BlockId.scala | 10 ++++++++++ .../apache/spark/storage/DiskBlockManager.scala | 14 ++++++++++++++ 7 files changed, 30 insertions(+), 12 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 b85007eebd0c2..9db28a5d71425 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 @@ -127,9 +127,7 @@ public BypassMergeSortShuffleWriter( @Override public Seq> write(Iterator> records) throws IOException { assert (partitionWriters == null); - final File indexFile = blockManager.diskBlockManager().getFile(new ShuffleIndexBlockId( - shuffleId, mapId, IndexShuffleBlockResolver$.MODULE$.NOOP_REDUCE_ID()) - ); + final File indexFile = shuffleBlockResolver.getIndexFile(shuffleId, mapId); final File dataFile = shuffleBlockResolver.getDataFile(shuffleId, mapId); if (!records.hasNext()) { partitionLengths = new long[numPartitions]; 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 48efd0156f8a1..d1f8ea7e8bb1e 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,7 +57,6 @@ import org.apache.spark.shuffle.ShuffleWriter; import org.apache.spark.storage.BlockManager; import org.apache.spark.storage.ShuffleIndexBlockId; -import org.apache.spark.storage.ShuffleMapStatusBlockId; import org.apache.spark.storage.TimeTrackingOutputStream; import org.apache.spark.unsafe.Platform; @@ -234,9 +233,7 @@ Seq> closeAndWriteOutput() throws IOException { final File tmpIndexFile = shuffleBlockResolver.writeIndexFile(shuffleId, mapId, partitionLengths); mapStatus = MapStatus$.MODULE$.apply(blockManager.shuffleServerId(), partitionLengths); final File dataFile = shuffleBlockResolver.getDataFile(shuffleId, mapId); - final File indexFile = blockManager.diskBlockManager().getFile( - new ShuffleIndexBlockId(shuffleId, mapId, IndexShuffleBlockResolver$.MODULE$.NOOP_REDUCE_ID()) - ); + final File indexFile = shuffleBlockResolver.getIndexFile(shuffleId, mapId); return JavaConverters.asScalaBufferConverter(Arrays.asList( new Tuple2<>(tmpIndexFile, indexFile), diff --git a/core/src/main/scala/org/apache/spark/shuffle/FileShuffleBlockResolver.scala b/core/src/main/scala/org/apache/spark/shuffle/FileShuffleBlockResolver.scala index 2805e5c1655b2..86d47e5e20500 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/FileShuffleBlockResolver.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/FileShuffleBlockResolver.scala @@ -85,7 +85,7 @@ private[spark] class FileShuffleBlockResolver(conf: SparkConf) Array.tabulate[(DiskBlockObjectWriter, File)](numReducers) { bucketId => val blockId = ShuffleBlockId(shuffleId, mapId, bucketId) val blockFile = blockManager.diskBlockManager.getFile(blockId) - val (_, tmpBlockFile) = blockManager.diskBlockManager.createTempLocalBlock() + val (_, tmpBlockFile) = blockManager.diskBlockManager.createTempShuffleBlock() // Because of previous failures, the shuffle file may already exist on this machine. // If so, remove it. if (blockFile.exists) { 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 fb5b981f427c4..f0a5c405197da 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala @@ -51,7 +51,7 @@ private[spark] class IndexShuffleBlockResolver(conf: SparkConf) extends ShuffleB blockManager.diskBlockManager.getFile(ShuffleDataBlockId(shuffleId, mapId, NOOP_REDUCE_ID)) } - private def getIndexFile(shuffleId: Int, mapId: Int): File = { + private[shuffle] def getIndexFile(shuffleId: Int, mapId: Int): File = { blockManager.diskBlockManager.getFile(ShuffleIndexBlockId(shuffleId, mapId, NOOP_REDUCE_ID)) } 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 1ff553d5dc603..a2fc33f1c9b4f 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala @@ -67,13 +67,12 @@ private[spark] class SortShuffleWriter[K, V, C]( // Don't bother including the time to open the merged output file in the shuffle write time, // because it just opens a single file, so is typically too fast to measure accurately // (see SPARK-3570). - val (_, tmpDataFile) = blockManager.diskBlockManager.createTempShuffleBlock() + val (_, tmpDataFile) = blockManager.diskBlockManager.createUncompressedTempShuffleBlock() val blockId = ShuffleBlockId(dep.shuffleId, mapId, IndexShuffleBlockResolver.NOOP_REDUCE_ID) val partitionLengths = sorter.writePartitionedFile(blockId, tmpDataFile) val tmpIndexFile = shuffleBlockResolver.writeIndexFile(dep.shuffleId, mapId, partitionLengths) val dataFile = shuffleBlockResolver.getDataFile(dep.shuffleId, mapId) - val indexFile = blockManager.diskBlockManager.getFile( - ShuffleIndexBlockId(handle.shuffleId, mapId, IndexShuffleBlockResolver.NOOP_REDUCE_ID)) + val indexFile = shuffleBlockResolver.getIndexFile(dep.shuffleId, mapId) mapStatus = MapStatus(blockManager.shuffleServerId, partitionLengths) Seq( 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 1eaf43f23cefa..6d2240a853a52 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockId.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockId.scala @@ -105,6 +105,16 @@ private[spark] case class TempShuffleBlockId(id: UUID) extends BlockId { override def name: String = "temp_shuffle_" + id } +/** + * Id associated with temporary shuffle data managed as blocks, which is not + * compressed, regardless of spark.shuffle.compress and spark.shuffle.spill.compress. Used + * for the temporary location of data files until they are moved into place by the + * [[org.apache.spark.shuffle.ShuffleOutputCoordinator]]. Not serializable. + */ +private[spark] case class TempUncompressedShuffleBlockId(id: UUID) extends BlockId { + override def name: String = "temp_uncompressed_shuffle_" + id +} + // Intended only for testing purposes private[spark] case class TestBlockId(id: String) extends BlockId { override def name: String = "test_" + id diff --git a/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala b/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala index f7e84a2c2e14c..6090b98d23217 100644 --- a/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala @@ -124,6 +124,20 @@ private[spark] class DiskBlockManager(blockManager: BlockManager, conf: SparkCon (blockId, getFile(blockId)) } + /** + * Produces a unique block id and File suitable for storing shuffled data files, which are + * uncompressed, before they are moved to their final location by the + * [[org.apache.spark.shuffle.ShuffleOutputCoordinator]] + */ + def createUncompressedTempShuffleBlock(): (TempUncompressedShuffleBlockId, File) = { + var blockId = new TempUncompressedShuffleBlockId(UUID.randomUUID()) + while (getFile(blockId).exists()) { + blockId = new TempUncompressedShuffleBlockId(UUID.randomUUID()) + } + (blockId, getFile(blockId)) + + } + /** * Create local directories for storing block data. These directories are * located inside configured local directories and won't From dc076b83f892723fe614e80b42da562b5ebf8d9f Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Tue, 27 Oct 2015 12:39:22 -0500 Subject: [PATCH 24/33] fix tests --- .../shuffle/sort/UnsafeShuffleWriterSuite.java | 16 ++++------------ .../sort/BypassMergeSortShuffleWriterSuite.scala | 8 ++------ 2 files changed, 6 insertions(+), 18 deletions(-) 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 45a328b5540e2..7ea444249b4de 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 @@ -66,6 +66,7 @@ public class UnsafeShuffleWriterSuite { TaskMemoryManager taskMemoryManager; final HashPartitioner hashPartitioner = new HashPartitioner(NUM_PARTITITONS); File mergedOutputFile; + File indexFile; File mapStatusFile; File tempDir; long[] partitionSizesInMergedFile; @@ -106,9 +107,11 @@ public void setUp() throws IOException { MockitoAnnotations.initMocks(this); tempDir = Utils.createTempDir("test", "test"); mergedOutputFile = File.createTempFile("mergedoutput", "", tempDir); + indexFile = File.createTempFile("shuffle",".index", tempDir); mapStatusFile = File.createTempFile("shuffle", ".mapstatus", tempDir); // the ShuffleOutputCoordinator requires that this file does not exist mergedOutputFile.delete(); + indexFile.delete(); mapStatusFile.delete(); partitionSizesInMergedFile = null; tmpShuffleFilesCreated.clear(); @@ -171,6 +174,7 @@ public OutputStream answer(InvocationOnMock invocation) throws Throwable { when(blockManager.shuffleServerId()).thenReturn(BlockManagerId$.MODULE$.apply("1", "a.b.c", 1)); when(shuffleBlockResolver.getDataFile(anyInt(), anyInt())).thenReturn(mergedOutputFile); + when(shuffleBlockResolver.getIndexFile(anyInt(), anyInt())).thenReturn(indexFile); doAnswer(new Answer() { @Override public File answer(InvocationOnMock invocationOnMock) throws Throwable { @@ -191,18 +195,6 @@ public Tuple2 answer( } }); - when(diskBlockManager.getFile(any(BlockId.class))).thenAnswer( - new Answer() { - @Override - public File answer(InvocationOnMock invocationOnMock) throws Throwable { - File f = File.createTempFile("shuffleFile",".index", tempDir); - // the ShuffleOutputCoordinator requires that this file does not exist - f.delete(); - return f; - } - } - ); - when(taskContext.taskMetrics()).thenReturn(taskMetrics); when(taskContext.internalMetricsToAccumulators()).thenReturn(null); 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 c0a97f0e1d721..28af9dbea4647 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 @@ -76,6 +76,7 @@ class BypassMergeSortShuffleWriterSuite extends SparkFunSuite with BeforeAndAfte when(dependency.serializer).thenReturn(Some(new JavaSerializer(conf))) when(taskContext.taskMetrics()).thenReturn(taskMetrics) when(blockResolver.getDataFile(0, 0)).thenReturn(outputFile) + when(blockResolver.getIndexFile(0, 0)).thenReturn(indexFile) // the index file will be empty, but that is fine for these tests when(blockResolver.writeIndexFile(anyInt(), anyInt(), any())).thenAnswer(new Answer[File] { override def answer(invocationOnMock: InvocationOnMock): File = { @@ -120,12 +121,7 @@ class BypassMergeSortShuffleWriterSuite extends SparkFunSuite with BeforeAndAfte when(diskBlockManager.getFile(any[BlockId])).thenAnswer( new Answer[File] { override def answer(invocation: InvocationOnMock): File = { - val blk = invocation.getArguments.head.asInstanceOf[BlockId] - if (blk == new ShuffleIndexBlockId(0, 0, IndexShuffleBlockResolver.NOOP_REDUCE_ID)) { - indexFile - } else { - blockIdToFileMap.get(invocation.getArguments.head.asInstanceOf[BlockId]).get - } + blockIdToFileMap.get(invocation.getArguments.head.asInstanceOf[BlockId]).get } }) } From cfdfd2c7aca33a40e687ed6c93859a72d8f8e280 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Tue, 3 Nov 2015 15:39:55 -0600 Subject: [PATCH 25/33] review feedback --- .../sort/BypassMergeSortShuffleWriter.java | 23 +++++------ .../shuffle/sort/UnsafeShuffleWriter.java | 15 ++++---- .../shuffle/FileShuffleBlockResolver.scala | 6 +-- .../shuffle/IndexShuffleBlockResolver.scala | 6 +-- .../shuffle/ShuffleOutputCoordinator.scala | 38 +++++++++++-------- .../apache/spark/shuffle/ShuffleWriter.scala | 18 ++++++++- .../shuffle/hash/HashShuffleWriter.scala | 4 +- .../shuffle/sort/SortShuffleWriter.scala | 12 +++--- .../spark/storage/DiskBlockManager.scala | 1 - .../sort/UnsafeShuffleWriterSuite.java | 15 ++++---- .../ShuffleOutputCoordinatorSuite.scala | 18 ++++----- 11 files changed, 86 insertions(+), 70 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 9db28a5d71425..38e28ff2329c7 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 @@ -24,6 +24,7 @@ import java.util.Arrays; import javax.annotation.Nullable; +import org.apache.spark.*; import scala.None$; import scala.Option; import scala.Product2; @@ -37,14 +38,10 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.spark.Partitioner; -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.scheduler.MapStatus; import org.apache.spark.scheduler.MapStatus$; -import org.apache.spark.shuffle.IndexShuffleBlockResolver$; +import org.apache.spark.shuffle.TmpDestShuffleFile; import org.apache.spark.serializer.Serializer; import org.apache.spark.serializer.SerializerInstance; import org.apache.spark.shuffle.IndexShuffleBlockResolver; @@ -125,7 +122,7 @@ public BypassMergeSortShuffleWriter( } @Override - public Seq> write(Iterator> records) throws IOException { + public Seq write(Iterator> records) throws IOException { assert (partitionWriters == null); final File indexFile = shuffleBlockResolver.getIndexFile(shuffleId, mapId); final File dataFile = shuffleBlockResolver.getDataFile(shuffleId, mapId); @@ -136,10 +133,14 @@ public Seq> write(Iterator> records) throws IO // create empty data file so we always commit same set of shuffle output files, even if // data is non-deterministic final File tmpDataFile = blockManager.diskBlockManager().createTempShuffleBlock()._2(); - tmpDataFile.createNewFile(); + if (!tmpDataFile.createNewFile()) { + // only possible if the file already exists, from a race in createTempShuffleBlock, which + // should be super-rare + throw new IOException("could not create shuffle data file: " + tmpDataFile); + } return JavaConverters.asScalaBufferConverter(Arrays.asList( - new Tuple2<>(tmpIndexFile, indexFile), - new Tuple2<>(tmpDataFile, dataFile) + new TmpDestShuffleFile(tmpIndexFile, indexFile), + new TmpDestShuffleFile(tmpDataFile, dataFile) )).asScala(); } final SerializerInstance serInstance = serializer.newInstance(); @@ -174,8 +175,8 @@ public Seq> write(Iterator> records) throws IO final File tmpIndexFile = shuffleBlockResolver.writeIndexFile(shuffleId, mapId, partitionLengths); mapStatus = MapStatus$.MODULE$.apply(blockManager.shuffleServerId(), partitionLengths); return JavaConverters.asScalaBufferConverter(Arrays.asList( - new Tuple2<>(tmpIndexFile, indexFile), - new Tuple2<>(tmpDataFile, dataFile) + new TmpDestShuffleFile(tmpIndexFile, indexFile), + new TmpDestShuffleFile(tmpDataFile, dataFile) )).asScala(); } 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 d1f8ea7e8bb1e..5d177362abe64 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 @@ -53,10 +53,9 @@ import org.apache.spark.serializer.Serializer; import org.apache.spark.serializer.SerializerInstance; import org.apache.spark.shuffle.IndexShuffleBlockResolver; -import org.apache.spark.shuffle.IndexShuffleBlockResolver$; +import org.apache.spark.shuffle.TmpDestShuffleFile; import org.apache.spark.shuffle.ShuffleWriter; import org.apache.spark.storage.BlockManager; -import org.apache.spark.storage.ShuffleIndexBlockId; import org.apache.spark.storage.TimeTrackingOutputStream; import org.apache.spark.unsafe.Platform; @@ -160,12 +159,12 @@ public long getPeakMemoryUsedBytes() { * This convenience method should only be called in test code. */ @VisibleForTesting - public Seq> write(Iterator> records) throws IOException { + public Seq write(Iterator> records) throws IOException { return write(JavaConverters.asScalaIteratorConverter(records).asScala()); } @Override - public Seq> write(scala.collection.Iterator> records) throws IOException { + public Seq write(scala.collection.Iterator> records) throws IOException { // Keep track of success so we know if we encountered an exception // We do this rather than a standard try/catch/re-throw to handle // generic throwables. @@ -174,7 +173,7 @@ public Seq> write(scala.collection.Iterator> r while (records.hasNext()) { insertRecordIntoSorter(records.next()); } - final Seq> result = closeAndWriteOutput(); + final Seq result = closeAndWriteOutput(); success = true; return result; } finally { @@ -210,7 +209,7 @@ private void open() throws IOException { } @VisibleForTesting - Seq> closeAndWriteOutput() throws IOException { + Seq closeAndWriteOutput() throws IOException { assert(sorter != null); updatePeakMemoryUsed(); serBuffer = null; @@ -236,8 +235,8 @@ Seq> closeAndWriteOutput() throws IOException { final File indexFile = shuffleBlockResolver.getIndexFile(shuffleId, mapId); return JavaConverters.asScalaBufferConverter(Arrays.asList( - new Tuple2<>(tmpIndexFile, indexFile), - new Tuple2<>(tmpDataFile, dataFile) + new TmpDestShuffleFile(tmpIndexFile, indexFile), + new TmpDestShuffleFile(tmpDataFile, dataFile) )).asScala(); } diff --git a/core/src/main/scala/org/apache/spark/shuffle/FileShuffleBlockResolver.scala b/core/src/main/scala/org/apache/spark/shuffle/FileShuffleBlockResolver.scala index 86d47e5e20500..34652b67ffbff 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/FileShuffleBlockResolver.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/FileShuffleBlockResolver.scala @@ -137,10 +137,8 @@ private[spark] class FileShuffleBlockResolver(conf: SparkConf) for (mapId <- state.completedMapTasks.asScala) { val mapStatusFile = blockManager.diskBlockManager.getFile(ShuffleMapStatusBlockId(shuffleId, mapId)) - if (mapStatusFile.exists()) { - if (!mapStatusFile.delete()) { - logWarning(s"Error deleting MapStatus file ${mapStatusFile.getPath()}") - } + if (mapStatusFile.exists() && !mapStatusFile.delete()) { + logWarning(s"Error deleting MapStatus file ${mapStatusFile.getPath()}") } } logInfo("Deleted all files for shuffle " + shuffleId) 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 f0a5c405197da..8c8bf661e8816 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala @@ -74,10 +74,8 @@ private[spark] class IndexShuffleBlockResolver(conf: SparkConf) extends ShuffleB } file = blockManager.diskBlockManager.getFile(ShuffleMapStatusBlockId(shuffleId, mapId)) - if (file.exists()) { - if (!file.delete()) { - logWarning(s"Error deleting MapStatus file ${file.getPath()}") - } + if (file.exists() && !file.delete()) { + logWarning(s"Error deleting MapStatus file ${file.getPath()}") } } diff --git a/core/src/main/scala/org/apache/spark/shuffle/ShuffleOutputCoordinator.scala b/core/src/main/scala/org/apache/spark/shuffle/ShuffleOutputCoordinator.scala index a195316409866..07e6cbdd92244 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/ShuffleOutputCoordinator.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/ShuffleOutputCoordinator.scala @@ -16,14 +16,15 @@ */ package org.apache.spark.shuffle -import java.io.{FileOutputStream, FileInputStream, File} +import java.io.{File, FileInputStream, FileOutputStream, IOException} import com.google.common.annotations.VisibleForTesting -import org.apache.spark.storage.ShuffleMapStatusBlockId -import org.apache.spark.{SparkEnv, Logging} +import org.apache.spark.{Logging, SparkEnv} import org.apache.spark.scheduler.MapStatus import org.apache.spark.serializer.SerializerInstance +import org.apache.spark.storage.ShuffleMapStatusBlockId +import org.apache.spark.util.Utils /** * Ensures that on each executor, there are no conflicting writes to the same shuffle files. It @@ -47,12 +48,11 @@ private[spark] object ShuffleOutputCoordinator extends Logging { * @param mapStatus the [[MapStatus]] for the output already written to the the temporary files * @return pair of: (1) true iff the set of temporary files was moved to the destination and (2) * the MapStatus of the committed attempt. - * */ def commitOutputs( shuffleId: Int, partitionId: Int, - tmpToDest: Seq[(File, File)], + tmpToDest: Seq[TmpDestShuffleFile], mapStatus: MapStatus, sparkEnv: SparkEnv): (Boolean, MapStatus) = synchronized { val mapStatusFile = sparkEnv.blockManager.diskBlockManager.getFile( @@ -65,17 +65,21 @@ private[spark] object ShuffleOutputCoordinator extends Logging { def commitOutputs( shuffleId: Int, partitionId: Int, - tmpToDest: Seq[(File, File)], + tmpToDest: Seq[TmpDestShuffleFile], mapStatus: MapStatus, mapStatusFile: File, serializer: SerializerInstance): (Boolean, MapStatus) = synchronized { - val destAlreadyExists = tmpToDest.forall{_._2.exists()} && mapStatusFile.exists() + // due to SPARK-4085, we only consider the previous attempt "committed" if all its output + // files are present + val destAlreadyExists = tmpToDest.forall(_.dstFile.exists()) && mapStatusFile.exists() if (!destAlreadyExists) { - tmpToDest.foreach { case (tmp, dest) => - // If *some* of the destination files exist, but not all of them, then its not clear + tmpToDest.foreach { case TmpDestShuffleFile(tmp, dest) => + // If *some* of the destination files exist, but not all of them, then it's not clear // what to do. There could be a task already reading from this dest file when we delete // it -- but then again, something in that taskset would be doomed to fail in any case when - // it got to the missing files. Better to just put consistent output into place + // it got to the missing files. Better to just put consistent output into place. + // Note that for this to work with non-determinstic data, it is *critical* that each + // attempt always produces the exact same set of destination files (even if they are empty). if (dest.exists()) { dest.delete() } @@ -85,7 +89,9 @@ private[spark] object ShuffleOutputCoordinator extends Logging { // we always create the destination files, so this works correctly even when the // input data is non-deterministic (potentially empty in one iteration, and non-empty // in another) - dest.createNewFile() + if (!dest.createNewFile()) { + throw new IOException("could not create file: $file") + } } } val out = serializer.serializeStream(new FileOutputStream(mapStatusFile)) @@ -95,11 +101,13 @@ private[spark] object ShuffleOutputCoordinator extends Logging { } else { logInfo(s"shuffle output for shuffle $shuffleId, partition $partitionId already exists, " + s"not overwriting. Another task must have created this shuffle output.") - tmpToDest.foreach{ case (tmp, _) => tmp.delete()} + tmpToDest.foreach{ tmpAndDest => tmpAndDest.tmpFile.delete()} val in = serializer.deserializeStream(new FileInputStream(mapStatusFile)) - val readStatus = in.readObject[MapStatus] - in.close() - (false, readStatus) + Utils.tryWithSafeFinally { + (false, in.readObject[MapStatus]()) + } { + in.close() + } } } } diff --git a/core/src/main/scala/org/apache/spark/shuffle/ShuffleWriter.scala b/core/src/main/scala/org/apache/spark/shuffle/ShuffleWriter.scala index 56af1590b088c..04a8382676d4c 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/ShuffleWriter.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/ShuffleWriter.scala @@ -29,11 +29,25 @@ private[spark] abstract class ShuffleWriter[K, V] { * Write a sequence of records to this task's output. This should write all data * to temporary files, but return (temporaryFile, destinationFile) pairs for each * file written. The temporary files will get moved to their destination or deleted - * by the [[ShuffleOutputCoordinator]] + * by the [[ShuffleOutputCoordinator]]. Note that for the ShuffleOutputCoordinator + * to work correctly, each attempt *must* have the exact same set of destination files. + * If the temporary file is empty, the ShuffleWriter does not have to create the file -- however + * it *must* still be in the result Seq, just pointing to a non-existent file. */ @throws[IOException] - def write(records: Iterator[Product2[K, V]]): Seq[(File, File)] + def write(records: Iterator[Product2[K, V]]): Seq[TmpDestShuffleFile] /** Close this writer, passing along whether the map completed */ def stop(success: Boolean): Option[MapStatus] } + +/** + * The location of one shuffle file written by a [[ShuffleWriter]]. Holds both the temporary + * file, which is written to by the ShuffleWriter itself, and the destination file, where the + * file should get moved by the [[ShuffleOutputCoordinator]]. The ShuffleWriter is responsible + * for specifying both locations, though it only writes the temp file. + */ +private[shuffle] case class TmpDestShuffleFile( + val tmpFile: File, + val dstFile: File +) diff --git a/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleWriter.scala b/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleWriter.scala index 42050b4116584..011270c050459 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleWriter.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleWriter.scala @@ -51,7 +51,7 @@ private[spark] class HashShuffleWriter[K, V]( writeMetrics) /** Write a bunch of records to this task's output */ - override def write(records: Iterator[Product2[K, V]]): Seq[(File, File)] = { + override def write(records: Iterator[Product2[K, V]]): Seq[TmpDestShuffleFile] = { val iter = if (dep.aggregator.isDefined) { if (dep.mapSideCombine) { dep.aggregator.get.combineValuesByKey(records, context) @@ -67,7 +67,7 @@ private[spark] class HashShuffleWriter[K, V]( val bucketId = dep.partitioner.getPartition(elem._1) shuffle.writers(bucketId)._1.write(elem._1, elem._2) } - shuffle.writers.map { case (writer, destFile) => writer.file -> destFile} + shuffle.writers.map { case (writer, destFile) => TmpDestShuffleFile(writer.file, destFile) } } /** Close this writer, passing along whether the map completed */ 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 a2fc33f1c9b4f..c20aea8db65d0 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 @@ -17,13 +17,11 @@ package org.apache.spark.shuffle.sort -import java.io.File - import org.apache.spark._ import org.apache.spark.executor.ShuffleWriteMetrics import org.apache.spark.scheduler.MapStatus -import org.apache.spark.shuffle.{BaseShuffleHandle, IndexShuffleBlockResolver, ShuffleWriter} -import org.apache.spark.storage.{ShuffleBlockId, ShuffleIndexBlockId, ShuffleMapStatusBlockId} +import org.apache.spark.shuffle.{BaseShuffleHandle, IndexShuffleBlockResolver, ShuffleWriter, TmpDestShuffleFile} +import org.apache.spark.storage.ShuffleBlockId import org.apache.spark.util.collection.ExternalSorter private[spark] class SortShuffleWriter[K, V, C]( @@ -50,7 +48,7 @@ private[spark] class SortShuffleWriter[K, V, C]( context.taskMetrics.shuffleWriteMetrics = Some(writeMetrics) /** Write a bunch of records to this task's output */ - override def write(records: Iterator[Product2[K, V]]): Seq[(File, File)] = { + override def write(records: Iterator[Product2[K, V]]): Seq[TmpDestShuffleFile] = { sorter = if (dep.mapSideCombine) { require(dep.aggregator.isDefined, "Map-side combine without Aggregator specified!") new ExternalSorter[K, V, C]( @@ -76,8 +74,8 @@ private[spark] class SortShuffleWriter[K, V, C]( mapStatus = MapStatus(blockManager.shuffleServerId, partitionLengths) Seq( - tmpDataFile -> dataFile, - tmpIndexFile -> indexFile + TmpDestShuffleFile(tmpDataFile, dataFile), + TmpDestShuffleFile(tmpIndexFile, indexFile) ) } diff --git a/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala b/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala index 6090b98d23217..be26648c556e2 100644 --- a/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala @@ -135,7 +135,6 @@ private[spark] class DiskBlockManager(blockManager: BlockManager, conf: SparkCon blockId = new TempUncompressedShuffleBlockId(UUID.randomUUID()) } (blockId, getFile(blockId)) - } /** 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 7ea444249b4de..7cd966d42c7b3 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 @@ -57,6 +57,7 @@ import org.apache.spark.scheduler.MapStatus; import org.apache.spark.shuffle.IndexShuffleBlockResolver; import org.apache.spark.shuffle.ShuffleOutputCoordinator; +import org.apache.spark.shuffle.TmpDestShuffleFile; import org.apache.spark.storage.*; import org.apache.spark.util.Utils; @@ -279,7 +280,7 @@ class BadRecords extends scala.collection.AbstractIterator writer = createWriter(true); - Seq> files = writer.write(Iterators.>emptyIterator()); + Seq files = writer.write(Iterators.>emptyIterator()); final Option mapStatus = writer.stop(true); assertTrue(mapStatus.isDefined()); ShuffleOutputCoordinator.commitOutputs(0, 0, files, mapStatus.get(), mapStatusFile, @@ -301,7 +302,7 @@ public void writeWithoutSpilling() throws Exception { dataToWrite.add(new Tuple2(i, i)); } final UnsafeShuffleWriter writer = createWriter(true); - Seq> files = writer.write(dataToWrite.iterator()); + Seq files = writer.write(dataToWrite.iterator()); final Option mapStatus = writer.stop(true); ShuffleOutputCoordinator.commitOutputs(0, 0, files, mapStatus.get(), mapStatusFile, serializer.newInstance()); @@ -348,7 +349,7 @@ private void testMergingSpills( writer.forceSorterToSpill(); writer.insertRecordIntoSorter(dataToWrite.get(4)); writer.insertRecordIntoSorter(dataToWrite.get(5)); - Seq> files = writer.closeAndWriteOutput(); + Seq files = writer.closeAndWriteOutput(); final Option mapStatus = writer.stop(true); ShuffleOutputCoordinator.commitOutputs(0, 0, files, mapStatus.get(), mapStatusFile, serializer.newInstance()); @@ -429,7 +430,7 @@ public void writeEnoughDataToTriggerSpill() throws Exception { for (int i = 0; i < 128 + 1; i++) { dataToWrite.add(new Tuple2(i, bigByteArray)); } - Seq> files = writer.write(dataToWrite.iterator()); + Seq files = writer.write(dataToWrite.iterator()); verify(taskMemoryManager, times(5)).acquireExecutionMemory(anyLong()); // this includes the tmp index & data files, before the output is committed assertEquals(4, tmpShuffleFilesCreated.size()); @@ -459,7 +460,7 @@ public void writeEnoughRecordsToTriggerSortBufferExpansionAndSpill() throws Exce for (int i = 0; i < UnsafeShuffleWriter.INITIAL_SORT_BUFFER_SIZE; i++) { dataToWrite.add(new Tuple2(i, i)); } - Seq> files = writer.write(dataToWrite.iterator()); + Seq files = writer.write(dataToWrite.iterator()); verify(taskMemoryManager, times(5)).acquireExecutionMemory(anyLong()); // this includes the tmp index & data files, before the output is committed assertEquals(4, tmpShuffleFilesCreated.size()); @@ -484,7 +485,7 @@ public void writeRecordsThatAreBiggerThanDiskWriteBufferSize() throws Exception final byte[] bytes = new byte[(int) (ShuffleExternalSorter.DISK_WRITE_BUFFER_SIZE * 2.5)]; new Random(42).nextBytes(bytes); dataToWrite.add(new Tuple2(1, ByteBuffer.wrap(bytes))); - Seq> files = writer.write(dataToWrite.iterator()); + Seq files = writer.write(dataToWrite.iterator()); MapStatus mapStatus = writer.stop(true).get(); ShuffleOutputCoordinator.commitOutputs(0, 0, files, mapStatus, mapStatusFile, serializer.newInstance()); @@ -507,7 +508,7 @@ public void writeRecordsThatAreBiggerThanMaxRecordSize() throws Exception { final byte[] exceedsMaxRecordSize = new byte[writer.maxRecordSizeBytes() + 1]; new Random(42).nextBytes(exceedsMaxRecordSize); dataToWrite.add(new Tuple2(3, ByteBuffer.wrap(exceedsMaxRecordSize))); - Seq> files = writer.write(dataToWrite.iterator()); + Seq files = writer.write(dataToWrite.iterator()); MapStatus mapStatus = writer.stop(true).get(); ShuffleOutputCoordinator.commitOutputs(0, 0, files, mapStatus, mapStatusFile, serializer.newInstance()); diff --git a/core/src/test/scala/org/apache/spark/shuffle/ShuffleOutputCoordinatorSuite.scala b/core/src/test/scala/org/apache/spark/shuffle/ShuffleOutputCoordinatorSuite.scala index d806ea6a250e9..8a1bb74f5b77f 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/ShuffleOutputCoordinatorSuite.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/ShuffleOutputCoordinatorSuite.scala @@ -66,14 +66,14 @@ class ShuffleOutputCoordinatorSuite extends SparkFunSuite with BeforeAndAfterEac } } - private def generateAttempt(attempt: Int): Seq[(File, File)] = { + private def generateAttempt(attempt: Int): Seq[TmpDestShuffleFile] = { (0 until 3).map { idx => val j = attempt * 3 + idx - writeFile(s"t$j", j) -> new File(tempDir, s"d$idx") + TmpDestShuffleFile(writeFile(s"t$j", j), new File(tempDir, s"d$idx")) } } - private def commit(files: Seq[(File, File)], id: Int): (Boolean, MapStatus) = { + private def commit(files: Seq[TmpDestShuffleFile], id: Int): (Boolean, MapStatus) = { ShuffleOutputCoordinator.commitOutputs(0, 0, files, mapStatus(id), mapStatusFile, ser) } @@ -83,7 +83,7 @@ class ShuffleOutputCoordinatorSuite extends SparkFunSuite with BeforeAndAfterEac assert(firstCommit._1) assert(firstCommit._2.location.port === 1) verifyFiles(0) - firstAttempt.foreach{ case (t, d) => assert(!t.exists())} + firstAttempt.foreach{ case TmpDestShuffleFile(t, d) => assert(!t.exists())} val secondAttempt = generateAttempt(1) // second commit fails, and also deletes the tmp files @@ -93,7 +93,7 @@ class ShuffleOutputCoordinatorSuite extends SparkFunSuite with BeforeAndAfterEac assert(firstCommit._2.location.port === 1) verifyFiles(0) // make sure we delete the temp files if the dest exists - secondAttempt.foreach{ case (t, d) => assert(!t.exists())} + secondAttempt.foreach{ case TmpDestShuffleFile(t, d) => assert(!t.exists())} } test("move files if just map status file missing") { @@ -102,7 +102,7 @@ class ShuffleOutputCoordinatorSuite extends SparkFunSuite with BeforeAndAfterEac assert(firstCommit._1) assert(firstCommit._2.location.port === 1) verifyFiles(0) - firstAttempt.foreach{ case (t, d) => assert(!t.exists())} + firstAttempt.foreach{ case TmpDestShuffleFile(t, d) => assert(!t.exists())} val secondAttempt = generateAttempt(1) mapStatusFile.delete() @@ -111,13 +111,13 @@ class ShuffleOutputCoordinatorSuite extends SparkFunSuite with BeforeAndAfterEac assert(secondCommit._1) assert(secondCommit._2.location.port === 2) verifyFiles(1) - secondAttempt.foreach{ case (t, d) => assert(!t.exists())} + secondAttempt.foreach{ case TmpDestShuffleFile(t, d) => assert(!t.exists())} } test("missing tmp files become zero-length destination files") { val extraDestFile = new File(tempDir, "blah") val firstAttempt = generateAttempt(0) ++ - Seq(new File(tempDir, "bogus") -> extraDestFile) + Seq(TmpDestShuffleFile(new File(tempDir, "bogus"), extraDestFile)) assert(commit(firstAttempt, 1)._1) verifyFiles(0) assert(extraDestFile.exists()) @@ -127,7 +127,7 @@ class ShuffleOutputCoordinatorSuite extends SparkFunSuite with BeforeAndAfterEac // do the move extraDestFile.delete() val secondAttempt = generateAttempt(1) ++ - Seq(new File(tempDir, "flippy") -> extraDestFile) + Seq(TmpDestShuffleFile(new File(tempDir, "flippy"), extraDestFile)) assert(commit(secondAttempt, 2)._1) verifyFiles(1) assert(extraDestFile.exists()) From 5c8b24741982a2e14f5bb2f9e65625ee7364ef01 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Wed, 4 Nov 2015 12:53:45 -0600 Subject: [PATCH 26/33] fix imports --- .../shuffle/sort/BypassMergeSortShuffleWriterSuite.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) 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 28af9dbea4647..b5704eb58edfd 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 @@ -32,9 +32,9 @@ import org.mockito.stubbing.Answer import org.scalatest.BeforeAndAfterEach import org.apache.spark._ -import org.apache.spark.executor.{TaskMetrics, ShuffleWriteMetrics} -import org.apache.spark.shuffle.{ShuffleOutputCoordinator, IndexShuffleBlockResolver} +import org.apache.spark.executor.{ShuffleWriteMetrics, TaskMetrics} import org.apache.spark.serializer.{JavaSerializer, SerializerInstance} +import org.apache.spark.shuffle.{IndexShuffleBlockResolver, ShuffleOutputCoordinator} import org.apache.spark.storage._ import org.apache.spark.util.Utils From 4d66df1ac9381cdecb5b0a045046e048c5333a87 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Wed, 4 Nov 2015 12:58:59 -0600 Subject: [PATCH 27/33] fix more imports --- .../apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java | 2 +- 1 file changed, 1 insertion(+), 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 38e28ff2329c7..2b5721c1e216f 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 @@ -24,7 +24,6 @@ import java.util.Arrays; import javax.annotation.Nullable; -import org.apache.spark.*; import scala.None$; import scala.Option; import scala.Product2; @@ -38,6 +37,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import org.apache.spark.*; import org.apache.spark.executor.ShuffleWriteMetrics; import org.apache.spark.scheduler.MapStatus; import org.apache.spark.scheduler.MapStatus$; From e59df413f206b7c727d276dca0bbd6de1180ad63 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Wed, 4 Nov 2015 13:03:13 -0600 Subject: [PATCH 28/33] couple more nits ... --- .../org/apache/spark/shuffle/ShuffleOutputCoordinator.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/shuffle/ShuffleOutputCoordinator.scala b/core/src/main/scala/org/apache/spark/shuffle/ShuffleOutputCoordinator.scala index 07e6cbdd92244..0d0d5f2247174 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/ShuffleOutputCoordinator.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/ShuffleOutputCoordinator.scala @@ -41,7 +41,7 @@ private[spark] object ShuffleOutputCoordinator extends Logging { * * Note that this will write to all destination files. If the tmp file is missing, then a * zero-length destination file will be created. This is so the ShuffleOutputCoordinator can work - * even when there is a non-determinstic data, where the output exists in one attempt, but is + * even when there is non-determinstic data, where the output exists in one attempt, but is * empty in another attempt. * * @param tmpToDest Seq of (temporary, destination) file pairs @@ -101,7 +101,7 @@ private[spark] object ShuffleOutputCoordinator extends Logging { } else { logInfo(s"shuffle output for shuffle $shuffleId, partition $partitionId already exists, " + s"not overwriting. Another task must have created this shuffle output.") - tmpToDest.foreach{ tmpAndDest => tmpAndDest.tmpFile.delete()} + tmpToDest.foreach { tmpAndDest => tmpAndDest.tmpFile.delete() } val in = serializer.deserializeStream(new FileInputStream(mapStatusFile)) Utils.tryWithSafeFinally { (false, in.readObject[MapStatus]()) From c206fc59239f8d0f90aa018aa3dfb0c309a174d0 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Wed, 4 Nov 2015 20:49:37 -0600 Subject: [PATCH 29/33] minor cleanup --- core/src/test/scala/org/apache/spark/ShuffleSuite.scala | 4 ++-- .../spark/shuffle/ShuffleOutputCoordinatorSuite.scala | 8 +++++--- 2 files changed, 7 insertions(+), 5 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/ShuffleSuite.scala b/core/src/test/scala/org/apache/spark/ShuffleSuite.scala index aeedbe4c565df..6d5d7dd5f5c2d 100644 --- a/core/src/test/scala/org/apache/spark/ShuffleSuite.scala +++ b/core/src/test/scala/org/apache/spark/ShuffleSuite.scala @@ -378,8 +378,8 @@ abstract class ShuffleSuite extends SparkFunSuite with Matchers with LocalSparkC // simultaneously, and everything is still OK def writeAndClose( - writer: ShuffleWriter[Int, Int])( - iter: Iterator[(Int, Int)]): Option[(Boolean, MapStatus)] = { + writer: ShuffleWriter[Int, Int])( + iter: Iterator[(Int, Int)]): Option[(Boolean, MapStatus)] = { val files = writer.write(iter) val output = writer.stop(true) output.map(ShuffleOutputCoordinator.commitOutputs(0, 0, files, _, SparkEnv.get)) diff --git a/core/src/test/scala/org/apache/spark/shuffle/ShuffleOutputCoordinatorSuite.scala b/core/src/test/scala/org/apache/spark/shuffle/ShuffleOutputCoordinatorSuite.scala index 8a1bb74f5b77f..c70aa64c2e268 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/ShuffleOutputCoordinatorSuite.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/ShuffleOutputCoordinatorSuite.scala @@ -31,7 +31,7 @@ class ShuffleOutputCoordinatorSuite extends SparkFunSuite with BeforeAndAfterEac var tempDir: File = _ var mapStatusFile: File = _ // use the "port" as a way to distinguish mapstatuses, just for the test - def mapStatus(id: Int): MapStatus = MapStatus(BlockManagerId("1", "a.b.c", id), Array(0L, 1L)) + def mapStatus(attemptId: Int): MapStatus = MapStatus(BlockManagerId("1", "a.b.c", attemptId), Array(0L, 1L)) def ser: SerializerInstance = new JavaSerializer(new SparkConf()).newInstance() override def beforeEach(): Unit = { @@ -73,14 +73,15 @@ class ShuffleOutputCoordinatorSuite extends SparkFunSuite with BeforeAndAfterEac } } - private def commit(files: Seq[TmpDestShuffleFile], id: Int): (Boolean, MapStatus) = { - ShuffleOutputCoordinator.commitOutputs(0, 0, files, mapStatus(id), mapStatusFile, ser) + private def commit(files: Seq[TmpDestShuffleFile], attemptId: Int): (Boolean, MapStatus) = { + ShuffleOutputCoordinator.commitOutputs(0, 0, files, mapStatus(attemptId), mapStatusFile, ser) } test("move files if dest missing") { val firstAttempt = generateAttempt(0) val firstCommit = commit(firstAttempt, 1) assert(firstCommit._1) + // "port" is just our holder for the attempt that succeeded in this test setup assert(firstCommit._2.location.port === 1) verifyFiles(0) firstAttempt.foreach{ case TmpDestShuffleFile(t, d) => assert(!t.exists())} @@ -100,6 +101,7 @@ class ShuffleOutputCoordinatorSuite extends SparkFunSuite with BeforeAndAfterEac val firstAttempt = generateAttempt(0) val firstCommit = commit(firstAttempt, 1) assert(firstCommit._1) + // "port" is just our holder for the attempt that succeeded in this test setup assert(firstCommit._2.location.port === 1) verifyFiles(0) firstAttempt.foreach{ case TmpDestShuffleFile(t, d) => assert(!t.exists())} From c0edff1cd33ed7aa74ecbde770f835ca9b51fc08 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Thu, 5 Nov 2015 00:14:09 -0600 Subject: [PATCH 30/33] style --- .../apache/spark/shuffle/ShuffleOutputCoordinatorSuite.scala | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/core/src/test/scala/org/apache/spark/shuffle/ShuffleOutputCoordinatorSuite.scala b/core/src/test/scala/org/apache/spark/shuffle/ShuffleOutputCoordinatorSuite.scala index c70aa64c2e268..62ccb6932a395 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/ShuffleOutputCoordinatorSuite.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/ShuffleOutputCoordinatorSuite.scala @@ -31,7 +31,9 @@ class ShuffleOutputCoordinatorSuite extends SparkFunSuite with BeforeAndAfterEac var tempDir: File = _ var mapStatusFile: File = _ // use the "port" as a way to distinguish mapstatuses, just for the test - def mapStatus(attemptId: Int): MapStatus = MapStatus(BlockManagerId("1", "a.b.c", attemptId), Array(0L, 1L)) + def mapStatus(attemptId: Int): MapStatus = { + MapStatus(BlockManagerId("1", "a.b.c", attemptId), Array(0L, 1L)) + } def ser: SerializerInstance = new JavaSerializer(new SparkConf()).newInstance() override def beforeEach(): Unit = { From c0b93a503f0461873482e6e9b88adbfeb4b0635e Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Tue, 10 Nov 2015 21:45:01 -0600 Subject: [PATCH 31/33] create temporary files in same location as destination files --- .../sort/BypassMergeSortShuffleWriter.java | 8 +++--- .../shuffle/sort/UnsafeShuffleWriter.java | 23 +++++++--------- .../shuffle/FileShuffleBlockResolver.scala | 2 +- .../shuffle/IndexShuffleBlockResolver.scala | 6 ++--- .../apache/spark/shuffle/ShuffleWriter.scala | 16 ++++++++++++ .../shuffle/sort/SortShuffleWriter.scala | 9 ++++--- .../org/apache/spark/storage/BlockId.scala | 10 ------- .../spark/storage/DiskBlockManager.scala | 13 ---------- .../sort/UnsafeShuffleWriterSuite.java | 26 +++++++++---------- .../BypassMergeSortShuffleWriterSuite.scala | 9 +------ 10 files changed, 50 insertions(+), 72 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 2b5721c1e216f..a348f2e0176b3 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 @@ -126,13 +126,13 @@ public Seq write(Iterator> records) throws IO assert (partitionWriters == null); final File indexFile = shuffleBlockResolver.getIndexFile(shuffleId, mapId); final File dataFile = shuffleBlockResolver.getDataFile(shuffleId, mapId); + final File tmpIndexFile = tmpShuffleFile(indexFile); + final File tmpDataFile = tmpShuffleFile(dataFile); if (!records.hasNext()) { partitionLengths = new long[numPartitions]; - final File tmpIndexFile = shuffleBlockResolver.writeIndexFile(shuffleId, mapId, partitionLengths); mapStatus = MapStatus$.MODULE$.apply(blockManager.shuffleServerId(), partitionLengths); // create empty data file so we always commit same set of shuffle output files, even if // data is non-deterministic - final File tmpDataFile = blockManager.diskBlockManager().createTempShuffleBlock()._2(); if (!tmpDataFile.createNewFile()) { // only possible if the file already exists, from a race in createTempShuffleBlock, which // should be super-rare @@ -169,10 +169,8 @@ public Seq write(Iterator> records) throws IO writer.commitAndClose(); } - final File tmpDataFile = blockManager.diskBlockManager().createTempShuffleBlock()._2(); - partitionLengths = writePartitionedFile(tmpDataFile); - final File tmpIndexFile = shuffleBlockResolver.writeIndexFile(shuffleId, mapId, partitionLengths); + shuffleBlockResolver.writeIndexFile(shuffleId, mapId, partitionLengths, tmpIndexFile); mapStatus = MapStatus$.MODULE$.apply(blockManager.shuffleServerId(), partitionLengths); return JavaConverters.asScalaBufferConverter(Arrays.asList( new TmpDestShuffleFile(tmpIndexFile, indexFile), 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 eb34e44c93fe7..27e8357ac3e2e 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 @@ -25,7 +25,6 @@ import scala.Option; import scala.Product2; -import scala.Tuple2; import scala.collection.JavaConverters; import scala.collection.Seq; import scala.collection.immutable.Map; @@ -211,11 +210,12 @@ Seq closeAndWriteOutput() throws IOException { final SpillInfo[] spills = sorter.closeAndGetSpills(); sorter = null; final long[] partitionLengths; - final File tmpDataFile; + final File dataFile = shuffleBlockResolver.getDataFile(shuffleId, mapId); + final File indexFile = shuffleBlockResolver.getIndexFile(shuffleId, mapId); + final File tmpDataFile = tmpShuffleFile(dataFile); + final File tmpIndexFile = tmpShuffleFile(indexFile); try { - final Tuple2 t = mergeSpills(spills); - partitionLengths = t._2(); - tmpDataFile = t._1(); + partitionLengths = mergeSpills(spills, tmpDataFile); } finally { for (SpillInfo spill : spills) { if (spill.file.exists() && ! spill.file.delete()) { @@ -223,10 +223,8 @@ Seq closeAndWriteOutput() throws IOException { } } } - final File tmpIndexFile = shuffleBlockResolver.writeIndexFile(shuffleId, mapId, partitionLengths); + shuffleBlockResolver.writeIndexFile(shuffleId, mapId, partitionLengths, tmpIndexFile); mapStatus = MapStatus$.MODULE$.apply(blockManager.shuffleServerId(), partitionLengths); - final File dataFile = shuffleBlockResolver.getDataFile(shuffleId, mapId); - final File indexFile = shuffleBlockResolver.getIndexFile(shuffleId, mapId); return JavaConverters.asScalaBufferConverter(Arrays.asList( new TmpDestShuffleFile(tmpIndexFile, indexFile), @@ -263,8 +261,7 @@ void forceSorterToSpill() throws IOException { * * @return the partition lengths in the merged file. */ - private Tuple2 mergeSpills(SpillInfo[] spills) throws IOException { - final File outputFile = blockManager.diskBlockManager().createTempShuffleBlock()._2(); + private long[] mergeSpills(SpillInfo[] spills, File outputFile) throws IOException { final boolean compressionEnabled = sparkConf.getBoolean("spark.shuffle.compress", true); final CompressionCodec compressionCodec = CompressionCodec$.MODULE$.createCodec(sparkConf); final boolean fastMergeEnabled = @@ -274,12 +271,12 @@ private Tuple2 mergeSpills(SpillInfo[] spills) throws IOException try { if (spills.length == 0) { new FileOutputStream(outputFile).close(); // Create an empty file - return new Tuple2<>(outputFile, new long[partitioner.numPartitions()]); + return new long[partitioner.numPartitions()]; } else if (spills.length == 1) { // Here, we don't need to perform any metrics updates because the bytes written to this // output file would have already been counted as shuffle bytes written. Files.move(spills[0].file, outputFile); - return new Tuple2<>(outputFile, spills[0].partitionLengths); + return spills[0].partitionLengths; } else { final long[] partitionLengths; // There are multiple spills to merge, so none of these spill files' lengths were counted @@ -314,7 +311,7 @@ private Tuple2 mergeSpills(SpillInfo[] spills) throws IOException // SpillInfo's bytes. writeMetrics.decShuffleBytesWritten(spills[spills.length - 1].file.length()); writeMetrics.incShuffleBytesWritten(outputFile.length()); - return new Tuple2<>(outputFile, partitionLengths); + return partitionLengths; } } catch (IOException e) { if (outputFile.exists() && !outputFile.delete()) { diff --git a/core/src/main/scala/org/apache/spark/shuffle/FileShuffleBlockResolver.scala b/core/src/main/scala/org/apache/spark/shuffle/FileShuffleBlockResolver.scala index 34652b67ffbff..1f2bf387e9b53 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/FileShuffleBlockResolver.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/FileShuffleBlockResolver.scala @@ -85,7 +85,7 @@ private[spark] class FileShuffleBlockResolver(conf: SparkConf) Array.tabulate[(DiskBlockObjectWriter, File)](numReducers) { bucketId => val blockId = ShuffleBlockId(shuffleId, mapId, bucketId) val blockFile = blockManager.diskBlockManager.getFile(blockId) - val (_, tmpBlockFile) = blockManager.diskBlockManager.createTempShuffleBlock() + val tmpBlockFile = ShuffleWriter.tmpShuffleFile(blockFile) // Because of previous failures, the shuffle file may already exist on this machine. // If so, remove it. if (blockFile.exists) { 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 8c8bf661e8816..be6cd20f47dc1 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala @@ -83,9 +83,8 @@ private[spark] class IndexShuffleBlockResolver(conf: SparkConf) extends ShuffleB * 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. Writes to a temp file, and returns that file. - * */ - def writeIndexFile(shuffleId: Int, mapId: Int, lengths: Array[Long]): File = { - val (_, tmpIndexFile) = blockManager.diskBlockManager.createTempShuffleBlock() + */ + def writeIndexFile(shuffleId: Int, mapId: Int, lengths: Array[Long], tmpIndexFile: File): Unit = { val out = new DataOutputStream(new BufferedOutputStream(new FileOutputStream(tmpIndexFile))) Utils.tryWithSafeFinally { // We take in lengths of each block, need to convert it to offsets. @@ -95,7 +94,6 @@ private[spark] class IndexShuffleBlockResolver(conf: SparkConf) extends ShuffleB offset += length out.writeLong(offset) } - tmpIndexFile } { out.close() } diff --git a/core/src/main/scala/org/apache/spark/shuffle/ShuffleWriter.scala b/core/src/main/scala/org/apache/spark/shuffle/ShuffleWriter.scala index 04a8382676d4c..e84c4f377a621 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/ShuffleWriter.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/ShuffleWriter.scala @@ -18,6 +18,7 @@ package org.apache.spark.shuffle import java.io.{File, IOException} +import java.util.UUID import org.apache.spark.scheduler.MapStatus @@ -39,6 +40,21 @@ private[spark] abstract class ShuffleWriter[K, V] { /** Close this writer, passing along whether the map completed */ def stop(success: Boolean): Option[MapStatus] + + /** + * Returns the file with a random UUID appended. Useful for getting a tmp file in the same + * dir which can be atomically renamed to final destination file. + */ + def tmpShuffleFile(file: File): File = ShuffleWriter.tmpShuffleFile(file) +} + + +private[spark] object ShuffleWriter { + /** + * Returns the file with a random UUID appended. Useful for getting a tmp file in the same + * dir which can be atomically renamed to final destination file. + */ + def tmpShuffleFile(file: File): File = new File(file.getAbsolutePath + "." + UUID.randomUUID()) } /** 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 c20aea8db65d0..f10ca6adb92e3 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 @@ -65,12 +65,13 @@ private[spark] class SortShuffleWriter[K, V, C]( // Don't bother including the time to open the merged output file in the shuffle write time, // because it just opens a single file, so is typically too fast to measure accurately // (see SPARK-3570). - val (_, tmpDataFile) = blockManager.diskBlockManager.createUncompressedTempShuffleBlock() - val blockId = ShuffleBlockId(dep.shuffleId, mapId, IndexShuffleBlockResolver.NOOP_REDUCE_ID) - val partitionLengths = sorter.writePartitionedFile(blockId, tmpDataFile) - val tmpIndexFile = shuffleBlockResolver.writeIndexFile(dep.shuffleId, mapId, partitionLengths) val dataFile = shuffleBlockResolver.getDataFile(dep.shuffleId, mapId) val indexFile = shuffleBlockResolver.getIndexFile(dep.shuffleId, mapId) + val tmpDataFile = tmpShuffleFile(dataFile) + val tmpIndexFile = tmpShuffleFile(indexFile) + val blockId = ShuffleBlockId(dep.shuffleId, mapId, IndexShuffleBlockResolver.NOOP_REDUCE_ID) + val partitionLengths = sorter.writePartitionedFile(blockId, tmpDataFile) + shuffleBlockResolver.writeIndexFile(dep.shuffleId, mapId, partitionLengths, tmpIndexFile) mapStatus = MapStatus(blockManager.shuffleServerId, partitionLengths) Seq( 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 6d2240a853a52..1eaf43f23cefa 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockId.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockId.scala @@ -105,16 +105,6 @@ private[spark] case class TempShuffleBlockId(id: UUID) extends BlockId { override def name: String = "temp_shuffle_" + id } -/** - * Id associated with temporary shuffle data managed as blocks, which is not - * compressed, regardless of spark.shuffle.compress and spark.shuffle.spill.compress. Used - * for the temporary location of data files until they are moved into place by the - * [[org.apache.spark.shuffle.ShuffleOutputCoordinator]]. Not serializable. - */ -private[spark] case class TempUncompressedShuffleBlockId(id: UUID) extends BlockId { - override def name: String = "temp_uncompressed_shuffle_" + id -} - // Intended only for testing purposes private[spark] case class TestBlockId(id: String) extends BlockId { override def name: String = "test_" + id diff --git a/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala b/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala index be26648c556e2..f7e84a2c2e14c 100644 --- a/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala @@ -124,19 +124,6 @@ private[spark] class DiskBlockManager(blockManager: BlockManager, conf: SparkCon (blockId, getFile(blockId)) } - /** - * Produces a unique block id and File suitable for storing shuffled data files, which are - * uncompressed, before they are moved to their final location by the - * [[org.apache.spark.shuffle.ShuffleOutputCoordinator]] - */ - def createUncompressedTempShuffleBlock(): (TempUncompressedShuffleBlockId, File) = { - var blockId = new TempUncompressedShuffleBlockId(UUID.randomUUID()) - while (getFile(blockId).exists()) { - blockId = new TempUncompressedShuffleBlockId(UUID.randomUUID()) - } - (blockId, getFile(blockId)) - } - /** * Create local directories for storing block data. These directories are * located inside configured local directories and won't 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 f9094b735962e..9b48425d812a9 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 @@ -72,7 +72,7 @@ public class UnsafeShuffleWriterSuite { File mapStatusFile; File tempDir; long[] partitionSizesInMergedFile; - final LinkedList tmpShuffleFilesCreated = new LinkedList(); + final LinkedList spillFilesCreated = new LinkedList(); SparkConf conf; final Serializer serializer = new KryoSerializer(new SparkConf()); TaskMetrics taskMetrics; @@ -116,7 +116,7 @@ public void setUp() throws IOException { indexFile.delete(); mapStatusFile.delete(); partitionSizesInMergedFile = null; - tmpShuffleFilesCreated.clear(); + spillFilesCreated.clear(); conf = new SparkConf() .set("spark.buffer.pageSize", "1m") .set("spark.unsafe.offHeap", "false"); @@ -178,13 +178,14 @@ public OutputStream answer(InvocationOnMock invocation) throws Throwable { when(shuffleBlockResolver.getDataFile(anyInt(), anyInt())).thenReturn(mergedOutputFile); when(shuffleBlockResolver.getIndexFile(anyInt(), anyInt())).thenReturn(indexFile); - doAnswer(new Answer() { + doAnswer(new Answer() { @Override - public File answer(InvocationOnMock invocationOnMock) throws Throwable { + public Void answer(InvocationOnMock invocationOnMock) throws Throwable { partitionSizesInMergedFile = (long[]) invocationOnMock.getArguments()[2]; - return diskBlockManager.createTempShuffleBlock()._2(); + return null; } - }).when(shuffleBlockResolver).writeIndexFile(anyInt(), anyInt(), any(long[].class)); + }).when(shuffleBlockResolver) + .writeIndexFile(anyInt(), anyInt(), any(long[].class), any(File.class)); when(diskBlockManager.createTempShuffleBlock()).thenAnswer( new Answer>() { @@ -193,7 +194,7 @@ public Tuple2 answer( InvocationOnMock invocationOnMock) throws Throwable { TempShuffleBlockId blockId = new TempShuffleBlockId(UUID.randomUUID()); File file = File.createTempFile("spillFile", ".spill", tempDir); - tmpShuffleFilesCreated.add(file); + spillFilesCreated.add(file); return Tuple2$.MODULE$.apply(blockId, file); } }); @@ -220,7 +221,7 @@ private UnsafeShuffleWriter createWriter( } private void assertSpillFilesWereCleanedUp() { - for (File spillFile : tmpShuffleFilesCreated) { + for (File spillFile : spillFilesCreated) { assertFalse("Spill file " + spillFile.getPath() + " was not cleaned up", spillFile.exists()); } @@ -357,8 +358,7 @@ private void testMergingSpills( serializer.newInstance()); assertTrue(mapStatus.isDefined()); assertTrue(mergedOutputFile.exists()); - // this includes the tmp index & data files, before the output is committed - assertEquals(4, tmpShuffleFilesCreated.size()); + assertEquals(2, spillFilesCreated.size()); long sumOfPartitionSizes = 0; for (long size: partitionSizesInMergedFile) { @@ -426,8 +426,7 @@ public void writeEnoughDataToTriggerSpill() throws Exception { dataToWrite.add(new Tuple2(i, bigByteArray)); } Seq files = writer.write(dataToWrite.iterator()); - // this includes the tmp index & data files, before the output is committed - assertEquals(4, tmpShuffleFilesCreated.size()); + assertEquals(2, spillFilesCreated.size()); MapStatus mapStatus = writer.stop(true).get(); ShuffleOutputCoordinator.commitOutputs(0, 0, files, mapStatus, mapStatusFile, serializer.newInstance()); @@ -450,8 +449,7 @@ public void writeEnoughRecordsToTriggerSortBufferExpansionAndSpill() throws Exce dataToWrite.add(new Tuple2(i, i)); } Seq files = writer.write(dataToWrite.iterator()); - // this includes the tmp index & data files, before the output is committed - assertEquals(4, tmpShuffleFilesCreated.size()); + assertEquals(2, spillFilesCreated.size()); MapStatus mapStatus = writer.stop(true).get(); ShuffleOutputCoordinator.commitOutputs(0, 0, files, mapStatus, mapStatusFile, serializer.newInstance()); 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 b5704eb58edfd..766f8fb915628 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 @@ -78,13 +78,6 @@ class BypassMergeSortShuffleWriterSuite extends SparkFunSuite with BeforeAndAfte when(blockResolver.getDataFile(0, 0)).thenReturn(outputFile) when(blockResolver.getIndexFile(0, 0)).thenReturn(indexFile) // the index file will be empty, but that is fine for these tests - when(blockResolver.writeIndexFile(anyInt(), anyInt(), any())).thenAnswer(new Answer[File] { - override def answer(invocationOnMock: InvocationOnMock): File = { - val f = diskBlockManager.createTempShuffleBlock()._2 - f.createNewFile() - f - } - }) when(blockManager.diskBlockManager).thenReturn(diskBlockManager) when(blockManager.getDiskWriter( any[BlockId], @@ -148,7 +141,7 @@ class BypassMergeSortShuffleWriterSuite extends SparkFunSuite with BeforeAndAfte assert(writer.getPartitionLengths.sum === 0) assert(outputFile.exists()) assert(outputFile.length() === 0) - assert(temporaryFilesCreated.size === 2) + assert(temporaryFilesCreated.isEmpty) val shuffleWriteMetrics = taskContext.taskMetrics().shuffleWriteMetrics.get assert(shuffleWriteMetrics.shuffleBytesWritten === 0) assert(shuffleWriteMetrics.shuffleRecordsWritten === 0) From 9d0d9d968dc3830a57490a47f84eaf6225e99396 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Tue, 10 Nov 2015 21:52:48 -0600 Subject: [PATCH 32/33] no more @VisibleForTesting --- .../org/apache/spark/shuffle/ShuffleOutputCoordinator.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/shuffle/ShuffleOutputCoordinator.scala b/core/src/main/scala/org/apache/spark/shuffle/ShuffleOutputCoordinator.scala index 0d0d5f2247174..abc58a5fc3543 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/ShuffleOutputCoordinator.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/ShuffleOutputCoordinator.scala @@ -61,7 +61,7 @@ private[spark] object ShuffleOutputCoordinator extends Logging { commitOutputs(shuffleId, partitionId, tmpToDest, mapStatus, mapStatusFile, ser) } - @VisibleForTesting + /** Exposed for testing. */ def commitOutputs( shuffleId: Int, partitionId: Int, From 80e037dcb731a918d516eb02c12440d8ba3e71a7 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Tue, 10 Nov 2015 21:59:57 -0600 Subject: [PATCH 33/33] unused import --- .../org/apache/spark/shuffle/ShuffleOutputCoordinator.scala | 2 -- 1 file changed, 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/shuffle/ShuffleOutputCoordinator.scala b/core/src/main/scala/org/apache/spark/shuffle/ShuffleOutputCoordinator.scala index abc58a5fc3543..0cc75f5e2dee4 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/ShuffleOutputCoordinator.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/ShuffleOutputCoordinator.scala @@ -18,8 +18,6 @@ package org.apache.spark.shuffle import java.io.{File, FileInputStream, FileOutputStream, IOException} -import com.google.common.annotations.VisibleForTesting - import org.apache.spark.{Logging, SparkEnv} import org.apache.spark.scheduler.MapStatus import org.apache.spark.serializer.SerializerInstance