From d08c20cd1fbb22bb5db191db3d4616e5ed8b6f52 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Wed, 6 May 2015 19:49:27 -0500 Subject: [PATCH 01/73] tasks know which stageAttempt they belong to --- .../scala/org/apache/spark/scheduler/DAGScheduler.scala | 4 ++-- .../main/scala/org/apache/spark/scheduler/ResultTask.scala | 3 ++- .../scala/org/apache/spark/scheduler/ShuffleMapTask.scala | 5 +++-- core/src/main/scala/org/apache/spark/scheduler/Task.scala | 6 +++++- core/src/test/java/org/apache/spark/JavaAPISuite.java | 2 +- .../test/scala/org/apache/spark/scheduler/FakeTask.scala | 2 +- .../apache/spark/scheduler/NotSerializableFakeTask.scala | 2 +- .../scala/org/apache/spark/scheduler/TaskContextSuite.scala | 4 ++-- .../org/apache/spark/scheduler/TaskSetManagerSuite.scala | 2 +- 9 files changed, 18 insertions(+), 12 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index 5d812918a13d1..5388a5922a374 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -891,7 +891,7 @@ class DAGScheduler( partitionsToCompute.map { id => val locs = getPreferredLocs(stage.rdd, id) val part = stage.rdd.partitions(id) - new ShuffleMapTask(stage.id, taskBinary, part, locs) + new ShuffleMapTask(stage.id, stage.attemptId, taskBinary, part, locs) } case stage: ResultStage => @@ -900,7 +900,7 @@ class DAGScheduler( val p: Int = job.partitions(id) val part = stage.rdd.partitions(p) val locs = getPreferredLocs(stage.rdd, p) - new ResultTask(stage.id, taskBinary, part, locs, id) + new ResultTask(stage.id, stage.attemptId, taskBinary, part, locs, id) } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/ResultTask.scala b/core/src/main/scala/org/apache/spark/scheduler/ResultTask.scala index c9a124113961f..9c2606e278c54 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ResultTask.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ResultTask.scala @@ -41,11 +41,12 @@ import org.apache.spark.rdd.RDD */ private[spark] class ResultTask[T, U]( stageId: Int, + stageAttemptId: Int, taskBinary: Broadcast[Array[Byte]], partition: Partition, @transient locs: Seq[TaskLocation], val outputId: Int) - extends Task[U](stageId, partition.index) with Serializable { + extends Task[U](stageId, stageAttemptId, partition.index) with Serializable { @transient private[this] val preferredLocs: Seq[TaskLocation] = { if (locs == null) Nil else locs.toSet.toSeq 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 bd3dd23dfe1ac..14c8c00961487 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala @@ -40,14 +40,15 @@ import org.apache.spark.shuffle.ShuffleWriter */ private[spark] class ShuffleMapTask( stageId: Int, + stageAttemptId: Int, taskBinary: Broadcast[Array[Byte]], partition: Partition, @transient private var locs: Seq[TaskLocation]) - extends Task[MapStatus](stageId, partition.index) with Logging { + extends Task[MapStatus](stageId, stageAttemptId, partition.index) with Logging { /** A constructor used only in test suites. This does not require passing in an RDD. */ def this(partitionId: Int) { - this(0, null, new Partition { override def index: Int = 0 }, null) + this(0, 0, null, new Partition { override def index: Int = 0 }, null) } @transient private val preferredLocs: Seq[TaskLocation] = { diff --git a/core/src/main/scala/org/apache/spark/scheduler/Task.scala b/core/src/main/scala/org/apache/spark/scheduler/Task.scala index 586d1e06204c1..b8bd0a5c10370 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/Task.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/Task.scala @@ -43,7 +43,10 @@ import org.apache.spark.util.Utils * @param stageId id of the stage this task belongs to * @param partitionId index of the number in the RDD */ -private[spark] abstract class Task[T](val stageId: Int, var partitionId: Int) extends Serializable { +private[spark] abstract class Task[T]( + val stageId: Int, + val stageAttemptId: Int, + var partitionId: Int) extends Serializable { /** * Called by [[Executor]] to run this task. @@ -55,6 +58,7 @@ private[spark] abstract class Task[T](val stageId: Int, var partitionId: Int) ex final def run(taskAttemptId: Long, attemptNumber: Int): T = { context = new TaskContextImpl( stageId = stageId, + stageAttemptId = stageAttemptId, partitionId = partitionId, taskAttemptId = taskAttemptId, attemptNumber = attemptNumber, diff --git a/core/src/test/java/org/apache/spark/JavaAPISuite.java b/core/src/test/java/org/apache/spark/JavaAPISuite.java index c2089b0e56a1f..8718f276a9245 100644 --- a/core/src/test/java/org/apache/spark/JavaAPISuite.java +++ b/core/src/test/java/org/apache/spark/JavaAPISuite.java @@ -1009,7 +1009,7 @@ public void persist() { @Test public void iterator() { JavaRDD rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5), 2); - TaskContext context = new TaskContextImpl(0, 0, 0L, 0, null, false, new TaskMetrics()); + TaskContext context = new TaskContextImpl(0, 0, 0L, 0, null, false, 0, new TaskMetrics()); Assert.assertEquals(1, rdd.iterator(rdd.partitions().get(0), context).next().intValue()); } diff --git a/core/src/test/scala/org/apache/spark/scheduler/FakeTask.scala b/core/src/test/scala/org/apache/spark/scheduler/FakeTask.scala index 0a7cb69416a08..188dded7c02f7 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/FakeTask.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/FakeTask.scala @@ -19,7 +19,7 @@ package org.apache.spark.scheduler import org.apache.spark.TaskContext -class FakeTask(stageId: Int, prefLocs: Seq[TaskLocation] = Nil) extends Task[Int](stageId, 0) { +class FakeTask(stageId: Int, prefLocs: Seq[TaskLocation] = Nil) extends Task[Int](stageId, 0, 0) { override def runTask(context: TaskContext): Int = 0 override def preferredLocations: Seq[TaskLocation] = prefLocs diff --git a/core/src/test/scala/org/apache/spark/scheduler/NotSerializableFakeTask.scala b/core/src/test/scala/org/apache/spark/scheduler/NotSerializableFakeTask.scala index 9b92f8de56759..383855caefa2f 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/NotSerializableFakeTask.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/NotSerializableFakeTask.scala @@ -25,7 +25,7 @@ import org.apache.spark.TaskContext * A Task implementation that fails to serialize. */ private[spark] class NotSerializableFakeTask(myId: Int, stageId: Int) - extends Task[Array[Byte]](stageId, 0) { + extends Task[Array[Byte]](stageId, 0, 0) { override def runTask(context: TaskContext): Array[Byte] = Array.empty[Byte] override def preferredLocations: Seq[TaskLocation] = Seq[TaskLocation]() diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskContextSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskContextSuite.scala index 83ae8701243e5..34602a95e53ec 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskContextSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskContextSuite.scala @@ -42,8 +42,8 @@ class TaskContextSuite extends FunSuite with BeforeAndAfter with LocalSparkConte } val closureSerializer = SparkEnv.get.closureSerializer.newInstance() val func = (c: TaskContext, i: Iterator[String]) => i.next() - val task = new ResultTask[String, String]( - 0, sc.broadcast(closureSerializer.serialize((rdd, func)).array), rdd.partitions(0), Seq(), 0) + val task = new ResultTask[String, String](0, 0, + sc.broadcast(closureSerializer.serialize((rdd, func)).array), rdd.partitions(0), Seq(), 0) intercept[RuntimeException] { task.run(0, 0) } diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala index 6198cea46ddf8..65688124607a7 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala @@ -137,7 +137,7 @@ class FakeTaskScheduler(sc: SparkContext, liveExecutors: (String, String)* /* ex /** * A Task implementation that results in a large serialized task. */ -class LargeTask(stageId: Int) extends Task[Array[Byte]](stageId, 0) { +class LargeTask(stageId: Int) extends Task[Array[Byte]](stageId, 0, 0) { val randomBuffer = new Array[Byte](TaskSetManager.TASK_SIZE_TO_WARN_KB * 1024) val random = new Random(0) random.nextBytes(randomBuffer) From 89e8428db2441258597e3962905da6317912cc12 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Wed, 6 May 2015 22:54:57 -0500 Subject: [PATCH 02/73] reproduce the failure --- .../org/apache/spark/TaskContextImpl.scala | 1 + .../spark/scheduler/DAGSchedulerSuite.scala | 59 +++++++++++++++++++ 2 files changed, 60 insertions(+) diff --git a/core/src/main/scala/org/apache/spark/TaskContextImpl.scala b/core/src/main/scala/org/apache/spark/TaskContextImpl.scala index b4d572cb52313..077fa038bbe69 100644 --- a/core/src/main/scala/org/apache/spark/TaskContextImpl.scala +++ b/core/src/main/scala/org/apache/spark/TaskContextImpl.scala @@ -30,6 +30,7 @@ private[spark] class TaskContextImpl( override val attemptNumber: Int, override val taskMemoryManager: TaskMemoryManager, val runningLocally: Boolean = false, + val stageAttemptId: Int = 0, // for testing val taskMetrics: TaskMetrics = TaskMetrics.empty) extends TaskContext with Logging { diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala index 6a8ae29aae675..433a5189b22d2 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -28,6 +28,7 @@ import org.scalatest.time.SpanSugar._ import org.apache.spark._ import org.apache.spark.rdd.RDD import org.apache.spark.scheduler.SchedulingMode.SchedulingMode +import org.apache.spark.shuffle.FetchFailedException import org.apache.spark.storage.{BlockId, BlockManagerId, BlockManagerMaster} import org.apache.spark.util.CallSite import org.apache.spark.executor.TaskMetrics @@ -773,6 +774,64 @@ class DAGSchedulerSuite assertDataStructuresEmpty() } + test("no concurrent retries for stage attempts (SPARK-7308)") { + // see SPARK-7308 for a detailed description of the conditions this is trying to recreate. + // note that this is somewhat convoluted for a test case, but isn't actually very unusual + // under a real workload. Note that we only fail the first attempt of stage 2, but that + // could be enough to cause havoc. + + val conf = new SparkConf().set("spark.executor.memory", "100m") + val clusterSc = new SparkContext("local-cluster[10,4,100]", "test-cluster", conf) + val bms = ArrayBuffer[BlockManagerId]() + val stageFailureCount = HashMap[Int, Int]() + clusterSc.addSparkListener(new SparkListener { + override def onBlockManagerAdded(blockManagerAdded: SparkListenerBlockManagerAdded): Unit = { + bms += blockManagerAdded.blockManagerId + } + override def onStageCompleted(stageCompleted: SparkListenerStageCompleted): Unit = { + if (stageCompleted.stageInfo.failureReason.isDefined) { + val stage = stageCompleted.stageInfo.stageId + stageFailureCount(stage) = stageFailureCount.getOrElse(stage, 0) + 1 + } + } + }) + try { + val rawData = clusterSc.parallelize(1 to 1e6.toInt, 500).map{x => (x % 100) -> x}.cache() + rawData.count() + val aBm = bms(0) + val shuffled = rawData.groupByKey(100).mapPartitionsWithIndex{ case (idx, itr) => + // we want one failure quickly, and more failures after stage 0 has finished its + // second attempt + if (TaskContext.get().asInstanceOf[TaskContextImpl].stageAttemptId == 0) { + if (idx == 0) { + throw new FetchFailedException(aBm, 0, 0, idx, cause = new RuntimeException("simulated fetch failure")) + } else if (idx > 0 && math.random < 0.1) { + Thread.sleep(10000) + throw new FetchFailedException(aBm, 0, 0, idx, cause = new RuntimeException("simulated fetch failure")) + } + } else { + Thread.sleep(10000) + } + Thread.sleep(500) // want to make sure plenty of these finish after task 0 fails + itr.map{x => ((x._1 + 5) % 100) -> x._2 } + } + val shuffledAgain = shuffled.flatMap{ case(k,vs) => vs.map{k -> _}}.groupByKey(100) + val data = shuffledAgain.mapPartitions { itr => + Thread.sleep(10000) + itr.flatMap{_._2} + }.cache().collect() + val count = data.size + assert(count === 1e6.toInt) + assert(data.toSet === (1 to 1e6.toInt).toSet) + // we should only get one failure from stage 2, everything else should be fine + assert(stageFailureCount(2) === 1) + assert(stageFailureCount.getOrElse(1, 0) === 0) + assert(stageFailureCount.getOrElse(3, 0) === 0) + } finally { + clusterSc.stop() + } + } + /** * Assert that the supplied TaskSet has exactly the given hosts as its preferred locations. * Note that this checks only the host and not the executor ID. From 70a787be6e55605365d84490e0d2072d4c7f5143 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Wed, 6 May 2015 23:13:23 -0500 Subject: [PATCH 03/73] ignore fetch failure from attempts that are already failed. only a partial fix, still have some concurrent attempts --- .../org/apache/spark/scheduler/DAGScheduler.scala | 11 ++++++++--- .../apache/spark/scheduler/DAGSchedulerSuite.scala | 2 +- 2 files changed, 9 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index 5388a5922a374..d02a4bf35bb3b 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -1103,9 +1103,14 @@ class DAGScheduler( // multiple tasks running concurrently on different executors). In that case, it is possible // the fetch failure has already been handled by the scheduler. if (runningStages.contains(failedStage)) { - logInfo(s"Marking $failedStage (${failedStage.name}) as failed " + - s"due to a fetch failure from $mapStage (${mapStage.name})") - markStageAsFinished(failedStage, Some(failureMessage)) + if (failedStage.attemptId - 1 > task.stageAttemptId) { + logInfo(s"Ignoring fetch failure from $task as it's from $failedStage attempt" + + s" ${task.stageAttemptId}, which has already failed") + } else { + logInfo(s"Marking $failedStage (${failedStage.name}) as failed " + + s"due to a fetch failure from $mapStage (${mapStage.name})") + markStageAsFinished(failedStage, Some(failureMessage)) + } } if (disallowStageRetryForTest) { diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala index 433a5189b22d2..7d11c8e316d5b 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -826,7 +826,7 @@ class DAGSchedulerSuite // we should only get one failure from stage 2, everything else should be fine assert(stageFailureCount(2) === 1) assert(stageFailureCount.getOrElse(1, 0) === 0) - assert(stageFailureCount.getOrElse(3, 0) === 0) + assert(stageFailureCount.getOrElse(3, 0) <= 2) // TODO this should be 0, bug still exists } finally { clusterSc.stop() } From 7fbcefbdb466daca0f492966492e4d7710247810 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Wed, 6 May 2015 23:15:11 -0500 Subject: [PATCH 04/73] ignore the test for now just to avoid swamping jenkins --- .../scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala index 7d11c8e316d5b..3f7d5b6d90d3a 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -774,7 +774,7 @@ class DAGSchedulerSuite assertDataStructuresEmpty() } - test("no concurrent retries for stage attempts (SPARK-7308)") { + ignore("no concurrent retries for stage attempts (SPARK-7308)") { // see SPARK-7308 for a detailed description of the conditions this is trying to recreate. // note that this is somewhat convoluted for a test case, but isn't actually very unusual // under a real workload. Note that we only fail the first attempt of stage 2, but that From 2eebbf214bb534c5bd7ebaa9de2be7b3471a497b Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Wed, 6 May 2015 23:50:44 -0500 Subject: [PATCH 05/73] style --- .../org/apache/spark/scheduler/DAGSchedulerSuite.scala | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala index 3f7d5b6d90d3a..68ec1ed0f24a6 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -804,10 +804,12 @@ class DAGSchedulerSuite // second attempt if (TaskContext.get().asInstanceOf[TaskContextImpl].stageAttemptId == 0) { if (idx == 0) { - throw new FetchFailedException(aBm, 0, 0, idx, cause = new RuntimeException("simulated fetch failure")) + throw new FetchFailedException(aBm, 0, 0, idx, + cause = new RuntimeException("simulated fetch failure")) } else if (idx > 0 && math.random < 0.1) { Thread.sleep(10000) - throw new FetchFailedException(aBm, 0, 0, idx, cause = new RuntimeException("simulated fetch failure")) + throw new FetchFailedException(aBm, 0, 0, idx, + cause = new RuntimeException("simulated fetch failure")) } } else { Thread.sleep(10000) From 7142242547074604c1a7ef9dd701473dd40d4693 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Tue, 12 May 2015 10:52:30 -0500 Subject: [PATCH 06/73] more rigorous test case --- .../org/apache/spark/scheduler/DAGSchedulerSuite.scala | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala index 68ec1ed0f24a6..b5bee5d9760a3 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -814,7 +814,9 @@ class DAGSchedulerSuite } else { Thread.sleep(10000) } - Thread.sleep(500) // want to make sure plenty of these finish after task 0 fails + // want to make sure plenty of these finish after task 0 fails, and some even finish + // after the previous stage is retried and this stage retry is started + Thread.sleep((500 + math.random * 5000).toLong) itr.map{x => ((x._1 + 5) % 100) -> x._2 } } val shuffledAgain = shuffled.flatMap{ case(k,vs) => vs.map{k -> _}}.groupByKey(100) @@ -828,7 +830,7 @@ class DAGSchedulerSuite // we should only get one failure from stage 2, everything else should be fine assert(stageFailureCount(2) === 1) assert(stageFailureCount.getOrElse(1, 0) === 0) - assert(stageFailureCount.getOrElse(3, 0) <= 2) // TODO this should be 0, bug still exists + assert(stageFailureCount.getOrElse(3, 0) == 0) } finally { clusterSc.stop() } From ccaa159a0d1a449acf271daa028a53228a879a4c Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Tue, 12 May 2015 10:53:09 -0500 Subject: [PATCH 07/73] index file needs to handle cases when data file already exist, and the actual data is in the middle of it --- .../apache/spark/shuffle/IndexShuffleBlockManager.scala | 8 ++++++-- .../org/apache/spark/shuffle/sort/SortShuffleWriter.scala | 3 ++- 2 files changed, 8 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockManager.scala b/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockManager.scala index a1741e2875c16..6ec29296ebca3 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockManager.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockManager.scala @@ -76,12 +76,16 @@ class IndexShuffleBlockManager(conf: SparkConf) extends ShuffleBlockResolver { * end of the output file. This will be used by getBlockLocation to figure out where each block * begins and ends. * */ - def writeIndexFile(shuffleId: Int, mapId: Int, lengths: Array[Long]): Unit = { + def writeIndexFile( + shuffleId: Int, + mapId: Int, + lengths: Array[Long], + initialFileLength: Long): Unit = { val indexFile = getIndexFile(shuffleId, mapId) val out = new DataOutputStream(new BufferedOutputStream(new FileOutputStream(indexFile))) Utils.tryWithSafeFinally { // We take in lengths of each block, need to convert it to offsets. - var offset = 0L + var offset = initialFileLength out.writeLong(offset) for (length <- lengths) { offset += length 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 a066435df6fb0..bb4db87b79cff 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 @@ -66,9 +66,10 @@ private[spark] class SortShuffleWriter[K, V, C]( // because it just opens a single file, so is typically too fast to measure accurately // (see SPARK-3570). val outputFile = shuffleBlockManager.getDataFile(dep.shuffleId, mapId) + val initialFileLength = outputFile.length() val blockId = ShuffleBlockId(dep.shuffleId, mapId, IndexShuffleBlockManager.NOOP_REDUCE_ID) val partitionLengths = sorter.writePartitionedFile(blockId, context, outputFile) - shuffleBlockManager.writeIndexFile(dep.shuffleId, mapId, partitionLengths) + shuffleBlockManager.writeIndexFile(dep.shuffleId, mapId, partitionLengths, initialFileLength) mapStatus = MapStatus(blockManager.shuffleServerId, partitionLengths) } From 3585b968f40817feb15c7056e70a4f83a6891012 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Tue, 12 May 2015 11:14:08 -0500 Subject: [PATCH 08/73] pare down the unit test --- .../spark/scheduler/DAGSchedulerSuite.scala | 21 ++++++++----------- 1 file changed, 9 insertions(+), 12 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala index b5bee5d9760a3..c3d009f186eed 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -781,7 +781,7 @@ class DAGSchedulerSuite // could be enough to cause havoc. val conf = new SparkConf().set("spark.executor.memory", "100m") - val clusterSc = new SparkContext("local-cluster[10,4,100]", "test-cluster", conf) + val clusterSc = new SparkContext("local-cluster[5,4,100]", "test-cluster", conf) val bms = ArrayBuffer[BlockManagerId]() val stageFailureCount = HashMap[Int, Int]() clusterSc.addSparkListener(new SparkListener { @@ -792,11 +792,12 @@ class DAGSchedulerSuite if (stageCompleted.stageInfo.failureReason.isDefined) { val stage = stageCompleted.stageInfo.stageId stageFailureCount(stage) = stageFailureCount.getOrElse(stage, 0) + 1 + println("stage " + stage + " failed: " + stageFailureCount(stage)) } } }) try { - val rawData = clusterSc.parallelize(1 to 1e6.toInt, 500).map{x => (x % 100) -> x}.cache() + val rawData = clusterSc.parallelize(1 to 1e6.toInt, 20).map{x => (x % 100) -> x}.cache() rawData.count() val aBm = bms(0) val shuffled = rawData.groupByKey(100).mapPartitionsWithIndex{ case (idx, itr) => @@ -810,20 +811,16 @@ class DAGSchedulerSuite Thread.sleep(10000) throw new FetchFailedException(aBm, 0, 0, idx, cause = new RuntimeException("simulated fetch failure")) + } else { + // want to make sure plenty of these finish after task 0 fails, and some even finish + // after the previous stage is retried and this stage retry is started + Thread.sleep((500 + math.random * 5000).toLong) } - } else { - Thread.sleep(10000) } - // want to make sure plenty of these finish after task 0 fails, and some even finish - // after the previous stage is retried and this stage retry is started - Thread.sleep((500 + math.random * 5000).toLong) itr.map{x => ((x._1 + 5) % 100) -> x._2 } } - val shuffledAgain = shuffled.flatMap{ case(k,vs) => vs.map{k -> _}}.groupByKey(100) - val data = shuffledAgain.mapPartitions { itr => - Thread.sleep(10000) - itr.flatMap{_._2} - }.cache().collect() + val shuffledAgain = shuffled.flatMap{ case(k,vs) => vs.map(k -> _) }.groupByKey(100) + val data = shuffledAgain.mapPartitions { itr => itr.flatMap(_._2) }.cache().collect() val count = data.size assert(count === 1e6.toInt) assert(data.toSet === (1 to 1e6.toInt).toSet) From de235303c4e708323bbdcf2b6bbc003ad230fbe6 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Tue, 12 May 2015 11:27:49 -0500 Subject: [PATCH 09/73] SparkIllegalStateException if we ever have multiple concurrent attempts for the same stage --- .../main/scala/org/apache/spark/SparkException.scala | 10 ++++++++++ .../org/apache/spark/scheduler/TaskSchedulerImpl.scala | 6 ++++++ 2 files changed, 16 insertions(+) diff --git a/core/src/main/scala/org/apache/spark/SparkException.scala b/core/src/main/scala/org/apache/spark/SparkException.scala index 2ebd7a7151a59..b7c2386fd7d87 100644 --- a/core/src/main/scala/org/apache/spark/SparkException.scala +++ b/core/src/main/scala/org/apache/spark/SparkException.scala @@ -30,3 +30,13 @@ class SparkException(message: String, cause: Throwable) */ private[spark] class SparkDriverExecutionException(cause: Throwable) extends SparkException("Execution error", cause) + +/** + * Exception indicating an error internal to Spark -- it is in an inconsistent state, not due + * to any error by the user + */ +class SparkIllegalStateException(message: String, cause: Throwable) + extends SparkException(message, cause) { + + def this(message: String) = this(message, null) +} diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala index b4b8a630694bb..d37f02e51e3bf 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -159,6 +159,12 @@ private[spark] class TaskSchedulerImpl( this.synchronized { val manager = createTaskSetManager(taskSet, maxTaskFailures) activeTaskSets(taskSet.id) = manager + val taskSetsPerStage = activeTaskSets.values.filterNot(_.isZombie).groupBy(_.stageId) + taskSetsPerStage.foreach { case (stage, taskSets) => + if (taskSets.size > 1) { + throw new SparkIllegalStateException("more than one active taskSet for stage " + stage) + } + } schedulableBuilder.addTaskSetManager(manager, manager.taskSet.properties) if (!isLocal && !hasReceivedTask) { From c91ee10e166f07828bed66146a2bbdd28633fb34 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Tue, 12 May 2015 11:32:39 -0500 Subject: [PATCH 10/73] better unit test --- .../scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala index c3d009f186eed..712c68de107b7 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -807,8 +807,8 @@ class DAGSchedulerSuite if (idx == 0) { throw new FetchFailedException(aBm, 0, 0, idx, cause = new RuntimeException("simulated fetch failure")) - } else if (idx > 0 && math.random < 0.1) { - Thread.sleep(10000) + } else if (idx > 0 && math.random < 0.2) { + Thread.sleep(5000) throw new FetchFailedException(aBm, 0, 0, idx, cause = new RuntimeException("simulated fetch failure")) } else { From 05c72fda875a08ee3077ad232796cdf3d54d815f Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Tue, 12 May 2015 17:51:46 -0500 Subject: [PATCH 11/73] handle more cases from bad ordering of task attempt completion --- .../org/apache/spark/MapOutputTracker.scala | 8 +++-- .../apache/spark/scheduler/DAGScheduler.scala | 35 +++++++++++++++++-- .../spark/scheduler/ShuffleMapStage.scala | 20 +++++++++++ .../spark/scheduler/TaskSetManager.scala | 8 +++-- .../shuffle/IndexShuffleBlockManager.scala | 4 +-- .../shuffle/sort/SortShuffleWriter.scala | 2 ++ 6 files changed, 67 insertions(+), 10 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala index 16072283edbe9..1ecadd6258168 100644 --- a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala +++ b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala @@ -378,12 +378,14 @@ private[spark] object MapOutputTracker extends Logging { reduceId: Int, statuses: Array[MapStatus]): Array[(BlockManagerId, Long)] = { assert (statuses != null) + var idx = -1 statuses.map { status => + idx += 1 if (status == null) { - logError("Missing an output location for shuffle " + shuffleId) - throw new MetadataFetchFailedException( - shuffleId, reduceId, "Missing an output location for shuffle " + shuffleId) + val msg = "Missing an output location for shuffle " + shuffleId + ": map " + idx + logError(msg) + throw new MetadataFetchFailedException(shuffleId, reduceId, msg) } else { (status.location, status.getSizeForBlock(reduceId)) } diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index d02a4bf35bb3b..0ced368ec5718 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -830,6 +830,10 @@ class DAGScheduler( logDebug("submitMissingTasks(" + stage + ")") // Get our pending tasks and remember them in our pendingTasks entry stage.pendingTasks.clear() + stage match { + case smt: ShuffleMapStage => smt.clearPartitionComputeCount() + case _ => + } // First figure out the indexes of partition ids to compute. @@ -974,6 +978,7 @@ class DAGScheduler( val stageId = task.stageId val taskType = Utils.getFormattedClassName(task) + // REVIEWERS: does this need special handling for multiple completions of the same task? outputCommitCoordinator.taskCompleted(stageId, task.partitionId, event.taskInfo.attempt, event.reason) @@ -1031,15 +1036,31 @@ class DAGScheduler( case smt: ShuffleMapTask => val shuffleStage = stage.asInstanceOf[ShuffleMapStage] + val computeCount = shuffleStage.incComputeCount(smt.partitionId) updateAccumulators(event) val status = event.result.asInstanceOf[MapStatus] val execId = status.location.executorId logDebug("ShuffleMapTask finished on " + execId) - if (failedEpoch.contains(execId) && smt.epoch <= failedEpoch(execId)) { - logInfo("Ignoring possibly bogus ShuffleMapTask completion from " + execId) + if (computeCount > 1) { + // REVIEWERS: do I need to worry about speculation here, when multiple completion events + // are normal? + + // REVIEWERS: is this really only a problem on a ShuffleMapTask?? does it also cause + // problems for ResultTask? + + // This can happen when a retry runs a task, but there was a lingering task from an + // earlier attempt which also finished. The results might be OK, or they might not. + // To be safe, we'll retry the task, and do it in yet another attempt, to avoid more + // task output clobbering. + logInfo(s"Multiple completion events for task $task. Results may be corrupt," + + s" assuming task needs to be rerun.") + shuffleStage.removeOutputLoc(task.partitionId) + } else if (failedEpoch.contains(execId) && smt.epoch <= failedEpoch(execId)) { + logInfo(s"Ignoring possibly bogus $smt completion from executor $execId") } else { shuffleStage.addOutputLoc(smt.partitionId, status) } + if (runningStages.contains(shuffleStage) && shuffleStage.pendingTasks.isEmpty) { markStageAsFinished(shuffleStage) logInfo("looking for newly runnable stages") @@ -1133,6 +1154,16 @@ class DAGScheduler( mapOutputTracker.unregisterMapOutput(shuffleId, mapId, bmAddress) } + // We also have to mark this map output as unavailable. Its possible that a *later* attempt + // has finished this task in the meantime, but when this task fails, it might end up + // deleting the mapOutput from the earlier successful attempt. + failedStage match { + case smt: ShuffleMapStage => + smt.incComputeCount(reduceId) + smt.removeOutputLoc(reduceId) + case _ => + } + // TODO: mark the executor as failed only if there were lots of fetch failures on it if (bmAddress != null) { handleExecutorLost(bmAddress.executorId, fetchFailed = true, Some(task.epoch)) diff --git a/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapStage.scala b/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapStage.scala index d02210743484c..da3916430c0b2 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapStage.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapStage.scala @@ -17,6 +17,8 @@ package org.apache.spark.scheduler +import scala.collection.mutable.HashMap + import org.apache.spark.ShuffleDependency import org.apache.spark.rdd.RDD import org.apache.spark.storage.BlockManagerId @@ -43,6 +45,17 @@ private[spark] class ShuffleMapStage( val outputLocs = Array.fill[List[MapStatus]](numPartitions)(Nil) + private val partitionComputeCount = HashMap[Int, Int]() + + def incComputeCount(partition: Int): Int = { + partitionComputeCount(partition) = partitionComputeCount.getOrElse(partition, 0) + 1 + partitionComputeCount(partition) + } + + def clearPartitionComputeCount(): Unit = { + partitionComputeCount.clear() + } + def addOutputLoc(partition: Int, status: MapStatus): Unit = { val prevList = outputLocs(partition) outputLocs(partition) = status :: prevList @@ -51,6 +64,13 @@ private[spark] class ShuffleMapStage( } } + def removeOutputLoc(partition: Int): Unit = { + if (outputLocs(partition) != Nil) { + outputLocs(partition) = Nil + numAvailableOutputs -= 1 + } + } + def removeOutputLoc(partition: Int, bmAddress: BlockManagerId): Unit = { val prevList = outputLocs(partition) val newList = prevList.filterNot(_.location == bmAddress) 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 7dc325283d961..4ecf8a85e875e 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala @@ -620,12 +620,14 @@ private[spark] class TaskSetManager( val index = info.index info.markSuccessful() removeRunningTask(tid) + val task = tasks(index) 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 retry, the partition is *not* the same as info.id + logInfo("Finished task %s (partition %d) in stage %s (TID %d) in %d ms on executor %s (%s) (%d/%d)".format( + info.id, task.partitionId, taskSet.id, info.taskId, info.duration, info.executorId, 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/IndexShuffleBlockManager.scala b/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockManager.scala index 6ec29296ebca3..468aab2fe9867 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockManager.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockManager.scala @@ -22,7 +22,7 @@ import java.nio.ByteBuffer import com.google.common.io.ByteStreams -import org.apache.spark.{SparkConf, SparkEnv} +import org.apache.spark.{TaskContext, Logging, SparkConf, SparkEnv} import org.apache.spark.network.buffer.{FileSegmentManagedBuffer, ManagedBuffer} import org.apache.spark.network.netty.SparkTransportConf import org.apache.spark.storage._ @@ -42,7 +42,7 @@ import IndexShuffleBlockManager.NOOP_REDUCE_ID // Note: Changes to the format in this file should be kept in sync with // org.apache.spark.network.shuffle.StandaloneShuffleBlockManager#getSortBasedShuffleBlockData(). private[spark] -class IndexShuffleBlockManager(conf: SparkConf) extends ShuffleBlockResolver { +class IndexShuffleBlockManager(conf: SparkConf) extends ShuffleBlockResolver with Logging { private lazy val blockManager = SparkEnv.get.blockManager 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 bb4db87b79cff..b5838ffa30469 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 @@ -66,6 +66,8 @@ private[spark] class SortShuffleWriter[K, V, C]( // because it just opens a single file, so is typically too fast to measure accurately // (see SPARK-3570). val outputFile = shuffleBlockManager.getDataFile(dep.shuffleId, mapId) + // Because we append to the data file, we need the index file to know the current size of the + // data file as a starting point val initialFileLength = outputFile.length() val blockId = ShuffleBlockId(dep.shuffleId, mapId, IndexShuffleBlockManager.NOOP_REDUCE_ID) val partitionLengths = sorter.writePartitionedFile(blockId, context, outputFile) From 37eece86b0ef00fe6408963ffe5a8f40f7ec6990 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Tue, 12 May 2015 19:48:26 -0500 Subject: [PATCH 12/73] cleanup imports --- .../org/apache/spark/shuffle/IndexShuffleBlockResolver.scala | 2 +- .../scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala b/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala index 7564cc3586e0e..a6bd3020d147f 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala @@ -21,7 +21,7 @@ import java.io._ import com.google.common.io.ByteStreams -import org.apache.spark.{TaskContext, Logging, SparkConf, SparkEnv} +import org.apache.spark.{SparkConf, SparkEnv} import org.apache.spark.network.buffer.{FileSegmentManagedBuffer, ManagedBuffer} import org.apache.spark.network.netty.SparkTransportConf import org.apache.spark.storage._ 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 bdca30b8d04f2..c7c7ce6860514 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,7 +17,7 @@ package org.apache.spark.shuffle.sort -import org.apache.spark.{MapOutputTracker, SparkEnv, Logging, TaskContext} +import org.apache.spark.{SparkEnv, Logging, TaskContext} import org.apache.spark.executor.ShuffleWriteMetrics import org.apache.spark.scheduler.MapStatus import org.apache.spark.shuffle.{IndexShuffleBlockResolver, ShuffleWriter, BaseShuffleHandle} From 31c21fae51e1aa304e4e53c7ef36d1e1689abdee Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Tue, 12 May 2015 20:26:50 -0500 Subject: [PATCH 13/73] style --- .../org/apache/spark/scheduler/DAGScheduler.scala | 4 ++-- .../org/apache/spark/scheduler/TaskSetManager.scala | 5 +++-- .../apache/spark/scheduler/DAGSchedulerSuite.scala | 13 +++++++------ 3 files changed, 12 insertions(+), 10 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index 0ced368ec5718..bb634343049b4 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -1042,8 +1042,8 @@ class DAGScheduler( val execId = status.location.executorId logDebug("ShuffleMapTask finished on " + execId) if (computeCount > 1) { - // REVIEWERS: do I need to worry about speculation here, when multiple completion events - // are normal? + // REVIEWERS: do I need to worry about speculation here, when multiple completion + // events are normal? // REVIEWERS: is this really only a problem on a ShuffleMapTask?? does it also cause // problems for ResultTask? 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 4ecf8a85e875e..1168b6b0d1627 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala @@ -626,8 +626,9 @@ private[spark] class TaskSetManager( if (!successful(index)) { tasksSuccessful += 1 // include the partition here b/c on a retry, the partition is *not* the same as info.id - logInfo("Finished task %s (partition %d) in stage %s (TID %d) in %d ms on executor %s (%s) (%d/%d)".format( - info.id, task.partitionId, taskSet.id, info.taskId, info.duration, info.executorId, info.host, tasksSuccessful, numTasks)) + logInfo(("Finished task %s in stage %s (TID %d, partition %d) in %d ms on executor %s (%s) " + + "(%d/%d)").format(info.id, taskSet.id, task.partitionId, info.taskId, info.duration, + info.executorId, 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/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala index 712c68de107b7..b4b6d66b6cafb 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -785,9 +785,10 @@ class DAGSchedulerSuite val bms = ArrayBuffer[BlockManagerId]() val stageFailureCount = HashMap[Int, Int]() clusterSc.addSparkListener(new SparkListener { - override def onBlockManagerAdded(blockManagerAdded: SparkListenerBlockManagerAdded): Unit = { - bms += blockManagerAdded.blockManagerId + override def onBlockManagerAdded(bmAdded: SparkListenerBlockManagerAdded): Unit = { + bms += bmAdded.blockManagerId } + override def onStageCompleted(stageCompleted: SparkListenerStageCompleted): Unit = { if (stageCompleted.stageInfo.failureReason.isDefined) { val stage = stageCompleted.stageInfo.stageId @@ -797,10 +798,10 @@ class DAGSchedulerSuite } }) try { - val rawData = clusterSc.parallelize(1 to 1e6.toInt, 20).map{x => (x % 100) -> x}.cache() + val rawData = clusterSc.parallelize(1 to 1e6.toInt, 20).map { x => (x % 100) -> x }.cache() rawData.count() val aBm = bms(0) - val shuffled = rawData.groupByKey(100).mapPartitionsWithIndex{ case (idx, itr) => + val shuffled = rawData.groupByKey(100).mapPartitionsWithIndex { case (idx, itr) => // we want one failure quickly, and more failures after stage 0 has finished its // second attempt if (TaskContext.get().asInstanceOf[TaskContextImpl].stageAttemptId == 0) { @@ -817,9 +818,9 @@ class DAGSchedulerSuite Thread.sleep((500 + math.random * 5000).toLong) } } - itr.map{x => ((x._1 + 5) % 100) -> x._2 } + itr.map { x => ((x._1 + 5) % 100) -> x._2 } } - val shuffledAgain = shuffled.flatMap{ case(k,vs) => vs.map(k -> _) }.groupByKey(100) + val shuffledAgain = shuffled.flatMap { case (k, vs) => vs.map(k -> _) }.groupByKey(100) val data = shuffledAgain.mapPartitions { itr => itr.flatMap(_._2) }.cache().collect() val count = data.size assert(count === 1e6.toInt) From a894be11d2d22841017d82676a66088f0d053dcf Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Wed, 13 May 2015 08:42:05 -0500 Subject: [PATCH 14/73] include all missing mapIds in error msg --- core/src/main/scala/org/apache/spark/MapOutputTracker.scala | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala index 663e434a1c1dc..3f77765d132c7 100644 --- a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala +++ b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala @@ -382,12 +382,11 @@ private[spark] object MapOutputTracker extends Logging { reduceId: Int, statuses: Array[MapStatus]): Array[(BlockManagerId, Long)] = { assert (statuses != null) - var idx = -1 statuses.map { status => - idx += 1 if (status == null) { - val msg = "Missing an output location for shuffle " + shuffleId + ": map " + idx + val missing = statuses.iterator.zipWithIndex.filter{_._1 == null}.map{_._2}.mkString(",") + val msg = "Missing an output location for shuffle =" + shuffleId + "; mapIds =" + missing logError(msg) throw new MetadataFetchFailedException(shuffleId, reduceId, msg) } else { From 93592b1554ed8091c93a3f58cfd2951cb4cac088 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Wed, 13 May 2015 08:44:38 -0500 Subject: [PATCH 15/73] update existing test since we now do more resubmitting than before --- .../spark/scheduler/DAGSchedulerSuite.scala | 16 ++++++++++++++-- 1 file changed, 14 insertions(+), 2 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala index b4b6d66b6cafb..d975d62f5dea7 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -536,7 +536,7 @@ class DAGSchedulerSuite // should be ignored for being too old runEvent(CompletionEvent( taskSet.tasks(0), Success, makeMapStatus("hostA", 1), null, createFakeTaskInfo(), null)) - // should work because it's a non-failed host + // its a non-failed host, but we can't be sure if the results were clobbered runEvent(CompletionEvent( taskSet.tasks(0), Success, makeMapStatus("hostB", 1), null, createFakeTaskInfo(), null)) // should be ignored for being too old @@ -546,9 +546,21 @@ class DAGSchedulerSuite taskSet.tasks(1).epoch = newEpoch runEvent(CompletionEvent( taskSet.tasks(1), Success, makeMapStatus("hostA", 1), null, createFakeTaskInfo(), null)) + + //now we should have a new taskSet for stage 0, which has us retry partition 0 + assert(taskSets.size === 2) + val newTaskSet = taskSets(1) + assert(newTaskSet.stageId === 0) + assert(newTaskSet.attempt === 1) + assert(newTaskSet.tasks.size === 1) + val newTask = newTaskSet.tasks(0) + assert(newTask.epoch === newEpoch + 1) + runEvent(CompletionEvent( + newTask, Success, makeMapStatus("hostB", 1), null, createFakeTaskInfo(), null)) + assert(mapOutputTracker.getServerStatuses(shuffleId, 0).map(_._1) === Array(makeBlockManagerId("hostB"), makeBlockManagerId("hostA"))) - complete(taskSets(1), Seq((Success, 42), (Success, 43))) + complete(taskSets(2), Seq((Success, 42), (Success, 43))) assert(results === Map(0 -> 42, 1 -> 43)) assertDataStructuresEmpty() } From ea2d9720b26e687ddde777a83c6a6b11db663af2 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Wed, 13 May 2015 12:54:52 -0500 Subject: [PATCH 16/73] style --- .../scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala index d975d62f5dea7..29056eff0e57e 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -547,7 +547,7 @@ class DAGSchedulerSuite runEvent(CompletionEvent( taskSet.tasks(1), Success, makeMapStatus("hostA", 1), null, createFakeTaskInfo(), null)) - //now we should have a new taskSet for stage 0, which has us retry partition 0 + // now we should have a new taskSet for stage 0, which has us retry partition 0 assert(taskSets.size === 2) val newTaskSet = taskSets(1) assert(newTaskSet.stageId === 0) From 6654c538cf83126d9ae0de7e39cc19829bfa6fba Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Thu, 21 May 2015 15:02:52 -0500 Subject: [PATCH 17/73] fixes from merge --- .../org/apache/spark/shuffle/unsafe/UnsafeShuffleWriter.java | 4 +++- .../apache/spark/shuffle/unsafe/UnsafeShuffleWriterSuite.java | 2 +- 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/core/src/main/java/org/apache/spark/shuffle/unsafe/UnsafeShuffleWriter.java b/core/src/main/java/org/apache/spark/shuffle/unsafe/UnsafeShuffleWriter.java index ad7eb04afcd8c..746676c7aa583 100644 --- a/core/src/main/java/org/apache/spark/shuffle/unsafe/UnsafeShuffleWriter.java +++ b/core/src/main/java/org/apache/spark/shuffle/unsafe/UnsafeShuffleWriter.java @@ -175,6 +175,8 @@ void closeAndWriteOutput() throws IOException { final SpillInfo[] spills = sorter.closeAndGetSpills(); sorter = null; final long[] partitionLengths; + final File outputFile = shuffleBlockResolver.getDataFile(shuffleId, mapId); + final long initialFileLength = outputFile.length(); try { partitionLengths = mergeSpills(spills); } finally { @@ -184,7 +186,7 @@ void closeAndWriteOutput() throws IOException { } } } - shuffleBlockResolver.writeIndexFile(shuffleId, mapId, partitionLengths); + shuffleBlockResolver.writeIndexFile(shuffleId, mapId, partitionLengths, initialFileLength); mapStatus = MapStatus$.MODULE$.apply(blockManager.shuffleServerId(), partitionLengths); } diff --git a/core/src/test/java/org/apache/spark/shuffle/unsafe/UnsafeShuffleWriterSuite.java b/core/src/test/java/org/apache/spark/shuffle/unsafe/UnsafeShuffleWriterSuite.java index 83d109115aa5c..17dffe505439d 100644 --- a/core/src/test/java/org/apache/spark/shuffle/unsafe/UnsafeShuffleWriterSuite.java +++ b/core/src/test/java/org/apache/spark/shuffle/unsafe/UnsafeShuffleWriterSuite.java @@ -175,7 +175,7 @@ public Void answer(InvocationOnMock invocationOnMock) throws Throwable { partitionSizesInMergedFile = (long[]) invocationOnMock.getArguments()[2]; return null; } - }).when(shuffleBlockResolver).writeIndexFile(anyInt(), anyInt(), any(long[].class)); + }).when(shuffleBlockResolver).writeIndexFile(anyInt(), anyInt(), any(long[].class), 0L); when(diskBlockManager.createTempShuffleBlock()).thenAnswer( new Answer>() { From dd2839d1aceef3ca038c0af5cb2b69955d10cc8f Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Thu, 21 May 2015 16:54:31 -0500 Subject: [PATCH 18/73] better fix from merge --- .../apache/spark/shuffle/unsafe/UnsafeShuffleWriterSuite.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/test/java/org/apache/spark/shuffle/unsafe/UnsafeShuffleWriterSuite.java b/core/src/test/java/org/apache/spark/shuffle/unsafe/UnsafeShuffleWriterSuite.java index 17dffe505439d..ffbe8d37d025b 100644 --- a/core/src/test/java/org/apache/spark/shuffle/unsafe/UnsafeShuffleWriterSuite.java +++ b/core/src/test/java/org/apache/spark/shuffle/unsafe/UnsafeShuffleWriterSuite.java @@ -175,7 +175,7 @@ public Void answer(InvocationOnMock invocationOnMock) throws Throwable { partitionSizesInMergedFile = (long[]) invocationOnMock.getArguments()[2]; return null; } - }).when(shuffleBlockResolver).writeIndexFile(anyInt(), anyInt(), any(long[].class), 0L); + }).when(shuffleBlockResolver).writeIndexFile(anyInt(), anyInt(), any(long[].class), eq(0L)); when(diskBlockManager.createTempShuffleBlock()).thenAnswer( new Answer>() { From e68492812d3384baefe29efa0a32fcca1fab12e6 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Fri, 29 May 2015 13:13:45 -0500 Subject: [PATCH 19/73] shuffle map output writes to a different file per attempt (main compiles, tests do not) --- .../shuffle/unsafe/UnsafeShuffleWriter.java | 14 ++++--- .../org/apache/spark/MapOutputTracker.scala | 6 +-- .../apache/spark/scheduler/MapStatus.scala | 34 ++++++++++++----- .../spark/scheduler/ShuffleMapTask.scala | 2 +- .../spark/shuffle/FetchFailedException.scala | 6 ++- .../shuffle/FileShuffleBlockResolver.scala | 16 ++++---- .../shuffle/IndexShuffleBlockResolver.scala | 28 +++++++------- .../spark/shuffle/ShuffleBlockResolver.scala | 2 +- .../apache/spark/shuffle/ShuffleManager.scala | 20 +++++++++- .../hash/BlockStoreShuffleFetcher.scala | 13 ++++--- .../shuffle/hash/HashShuffleManager.scala | 16 +++++--- .../shuffle/hash/HashShuffleWriter.scala | 7 ++-- .../shuffle/sort/SortShuffleManager.scala | 15 ++++++-- .../shuffle/sort/SortShuffleWriter.scala | 12 +++--- .../shuffle/unsafe/UnsafeShuffleManager.scala | 10 ++++- .../org/apache/spark/storage/BlockId.scala | 38 +++++++++++-------- .../unsafe/UnsafeShuffleWriterSuite.java | 1 + 17 files changed, 157 insertions(+), 83 deletions(-) diff --git a/core/src/main/java/org/apache/spark/shuffle/unsafe/UnsafeShuffleWriter.java b/core/src/main/java/org/apache/spark/shuffle/unsafe/UnsafeShuffleWriter.java index 746676c7aa583..1c57d600d217d 100644 --- a/core/src/main/java/org/apache/spark/shuffle/unsafe/UnsafeShuffleWriter.java +++ b/core/src/main/java/org/apache/spark/shuffle/unsafe/UnsafeShuffleWriter.java @@ -74,6 +74,7 @@ public class UnsafeShuffleWriter extends ShuffleWriter { private final ShuffleWriteMetrics writeMetrics; private final int shuffleId; private final int mapId; + private final int stageAttemptId; private final TaskContext taskContext; private final SparkConf sparkConf; private final boolean transferToEnabled; @@ -104,6 +105,7 @@ public UnsafeShuffleWriter( ShuffleMemoryManager shuffleMemoryManager, UnsafeShuffleHandle handle, int mapId, + int stageAttemptId, TaskContext taskContext, SparkConf sparkConf) throws IOException { final int numPartitions = handle.dependency().partitioner().numPartitions(); @@ -117,6 +119,7 @@ public UnsafeShuffleWriter( this.memoryManager = memoryManager; this.shuffleMemoryManager = shuffleMemoryManager; this.mapId = mapId; + this.stageAttemptId = stageAttemptId; final ShuffleDependency dep = handle.dependency(); this.shuffleId = dep.shuffleId(); this.serializer = Serializer.getSerializer(dep.serializer()).newInstance(); @@ -175,8 +178,6 @@ void closeAndWriteOutput() throws IOException { final SpillInfo[] spills = sorter.closeAndGetSpills(); sorter = null; final long[] partitionLengths; - final File outputFile = shuffleBlockResolver.getDataFile(shuffleId, mapId); - final long initialFileLength = outputFile.length(); try { partitionLengths = mergeSpills(spills); } finally { @@ -186,8 +187,9 @@ void closeAndWriteOutput() throws IOException { } } } - shuffleBlockResolver.writeIndexFile(shuffleId, mapId, partitionLengths, initialFileLength); - mapStatus = MapStatus$.MODULE$.apply(blockManager.shuffleServerId(), partitionLengths); + shuffleBlockResolver.writeIndexFile(shuffleId, mapId, stageAttemptId, partitionLengths); + mapStatus = MapStatus$.MODULE$.apply( + blockManager.shuffleServerId(), stageAttemptId, partitionLengths); } @VisibleForTesting @@ -219,7 +221,7 @@ 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); + final File outputFile = shuffleBlockResolver.getDataFile(shuffleId, mapId, stageAttemptId); final boolean compressionEnabled = sparkConf.getBoolean("spark.shuffle.compress", true); final CompressionCodec compressionCodec = CompressionCodec$.MODULE$.createCodec(sparkConf); final boolean fastMergeEnabled = @@ -425,7 +427,7 @@ public Option stop(boolean success) { return Option.apply(mapStatus); } else { // The map task failed, so delete our output data. - shuffleBlockResolver.removeDataByMap(shuffleId, mapId); + shuffleBlockResolver.removeDataByMap(shuffleId, mapId, stageAttemptId); return Option.apply(null); } } diff --git a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala index 3f77765d132c7..d439c42ce890f 100644 --- a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala +++ b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala @@ -127,7 +127,7 @@ private[spark] abstract class MapOutputTracker(conf: SparkConf) extends Logging * Called from executors to get the server URIs and output sizes of the map outputs of * a given shuffle. */ - def getServerStatuses(shuffleId: Int, reduceId: Int): Array[(BlockManagerId, Long)] = { + def getServerStatuses(shuffleId: Int, reduceId: Int): Array[(BlockManagerId, Int, Long)] = { val statuses = mapStatuses.get(shuffleId).orNull if (statuses == null) { logInfo("Don't have map outputs for shuffle " + shuffleId + ", fetching them") @@ -380,7 +380,7 @@ private[spark] object MapOutputTracker extends Logging { private def convertMapStatuses( shuffleId: Int, reduceId: Int, - statuses: Array[MapStatus]): Array[(BlockManagerId, Long)] = { + statuses: Array[MapStatus]): Array[(BlockManagerId, Int, Long)] = { assert (statuses != null) statuses.map { status => @@ -390,7 +390,7 @@ private[spark] object MapOutputTracker extends Logging { logError(msg) throw new MetadataFetchFailedException(shuffleId, reduceId, msg) } else { - (status.location, status.getSizeForBlock(reduceId)) + (status.location, status.stageAttemptId, status.getSizeForBlock(reduceId)) } } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/MapStatus.scala b/core/src/main/scala/org/apache/spark/scheduler/MapStatus.scala index 1efce124c0a6b..a5cb9d3d39e8c 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/MapStatus.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/MapStatus.scala @@ -32,6 +32,9 @@ private[spark] sealed trait MapStatus { /** Location where this task was run. */ def location: BlockManagerId + /** stage attempt for the ShuffleMapTask */ + def stageAttemptId: Int + /** * Estimated size for the reduce block, in bytes. * @@ -44,11 +47,11 @@ private[spark] sealed trait MapStatus { private[spark] object MapStatus { - def apply(loc: BlockManagerId, uncompressedSizes: Array[Long]): MapStatus = { + def apply(loc: BlockManagerId, stageAttemptId: Int, uncompressedSizes: Array[Long]): MapStatus = { if (uncompressedSizes.length > 2000) { - HighlyCompressedMapStatus(loc, uncompressedSizes) + HighlyCompressedMapStatus(loc, stageAttemptId, uncompressedSizes) } else { - new CompressedMapStatus(loc, uncompressedSizes) + new CompressedMapStatus(loc, stageAttemptId, uncompressedSizes) } } @@ -91,29 +94,34 @@ private[spark] object MapStatus { */ private[spark] class CompressedMapStatus( private[this] var loc: BlockManagerId, + private[this] var _stageAttemptId: Int, private[this] var compressedSizes: Array[Byte]) extends MapStatus with Externalizable { - protected def this() = this(null, null.asInstanceOf[Array[Byte]]) // For deserialization only + protected def this() = this(null, 0, null.asInstanceOf[Array[Byte]]) // For deserialization only - def this(loc: BlockManagerId, uncompressedSizes: Array[Long]) { - this(loc, uncompressedSizes.map(MapStatus.compressSize)) + def this(loc: BlockManagerId, stageAttemptId: Int, uncompressedSizes: Array[Long]) { + this(loc, stageAttemptId, uncompressedSizes.map(MapStatus.compressSize)) } override def location: BlockManagerId = loc + override def stageAttemptId = _stageAttemptId + override def getSizeForBlock(reduceId: Int): Long = { MapStatus.decompressSize(compressedSizes(reduceId)) } override def writeExternal(out: ObjectOutput): Unit = Utils.tryOrIOException { loc.writeExternal(out) + out.writeInt(_stageAttemptId) out.writeInt(compressedSizes.length) out.write(compressedSizes) } override def readExternal(in: ObjectInput): Unit = Utils.tryOrIOException { loc = BlockManagerId(in) + _stageAttemptId = in.readInt() val len = in.readInt() compressedSizes = new Array[Byte](len) in.readFully(compressedSizes) @@ -132,6 +140,7 @@ private[spark] class CompressedMapStatus( */ private[spark] class HighlyCompressedMapStatus private ( private[this] var loc: BlockManagerId, + private[this] var _stageAttemptId: Int, private[this] var numNonEmptyBlocks: Int, private[this] var emptyBlocks: RoaringBitmap, private[this] var avgSize: Long) @@ -141,10 +150,12 @@ private[spark] class HighlyCompressedMapStatus private ( require(loc == null || avgSize > 0 || numNonEmptyBlocks == 0, "Average size can only be zero for map stages that produced no output") - protected def this() = this(null, -1, null, -1) // For deserialization only + protected def this() = this(null, 0, -1, null, -1) // For deserialization only override def location: BlockManagerId = loc + override def stageAttemptId: Int = _stageAttemptId + override def getSizeForBlock(reduceId: Int): Long = { if (emptyBlocks.contains(reduceId)) { 0 @@ -155,12 +166,14 @@ private[spark] class HighlyCompressedMapStatus private ( override def writeExternal(out: ObjectOutput): Unit = Utils.tryOrIOException { loc.writeExternal(out) + out.writeInt(_stageAttemptId) emptyBlocks.writeExternal(out) out.writeLong(avgSize) } override def readExternal(in: ObjectInput): Unit = Utils.tryOrIOException { loc = BlockManagerId(in) + _stageAttemptId = in.readInt() emptyBlocks = new RoaringBitmap() emptyBlocks.readExternal(in) avgSize = in.readLong() @@ -168,7 +181,10 @@ private[spark] class HighlyCompressedMapStatus private ( } private[spark] object HighlyCompressedMapStatus { - def apply(loc: BlockManagerId, uncompressedSizes: Array[Long]): HighlyCompressedMapStatus = { + def apply( + loc: BlockManagerId, + stageAttemptId: Int, + uncompressedSizes: Array[Long]): HighlyCompressedMapStatus = { // We must keep track of which blocks are empty so that we don't report a zero-sized // block as being non-empty (or vice-versa) when using the average block size. var i = 0 @@ -194,6 +210,6 @@ private[spark] object HighlyCompressedMapStatus { } else { 0 } - new HighlyCompressedMapStatus(loc, numNonEmptyBlocks, emptyBlocks, avgSize) + new HighlyCompressedMapStatus(loc, stageAttemptId, numNonEmptyBlocks, emptyBlocks, avgSize) } } 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 14c8c00961487..34103a37649e5 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala @@ -67,7 +67,7 @@ private[spark] class ShuffleMapTask( var writer: ShuffleWriter[Any, Any] = null try { val manager = SparkEnv.get.shuffleManager - writer = manager.getWriter[Any, Any](dep.shuffleHandle, partitionId, context) + writer = manager.getWriter[Any, Any](dep.shuffleHandle, partitionId, stageAttemptId, context) writer.write(rdd.iterator(partition, context).asInstanceOf[Iterator[_ <: Product2[Any, Any]]]) return writer.stop(success = true).get } catch { diff --git a/core/src/main/scala/org/apache/spark/shuffle/FetchFailedException.scala b/core/src/main/scala/org/apache/spark/shuffle/FetchFailedException.scala index be184464e0ae9..fd848be90cc2d 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/FetchFailedException.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/FetchFailedException.scala @@ -32,6 +32,7 @@ private[spark] class FetchFailedException( shuffleId: Int, mapId: Int, reduceId: Int, + stageAttemptId: Int, message: String, cause: Throwable = null) extends Exception(message, cause) { @@ -41,8 +42,9 @@ private[spark] class FetchFailedException( shuffleId: Int, mapId: Int, reduceId: Int, + stageAttemptId: Int, cause: Throwable) { - this(bmAddress, shuffleId, mapId, reduceId, cause.getMessage, cause) + this(bmAddress, shuffleId, mapId, reduceId, stageAttemptId, cause.getMessage, cause) } def toTaskEndReason: TaskEndReason = FetchFailed(bmAddress, shuffleId, mapId, reduceId, @@ -56,4 +58,4 @@ private[spark] class MetadataFetchFailedException( shuffleId: Int, reduceId: Int, message: String) - extends FetchFailedException(null, shuffleId, -1, reduceId, message) + extends FetchFailedException(null, shuffleId, -1, reduceId, -1, message) 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 6c3b3080d2605..63dbb8ddf4655 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/FileShuffleBlockResolver.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/FileShuffleBlockResolver.scala @@ -104,11 +104,11 @@ private[spark] class FileShuffleBlockResolver(conf: SparkConf) * Get a ShuffleWriterGroup for the given map task, which will register it as complete * when the writers are closed successfully */ - def forMapTask(shuffleId: Int, mapId: Int, numBuckets: Int, serializer: Serializer, - writeMetrics: ShuffleWriteMetrics): ShuffleWriterGroup = { + def forMapTask(shuffleId: Int, mapId: Int, stageAttemptId: Int, numBuckets: Int, + serializer: Serializer, writeMetrics: ShuffleWriteMetrics): ShuffleWriterGroup = { new ShuffleWriterGroup { - shuffleStates.putIfAbsent(shuffleId, new ShuffleState(numBuckets)) - private val shuffleState = shuffleStates(shuffleId) + shuffleStates.putIfAbsent((shuffleId, stageAttemptId), new ShuffleState(numBuckets)) + private val shuffleState = shuffleStates((shuffleId, stageAttemptId)) private var fileGroup: ShuffleFileGroup = null val openStartTime = System.nanoTime @@ -116,13 +116,13 @@ private[spark] class FileShuffleBlockResolver(conf: SparkConf) val writers: Array[BlockObjectWriter] = if (consolidateShuffleFiles) { fileGroup = getUnusedFileGroup() Array.tabulate[BlockObjectWriter](numBuckets) { bucketId => - val blockId = ShuffleBlockId(shuffleId, mapId, bucketId) + val blockId = ShuffleBlockId(shuffleId, mapId, bucketId, stageAttemptId) blockManager.getDiskWriter(blockId, fileGroup(bucketId), serializerInstance, bufferSize, writeMetrics) } } else { Array.tabulate[BlockObjectWriter](numBuckets) { bucketId => - val blockId = ShuffleBlockId(shuffleId, mapId, bucketId) + val blockId = ShuffleBlockId(shuffleId, mapId, bucketId, stageAttemptId) val blockFile = blockManager.diskBlockManager.getFile(blockId) // Because of previous failures, the shuffle file may already exist on this machine. // If so, remove it. @@ -179,7 +179,7 @@ private[spark] class FileShuffleBlockResolver(conf: SparkConf) override def getBlockData(blockId: ShuffleBlockId): ManagedBuffer = { if (consolidateShuffleFiles) { // Search all file groups associated with this shuffle. - val shuffleState = shuffleStates(blockId.shuffleId) + val shuffleState = shuffleStates((blockId.shuffleId, blockId.stageAttemptId)) val iter = shuffleState.allFileGroups.iterator while (iter.hasNext) { val segmentOpt = iter.next.getFileSegmentFor(blockId.mapId, blockId.reduceId) @@ -215,7 +215,7 @@ private[spark] class FileShuffleBlockResolver(conf: SparkConf) } } else { for (mapId <- state.completedMapTasks; reduceId <- 0 until state.numBuckets) { - val blockId = new ShuffleBlockId(shuffleId, mapId, reduceId) + val blockId = new ShuffleBlockId(shuffleId._1, mapId, reduceId, shuffleId._2) blockManager.diskBlockManager.getFile(blockId).delete() } } 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 a6bd3020d147f..7bfde919002ae 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala @@ -46,24 +46,26 @@ private[spark] class IndexShuffleBlockResolver(conf: SparkConf) extends ShuffleB private val transportConf = SparkTransportConf.fromSparkConf(conf) - def getDataFile(shuffleId: Int, mapId: Int): File = { - blockManager.diskBlockManager.getFile(ShuffleDataBlockId(shuffleId, mapId, NOOP_REDUCE_ID)) + def getDataFile(shuffleId: Int, mapId: Int, stageAttemptId: Int): File = { + blockManager.diskBlockManager.getFile(ShuffleDataBlockId(shuffleId, mapId, NOOP_REDUCE_ID, + stageAttemptId)) } - private def getIndexFile(shuffleId: Int, mapId: Int): File = { - blockManager.diskBlockManager.getFile(ShuffleIndexBlockId(shuffleId, mapId, NOOP_REDUCE_ID)) + private def getIndexFile(shuffleId: Int, mapId: Int, stageAttemptId: Int): File = { + blockManager.diskBlockManager.getFile(ShuffleIndexBlockId(shuffleId, mapId, NOOP_REDUCE_ID, + stageAttemptId)) } /** * Remove data file and index file that contain the output data from one map. * */ - def removeDataByMap(shuffleId: Int, mapId: Int): Unit = { - var file = getDataFile(shuffleId, mapId) + def removeDataByMap(shuffleId: Int, mapId: Int, stageAttemptId: Int): Unit = { + var file = getDataFile(shuffleId, mapId, stageAttemptId) if (file.exists()) { file.delete() } - file = getIndexFile(shuffleId, mapId) + file = getIndexFile(shuffleId, mapId, stageAttemptId) if (file.exists()) { file.delete() } @@ -77,13 +79,13 @@ private[spark] class IndexShuffleBlockResolver(conf: SparkConf) extends ShuffleB def writeIndexFile( shuffleId: Int, mapId: Int, - lengths: Array[Long], - initialFileLength: Long): Unit = { - val indexFile = getIndexFile(shuffleId, mapId) + stageAttemptId: Int, + lengths: Array[Long]): Unit = { + val indexFile = getIndexFile(shuffleId, mapId, stageAttemptId) val out = new DataOutputStream(new BufferedOutputStream(new FileOutputStream(indexFile))) Utils.tryWithSafeFinally { // We take in lengths of each block, need to convert it to offsets. - var offset = initialFileLength + var offset = 0L out.writeLong(offset) for (length <- lengths) { offset += length @@ -97,7 +99,7 @@ private[spark] class IndexShuffleBlockResolver(conf: SparkConf) extends ShuffleB override def getBlockData(blockId: ShuffleBlockId): ManagedBuffer = { // The block is actually going to be a range of a single map output file for this map, so // find out the consolidated file, then the offset within that from our index - val indexFile = getIndexFile(blockId.shuffleId, blockId.mapId) + val indexFile = getIndexFile(blockId.shuffleId, blockId.mapId, blockId.stageAttemptId) val in = new DataInputStream(new FileInputStream(indexFile)) try { @@ -106,7 +108,7 @@ private[spark] class IndexShuffleBlockResolver(conf: SparkConf) extends ShuffleB val nextOffset = in.readLong() new FileSegmentManagedBuffer( transportConf, - getDataFile(blockId.shuffleId, blockId.mapId), + getDataFile(blockId.shuffleId, blockId.mapId, blockId.stageAttemptId), offset, nextOffset - offset) } finally { diff --git a/core/src/main/scala/org/apache/spark/shuffle/ShuffleBlockResolver.scala b/core/src/main/scala/org/apache/spark/shuffle/ShuffleBlockResolver.scala index 4342b0d598b16..36a7873859e8f 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/ShuffleBlockResolver.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/ShuffleBlockResolver.scala @@ -29,7 +29,7 @@ private[spark] * implementations when shuffle data is retrieved. */ trait ShuffleBlockResolver { - type ShuffleId = Int + type ShuffleId = (Int,Int) /** * Retrieve the data for the specified block. If the data for that block is not available, diff --git a/core/src/main/scala/org/apache/spark/shuffle/ShuffleManager.scala b/core/src/main/scala/org/apache/spark/shuffle/ShuffleManager.scala index 978366d1a1d1b..1b7865198de80 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/ShuffleManager.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/ShuffleManager.scala @@ -17,6 +17,10 @@ package org.apache.spark.shuffle +import java.util.concurrent.ConcurrentHashMap + +import scala.collection.JavaConverters._ + import org.apache.spark.{TaskContext, ShuffleDependency} /** @@ -37,7 +41,11 @@ private[spark] trait ShuffleManager { dependency: ShuffleDependency[K, V, C]): ShuffleHandle /** Get a writer for a given partition. Called on executors by map tasks. */ - def getWriter[K, V](handle: ShuffleHandle, mapId: Int, context: TaskContext): ShuffleWriter[K, V] + def getWriter[K, V]( + handle: ShuffleHandle, + mapId: Int, + stageAttemptId: Int, + context: TaskContext): ShuffleWriter[K, V] /** * Get a reader for a range of reduce partitions (startPartition to endPartition-1, inclusive). @@ -62,4 +70,14 @@ private[spark] trait ShuffleManager { /** Shut down this ShuffleManager. */ def stop(): Unit + + private[this] val shuffleToAttempts = new ConcurrentHashMap[Int, ConcurrentHashMap[Int, Int]]() + protected def addShuffleAttempt(shuffleId: Int, stageAttemptId: Int): Unit = { + shuffleToAttempts.putIfAbsent(shuffleId, new ConcurrentHashMap[Int,Int]()) + shuffleToAttempts.get(shuffleId).putIfAbsent(stageAttemptId, stageAttemptId) + } + protected def stageAttemptsForShuffle(shuffleId: Int): Iterable[Int] = { + val attempts = shuffleToAttempts.get(shuffleId) + attempts.values.asScala + } } diff --git a/core/src/main/scala/org/apache/spark/shuffle/hash/BlockStoreShuffleFetcher.scala b/core/src/main/scala/org/apache/spark/shuffle/hash/BlockStoreShuffleFetcher.scala index 80374adc44296..c5c934463a7c3 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/hash/BlockStoreShuffleFetcher.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/hash/BlockStoreShuffleFetcher.scala @@ -43,14 +43,14 @@ private[hash] object BlockStoreShuffleFetcher extends Logging { logDebug("Fetching map output location for shuffle %d, reduce %d took %d ms".format( shuffleId, reduceId, System.currentTimeMillis - startTime)) - val splitsByAddress = new HashMap[BlockManagerId, ArrayBuffer[(Int, Long)]] - for (((address, size), index) <- statuses.zipWithIndex) { - splitsByAddress.getOrElseUpdate(address, ArrayBuffer()) += ((index, size)) + val splitsByAddress = new HashMap[BlockManagerId, ArrayBuffer[(Int, Int, Long)]] + for (((address, stageAttempt, size), index) <- statuses.zipWithIndex) { + splitsByAddress.getOrElseUpdate(address, ArrayBuffer()) += ((index, stageAttempt, size)) } val blocksByAddress: Seq[(BlockManagerId, Seq[(BlockId, Long)])] = splitsByAddress.toSeq.map { case (address, splits) => - (address, splits.map(s => (ShuffleBlockId(shuffleId, s._1, reduceId), s._2))) + (address, splits.map(s => (ShuffleBlockId(shuffleId, s._1, reduceId, s._2), s._3))) } def unpackBlock(blockPair: (BlockId, Try[Iterator[Any]])) : Iterator[T] = { @@ -62,9 +62,10 @@ private[hash] object BlockStoreShuffleFetcher extends Logging { } case Failure(e) => { blockId match { - case ShuffleBlockId(shufId, mapId, _) => + case ShuffleBlockId(shufId, mapId, _, stageAttempt) => val address = statuses(mapId.toInt)._1 - throw new FetchFailedException(address, shufId.toInt, mapId.toInt, reduceId, e) + throw new FetchFailedException(address, shufId.toInt, mapId.toInt, reduceId, + stageAttempt, e) case _ => throw new SparkException( "Failed to get block " + blockId + ", which is not a shuffle block", e) diff --git a/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleManager.scala b/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleManager.scala index c089088f409dd..7896d8b77062f 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleManager.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleManager.scala @@ -50,15 +50,21 @@ private[spark] class HashShuffleManager(conf: SparkConf) extends ShuffleManager } /** Get a writer for a given partition. Called on executors by map tasks. */ - override def getWriter[K, V](handle: ShuffleHandle, mapId: Int, context: TaskContext) - : ShuffleWriter[K, V] = { - new HashShuffleWriter( - shuffleBlockResolver, handle.asInstanceOf[BaseShuffleHandle[K, V, _]], mapId, context) + override def getWriter[K, V]( + handle: ShuffleHandle, + mapId: Int, + stageAttemptId: Int, + context: TaskContext): ShuffleWriter[K, V] = { + addShuffleAttempt(handle.shuffleId, stageAttemptId) + new HashShuffleWriter(shuffleBlockResolver, handle.asInstanceOf[BaseShuffleHandle[K, V, _]], + mapId, stageAttemptId, context) } /** Remove a shuffle's metadata from the ShuffleManager. */ override def unregisterShuffle(shuffleId: Int): Boolean = { - shuffleBlockResolver.removeShuffle(shuffleId) + stageAttemptsForShuffle(shuffleId).forall { stageAttemptId => + shuffleBlockResolver.removeShuffle(shuffleId, stageAttemptId) + } } override def shuffleBlockResolver: FileShuffleBlockResolver = { 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 eb87cee15903c..cb0ed4e32007e 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 @@ -28,6 +28,7 @@ private[spark] class HashShuffleWriter[K, V]( shuffleBlockResolver: FileShuffleBlockResolver, handle: BaseShuffleHandle[K, V, _], mapId: Int, + stageAttemptId: Int, context: TaskContext) extends ShuffleWriter[K, V] with Logging { @@ -45,8 +46,8 @@ private[spark] class HashShuffleWriter[K, V]( private val blockManager = SparkEnv.get.blockManager private val ser = Serializer.getSerializer(dep.serializer.getOrElse(null)) - private val shuffle = shuffleBlockResolver.forMapTask(dep.shuffleId, mapId, numOutputSplits, ser, - writeMetrics) + private val shuffle = shuffleBlockResolver.forMapTask(dep.shuffleId, mapId, stageAttemptId, + numOutputSplits, ser, writeMetrics) /** Write a bunch of records to this task's output */ override def write(records: Iterator[Product2[K, V]]): Unit = { @@ -106,7 +107,7 @@ private[spark] class HashShuffleWriter[K, V]( writer.commitAndClose() writer.fileSegment().length } - MapStatus(blockManager.shuffleServerId, sizes) + MapStatus(blockManager.shuffleServerId, stageAttemptId, sizes) } private def revertWrites(): Unit = { diff --git a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleManager.scala b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleManager.scala index d7fab351ca3b8..99ea4f744bb0b 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleManager.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleManager.scala @@ -53,20 +53,29 @@ private[spark] class SortShuffleManager(conf: SparkConf) extends ShuffleManager } /** Get a writer for a given partition. Called on executors by map tasks. */ - override def getWriter[K, V](handle: ShuffleHandle, mapId: Int, context: TaskContext) + override def getWriter[K, V]( + handle: ShuffleHandle, + mapId: Int, + stageAttemptId: Int, + context: TaskContext) : ShuffleWriter[K, V] = { val baseShuffleHandle = handle.asInstanceOf[BaseShuffleHandle[K, V, _]] + val shuffleId = baseShuffleHandle.shuffleId + addShuffleAttempt(shuffleId, stageAttemptId) shuffleMapNumber.putIfAbsent(baseShuffleHandle.shuffleId, baseShuffleHandle.numMaps) new SortShuffleWriter( - shuffleBlockResolver, baseShuffleHandle, mapId, context) + shuffleBlockResolver, baseShuffleHandle, mapId, stageAttemptId, context) } /** Remove a shuffle's metadata from the ShuffleManager. */ override def unregisterShuffle(shuffleId: Int): Boolean = { if (shuffleMapNumber.containsKey(shuffleId)) { val numMaps = shuffleMapNumber.remove(shuffleId) + val attempts = stageAttemptsForShuffle(shuffleId) (0 until numMaps).map{ mapId => - shuffleBlockResolver.removeDataByMap(shuffleId, mapId) + attempts.foreach { stageAttemptId => + shuffleBlockResolver.removeDataByMap(shuffleId, mapId, stageAttemptId) + } } } true 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 bbdd7a5c5cb99..36608dad62ea4 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 @@ -28,6 +28,7 @@ private[spark] class SortShuffleWriter[K, V, C]( shuffleBlockResolver: IndexShuffleBlockResolver, handle: BaseShuffleHandle[K, V, C], mapId: Int, + stageAttemptId: Int, context: TaskContext) extends ShuffleWriter[K, V] with Logging { @@ -65,15 +66,16 @@ 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 outputFile = shuffleBlockResolver.getDataFile(dep.shuffleId, mapId, stageAttemptId) // Because we append to the data file, we need the index file to know the current size of the // data file as a starting point val initialFileLength = outputFile.length() - val blockId = ShuffleBlockId(dep.shuffleId, mapId, IndexShuffleBlockResolver.NOOP_REDUCE_ID) + val blockId = ShuffleBlockId(dep.shuffleId, mapId, IndexShuffleBlockResolver.NOOP_REDUCE_ID, + stageAttemptId) val partitionLengths = sorter.writePartitionedFile(blockId, context, outputFile) - shuffleBlockResolver.writeIndexFile(dep.shuffleId, mapId, partitionLengths, initialFileLength) + shuffleBlockResolver.writeIndexFile(dep.shuffleId, mapId, stageAttemptId, partitionLengths) - mapStatus = MapStatus(blockManager.shuffleServerId, partitionLengths) + mapStatus = MapStatus(blockManager.shuffleServerId, stageAttemptId, partitionLengths) } /** Close this writer, passing along whether the map completed */ @@ -87,7 +89,7 @@ private[spark] class SortShuffleWriter[K, V, C]( return Option(mapStatus) } else { // The map task failed, so delete our output data. - shuffleBlockResolver.removeDataByMap(dep.shuffleId, mapId) + shuffleBlockResolver.removeDataByMap(dep.shuffleId, mapId, stageAttemptId) return None } } finally { diff --git a/core/src/main/scala/org/apache/spark/shuffle/unsafe/UnsafeShuffleManager.scala b/core/src/main/scala/org/apache/spark/shuffle/unsafe/UnsafeShuffleManager.scala index f2bfef376d3ca..4baac3bcb8c9b 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/unsafe/UnsafeShuffleManager.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/unsafe/UnsafeShuffleManager.scala @@ -160,11 +160,13 @@ private[spark] class UnsafeShuffleManager(conf: SparkConf) extends ShuffleManage override def getWriter[K, V]( handle: ShuffleHandle, mapId: Int, + stageAttemptId:Int, context: TaskContext): ShuffleWriter[K, V] = { handle match { case unsafeShuffleHandle: UnsafeShuffleHandle[K, V] => numMapsForShufflesThatUsedNewPath.putIfAbsent(handle.shuffleId, unsafeShuffleHandle.numMaps) val env = SparkEnv.get + addShuffleAttempt(handle.shuffleId, stageAttemptId) new UnsafeShuffleWriter( env.blockManager, shuffleBlockResolver.asInstanceOf[IndexShuffleBlockResolver], @@ -172,11 +174,12 @@ private[spark] class UnsafeShuffleManager(conf: SparkConf) extends ShuffleManage env.shuffleMemoryManager, unsafeShuffleHandle, mapId, + stageAttemptId, context, env.conf) case other => shufflesThatFellBackToSortShuffle.add(handle.shuffleId) - sortShuffleManager.getWriter(handle, mapId, context) + sortShuffleManager.getWriter(handle, mapId, stageAttemptId, context) } } @@ -186,8 +189,11 @@ private[spark] class UnsafeShuffleManager(conf: SparkConf) extends ShuffleManage sortShuffleManager.unregisterShuffle(shuffleId) } else { Option(numMapsForShufflesThatUsedNewPath.remove(shuffleId)).foreach { numMaps => + val attempts = stageAttemptsForShuffle(shuffleId) (0 until numMaps).foreach { mapId => - shuffleBlockResolver.removeDataByMap(shuffleId, mapId) + attempts.foreach{ stageAttemptId => + shuffleBlockResolver.removeDataByMap(shuffleId, mapId, stageAttemptId) + } } } true 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..bf713521eb807 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockId.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockId.scala @@ -56,18 +56,26 @@ case class RDDBlockId(rddId: Int, splitIndex: Int) extends BlockId { // Format of the shuffle block ids (including data and index) should be kept in sync with // org.apache.spark.network.shuffle.ExternalShuffleBlockResolver#getBlockData(). @DeveloperApi -case class ShuffleBlockId(shuffleId: Int, mapId: Int, reduceId: Int) extends BlockId { - override def name: String = "shuffle_" + shuffleId + "_" + mapId + "_" + reduceId +case class ShuffleBlockId(shuffleId: Int, mapId: Int, reduceId: Int, stageAttemptId: Int) + extends BlockId { + override def name: String = { + "shuffle_" + shuffleId + "_" + mapId + "_" + reduceId + "_" + stageAttemptId + } } @DeveloperApi -case class ShuffleDataBlockId(shuffleId: Int, mapId: Int, reduceId: Int) extends BlockId { - override def name: String = "shuffle_" + shuffleId + "_" + mapId + "_" + reduceId + ".data" +case class ShuffleDataBlockId(shuffleId: Int, mapId: Int, stageAttemptId: Int, reduceId: Int) + extends BlockId { + override def name: String = { + "shuffle_" + shuffleId + "_" + mapId + "_" + reduceId + "_" + stageAttemptId + ".data" + } } @DeveloperApi -case class ShuffleIndexBlockId(shuffleId: Int, mapId: Int, reduceId: Int) extends BlockId { - override def name: String = "shuffle_" + shuffleId + "_" + mapId + "_" + reduceId + ".index" +case class ShuffleIndexBlockId(shuffleId: Int, mapId: Int, stageAttemptId: Int, reduceId: Int) extends BlockId { + override def name: String = { + "shuffle_" + shuffleId + "_" + mapId + "_" + reduceId + "_" + stageAttemptId + ".index" + } } @DeveloperApi @@ -103,9 +111,9 @@ private[spark] case class TestBlockId(id: String) extends BlockId { @DeveloperApi object BlockId { val RDD = "rdd_([0-9]+)_([0-9]+)".r - 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 = "shuffle_([0-9]+)_([0-9]+)_([0-9]+)_([0-9]+)".r + val SHUFFLE_DATA = "shuffle_([0-9]+)_([0-9]+)_([0-9]+)_([0-9]+).data".r + val SHUFFLE_INDEX = "shuffle_([0-9]+)_([0-9]+)_([0-9]+)_([0-9]+).index".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 @@ -115,12 +123,12 @@ object BlockId { def apply(id: String): BlockId = id match { case RDD(rddId, splitIndex) => RDDBlockId(rddId.toInt, splitIndex.toInt) - case SHUFFLE(shuffleId, mapId, reduceId) => - ShuffleBlockId(shuffleId.toInt, mapId.toInt, reduceId.toInt) - case SHUFFLE_DATA(shuffleId, mapId, reduceId) => - ShuffleDataBlockId(shuffleId.toInt, mapId.toInt, reduceId.toInt) - case SHUFFLE_INDEX(shuffleId, mapId, reduceId) => - ShuffleIndexBlockId(shuffleId.toInt, mapId.toInt, reduceId.toInt) + case SHUFFLE(shuffleId, mapId, reduceId, stageAttemptId) => + ShuffleBlockId(shuffleId.toInt, mapId.toInt, reduceId.toInt, stageAttemptId.toInt) + case SHUFFLE_DATA(shuffleId, mapId, reduceId, stageAttemptId) => + ShuffleDataBlockId(shuffleId.toInt, mapId.toInt, reduceId.toInt, stageAttemptId.toInt) + case SHUFFLE_INDEX(shuffleId, mapId, reduceId, stageAttemptId) => + ShuffleIndexBlockId(shuffleId.toInt, mapId.toInt, reduceId.toInt, stageAttemptId.toInt) case BROADCAST(broadcastId, field) => BroadcastBlockId(broadcastId.toLong, field.stripPrefix("_")) case TASKRESULT(taskId) => diff --git a/core/src/test/java/org/apache/spark/shuffle/unsafe/UnsafeShuffleWriterSuite.java b/core/src/test/java/org/apache/spark/shuffle/unsafe/UnsafeShuffleWriterSuite.java index ffbe8d37d025b..55b1328ce9117 100644 --- a/core/src/test/java/org/apache/spark/shuffle/unsafe/UnsafeShuffleWriterSuite.java +++ b/core/src/test/java/org/apache/spark/shuffle/unsafe/UnsafeShuffleWriterSuite.java @@ -205,6 +205,7 @@ private UnsafeShuffleWriter createWriter( shuffleMemoryManager, new UnsafeShuffleHandle(0, 1, shuffleDep), 0, // map id + 0, // stage attempt id taskContext, conf ); From 25234317978445b8bd10b6d6230000bfd7814267 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Fri, 29 May 2015 13:27:25 -0500 Subject: [PATCH 20/73] tests compile --- .../unsafe/UnsafeShuffleWriterSuite.java | 5 ++- .../apache/spark/ContextCleanerSuite.scala | 4 +- .../apache/spark/MapOutputTrackerSuite.scala | 18 ++++---- .../scala/org/apache/spark/ShuffleSuite.scala | 4 +- .../NettyBlockTransferSecuritySuite.scala | 2 +- .../spark/scheduler/DAGSchedulerSuite.scala | 6 +-- .../spark/scheduler/MapStatusSuite.scala | 8 ++-- .../serializer/KryoSerializerSuite.scala | 2 +- .../hash/HashShuffleManagerSuite.scala | 12 ++--- .../apache/spark/storage/BlockIdSuite.scala | 10 +++-- .../spark/storage/BlockManagerSuite.scala | 8 ++-- .../ShuffleBlockFetcherIteratorSuite.scala | 44 +++++++++---------- .../org/apache/spark/ui/UISeleniumSuite.scala | 4 +- .../apache/spark/util/AkkaUtilsSuite.scala | 8 ++-- .../apache/spark/util/JsonProtocolSuite.scala | 2 +- 15 files changed, 72 insertions(+), 65 deletions(-) diff --git a/core/src/test/java/org/apache/spark/shuffle/unsafe/UnsafeShuffleWriterSuite.java b/core/src/test/java/org/apache/spark/shuffle/unsafe/UnsafeShuffleWriterSuite.java index 55b1328ce9117..a08c90d6473a7 100644 --- a/core/src/test/java/org/apache/spark/shuffle/unsafe/UnsafeShuffleWriterSuite.java +++ b/core/src/test/java/org/apache/spark/shuffle/unsafe/UnsafeShuffleWriterSuite.java @@ -168,14 +168,15 @@ public OutputStream answer(InvocationOnMock invocation) throws Throwable { } ); - when(shuffleBlockResolver.getDataFile(anyInt(), anyInt())).thenReturn(mergedOutputFile); + when(shuffleBlockResolver.getDataFile(anyInt(), anyInt(), anyInt())) + .thenReturn(mergedOutputFile); doAnswer(new Answer() { @Override public Void answer(InvocationOnMock invocationOnMock) throws Throwable { partitionSizesInMergedFile = (long[]) invocationOnMock.getArguments()[2]; return null; } - }).when(shuffleBlockResolver).writeIndexFile(anyInt(), anyInt(), any(long[].class), eq(0L)); + }).when(shuffleBlockResolver).writeIndexFile(anyInt(), anyInt(), anyInt(), any(long[].class)); when(diskBlockManager.createTempShuffleBlock()).thenAnswer( new Answer>() { diff --git a/core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala b/core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala index 0922a2c3599cc..f5d6886d71a03 100644 --- a/core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala +++ b/core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala @@ -574,8 +574,8 @@ class CleanerTester( private def getShuffleBlocks(shuffleId: Int): Seq[BlockId] = { blockManager.master.getMatchingBlockIds( _ match { - case ShuffleBlockId(`shuffleId`, _, _) => true - case ShuffleIndexBlockId(`shuffleId`, _, _) => true + case ShuffleBlockId(`shuffleId`, _, _, _) => true + case ShuffleIndexBlockId(`shuffleId`, _, _, _) => true case _ => false }, askSlaves = true) } diff --git a/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala b/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala index 6ed057a7cab97..204230d77d885 100644 --- a/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala +++ b/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala @@ -52,9 +52,9 @@ class MapOutputTrackerSuite extends FunSuite { assert(tracker.containsShuffle(10)) val size1000 = MapStatus.decompressSize(MapStatus.compressSize(1000L)) val size10000 = MapStatus.decompressSize(MapStatus.compressSize(10000L)) - tracker.registerMapOutput(10, 0, MapStatus(BlockManagerId("a", "hostA", 1000), + tracker.registerMapOutput(10, 0, MapStatus(BlockManagerId("a", "hostA", 1000), 0, Array(1000L, 10000L))) - tracker.registerMapOutput(10, 1, MapStatus(BlockManagerId("b", "hostB", 1000), + tracker.registerMapOutput(10, 1, MapStatus(BlockManagerId("b", "hostB", 1000), 0, Array(10000L, 1000L))) val statuses = tracker.getServerStatuses(10, 0) assert(statuses.toSeq === Seq((BlockManagerId("a", "hostA", 1000), size1000), @@ -71,9 +71,9 @@ class MapOutputTrackerSuite extends FunSuite { tracker.registerShuffle(10, 2) val compressedSize1000 = MapStatus.compressSize(1000L) val compressedSize10000 = MapStatus.compressSize(10000L) - tracker.registerMapOutput(10, 0, MapStatus(BlockManagerId("a", "hostA", 1000), + tracker.registerMapOutput(10, 0, MapStatus(BlockManagerId("a", "hostA", 1000), 0, Array(compressedSize1000, compressedSize10000))) - tracker.registerMapOutput(10, 1, MapStatus(BlockManagerId("b", "hostB", 1000), + tracker.registerMapOutput(10, 1, MapStatus(BlockManagerId("b", "hostB", 1000), 0, Array(compressedSize10000, compressedSize1000))) assert(tracker.containsShuffle(10)) assert(tracker.getServerStatuses(10, 0).nonEmpty) @@ -93,9 +93,9 @@ class MapOutputTrackerSuite extends FunSuite { tracker.registerShuffle(10, 2) val compressedSize1000 = MapStatus.compressSize(1000L) val compressedSize10000 = MapStatus.compressSize(10000L) - tracker.registerMapOutput(10, 0, MapStatus(BlockManagerId("a", "hostA", 1000), + tracker.registerMapOutput(10, 0, MapStatus(BlockManagerId("a", "hostA", 1000), 0, Array(compressedSize1000, compressedSize1000, compressedSize1000))) - tracker.registerMapOutput(10, 1, MapStatus(BlockManagerId("b", "hostB", 1000), + tracker.registerMapOutput(10, 1, MapStatus(BlockManagerId("b", "hostB", 1000), 0, Array(compressedSize10000, compressedSize1000, compressedSize1000))) // As if we had two simultaneous fetch failures @@ -131,7 +131,7 @@ class MapOutputTrackerSuite extends FunSuite { val size1000 = MapStatus.decompressSize(MapStatus.compressSize(1000L)) masterTracker.registerMapOutput(10, 0, MapStatus( - BlockManagerId("a", "hostA", 1000), Array(1000L))) + BlockManagerId("a", "hostA", 1000), 0, Array(1000L))) masterTracker.incrementEpoch() slaveTracker.updateEpoch(masterTracker.getEpoch) assert(slaveTracker.getServerStatuses(10, 0).toSeq === @@ -164,7 +164,7 @@ class MapOutputTrackerSuite extends FunSuite { // Frame size should be ~123B, and no exception should be thrown masterTracker.registerShuffle(10, 1) masterTracker.registerMapOutput(10, 0, MapStatus( - BlockManagerId("88", "mph", 1000), Array.fill[Long](10)(0))) + BlockManagerId("88", "mph", 1000), 0, Array.fill[Long](10)(0))) val sender = mock(classOf[RpcEndpointRef]) when(sender.address).thenReturn(RpcAddress("localhost", 12345)) val rpcCallContext = mock(classOf[RpcCallContext]) @@ -193,7 +193,7 @@ class MapOutputTrackerSuite extends FunSuite { masterTracker.registerShuffle(20, 100) (0 until 100).foreach { i => masterTracker.registerMapOutput(20, i, new CompressedMapStatus( - BlockManagerId("999", "mps", 1000), Array.fill[Long](4000000)(0))) + BlockManagerId("999", "mps", 1000), 0, Array.fill[Long](4000000)(0))) } val sender = mock(classOf[RpcEndpointRef]) when(sender.address).thenReturn(RpcAddress("localhost", 12345)) diff --git a/core/src/test/scala/org/apache/spark/ShuffleSuite.scala b/core/src/test/scala/org/apache/spark/ShuffleSuite.scala index d7180516029d5..a3bc67726e1b1 100644 --- a/core/src/test/scala/org/apache/spark/ShuffleSuite.scala +++ b/core/src/test/scala/org/apache/spark/ShuffleSuite.scala @@ -268,8 +268,8 @@ abstract class ShuffleSuite extends FunSuite with Matchers with LocalSparkContex rdd.count() // Delete one of the local shuffle blocks. - val hashFile = sc.env.blockManager.diskBlockManager.getFile(new ShuffleBlockId(0, 0, 0)) - val sortFile = sc.env.blockManager.diskBlockManager.getFile(new ShuffleDataBlockId(0, 0, 0)) + val hashFile = sc.env.blockManager.diskBlockManager.getFile(new ShuffleBlockId(0, 0, 0, 0)) + val sortFile = sc.env.blockManager.diskBlockManager.getFile(new ShuffleDataBlockId(0, 0, 0, 0)) assert(hashFile.exists() || sortFile.exists()) if (hashFile.exists()) { diff --git a/core/src/test/scala/org/apache/spark/network/netty/NettyBlockTransferSecuritySuite.scala b/core/src/test/scala/org/apache/spark/network/netty/NettyBlockTransferSecuritySuite.scala index 46d2e5173acae..ec0894b8b20d9 100644 --- a/core/src/test/scala/org/apache/spark/network/netty/NettyBlockTransferSecuritySuite.scala +++ b/core/src/test/scala/org/apache/spark/network/netty/NettyBlockTransferSecuritySuite.scala @@ -100,7 +100,7 @@ class NettyBlockTransferSecuritySuite extends FunSuite with MockitoSugar with Sh */ private def testConnection(conf0: SparkConf, conf1: SparkConf): Try[Unit] = { val blockManager = mock[BlockDataManager] - val blockId = ShuffleBlockId(0, 1, 2) + val blockId = ShuffleBlockId(0, 1, 2, 0) val blockString = "Hello, world!" val blockBuffer = new NioManagedBuffer(ByteBuffer.wrap(blockString.getBytes)) when(blockManager.getBlockData(blockId)).thenReturn(blockBuffer) diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala index 29056eff0e57e..220a9540aa09e 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -818,11 +818,11 @@ class DAGSchedulerSuite // second attempt if (TaskContext.get().asInstanceOf[TaskContextImpl].stageAttemptId == 0) { if (idx == 0) { - throw new FetchFailedException(aBm, 0, 0, idx, + throw new FetchFailedException(aBm, 0, 0, idx, 0, cause = new RuntimeException("simulated fetch failure")) } else if (idx > 0 && math.random < 0.2) { Thread.sleep(5000) - throw new FetchFailedException(aBm, 0, 0, idx, + throw new FetchFailedException(aBm, 0, 0, idx, 0, cause = new RuntimeException("simulated fetch failure")) } else { // want to make sure plenty of these finish after task 0 fails, and some even finish @@ -858,7 +858,7 @@ class DAGSchedulerSuite } private def makeMapStatus(host: String, reduces: Int): MapStatus = - MapStatus(makeBlockManagerId(host), Array.fill[Long](reduces)(2)) + MapStatus(makeBlockManagerId(host), 0, Array.fill[Long](reduces)(2)) private def makeBlockManagerId(host: String): BlockManagerId = BlockManagerId("exec-" + host, host, 12345) diff --git a/core/src/test/scala/org/apache/spark/scheduler/MapStatusSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/MapStatusSuite.scala index 950c6dc58e332..464fe1d9047c0 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/MapStatusSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/MapStatusSuite.scala @@ -55,7 +55,7 @@ class MapStatusSuite extends FunSuite { stddev <- Seq(0.0, 0.01, 0.5, 1.0) ) { val sizes = Array.fill[Long](numSizes)(abs(round(Random.nextGaussian() * stddev)) + mean) - val status = MapStatus(BlockManagerId("a", "b", 10), sizes) + val status = MapStatus(BlockManagerId("a", "b", 10), 0, sizes) val status1 = compressAndDecompressMapStatus(status) for (i <- 0 until numSizes) { if (sizes(i) != 0) { @@ -69,8 +69,9 @@ class MapStatusSuite extends FunSuite { test("large tasks should use " + classOf[HighlyCompressedMapStatus].getName) { val sizes = Array.fill[Long](2001)(150L) - val status = MapStatus(null, sizes) + val status = MapStatus(null, 1, sizes) assert(status.isInstanceOf[HighlyCompressedMapStatus]) + assert(status.stageAttemptId === 1) assert(status.getSizeForBlock(10) === 150L) assert(status.getSizeForBlock(50) === 150L) assert(status.getSizeForBlock(99) === 150L) @@ -81,10 +82,11 @@ class MapStatusSuite extends FunSuite { val sizes = Array.tabulate[Long](3000) { i => i.toLong } val avg = sizes.sum / sizes.filter(_ != 0).length val loc = BlockManagerId("a", "b", 10) - val status = MapStatus(loc, sizes) + val status = MapStatus(loc, 1, sizes) val status1 = compressAndDecompressMapStatus(status) assert(status1.isInstanceOf[HighlyCompressedMapStatus]) assert(status1.location == loc) + assert(status1.stageAttemptId === 1) for (i <- 0 until 3000) { val estimate = status1.getSizeForBlock(i) if (sizes(i) > 0) { diff --git a/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala index c7369de24b81f..f850542cd42e4 100644 --- a/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala +++ b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala @@ -293,7 +293,7 @@ class KryoSerializerSuite extends FunSuite with SharedSparkContext { val denseBlockSizes = new Array[Long](5000) val sparseBlockSizes = Array[Long](0L, 1L, 0L, 2L) Seq(denseBlockSizes, sparseBlockSizes).foreach { blockSizes => - ser.serialize(HighlyCompressedMapStatus(BlockManagerId("exec-1", "host", 1234), blockSizes)) + ser.serialize(HighlyCompressedMapStatus(BlockManagerId("exec-1", "host", 1234), 0, blockSizes)) } } diff --git a/core/src/test/scala/org/apache/spark/shuffle/hash/HashShuffleManagerSuite.scala b/core/src/test/scala/org/apache/spark/shuffle/hash/HashShuffleManagerSuite.scala index 0537bf66ad020..f29f796810f92 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/hash/HashShuffleManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/hash/HashShuffleManagerSuite.scala @@ -56,7 +56,7 @@ class HashShuffleManagerSuite extends FunSuite with LocalSparkContext { val shuffleBlockResolver = SparkEnv.get.shuffleManager.shuffleBlockResolver.asInstanceOf[FileShuffleBlockResolver] - val shuffle1 = shuffleBlockResolver.forMapTask(1, 1, 1, new JavaSerializer(conf), + val shuffle1 = shuffleBlockResolver.forMapTask(1, 1, 0, 1, new JavaSerializer(conf), new ShuffleWriteMetrics) for (writer <- shuffle1.writers) { writer.write("test1", "value") @@ -69,7 +69,7 @@ class HashShuffleManagerSuite extends FunSuite with LocalSparkContext { val shuffle1Segment = shuffle1.writers(0).fileSegment() shuffle1.releaseWriters(success = true) - val shuffle2 = shuffleBlockResolver.forMapTask(1, 2, 1, new JavaSerializer(conf), + val shuffle2 = shuffleBlockResolver.forMapTask(1, 2, 0, 1, new JavaSerializer(conf), new ShuffleWriteMetrics) for (writer <- shuffle2.writers) { @@ -88,7 +88,7 @@ class HashShuffleManagerSuite extends FunSuite with LocalSparkContext { // of block based on remaining data in file : which could mess things up when there is // concurrent read and writes happening to the same shuffle group. - val shuffle3 = shuffleBlockResolver.forMapTask(1, 3, 1, new JavaSerializer(testConf), + val shuffle3 = shuffleBlockResolver.forMapTask(1, 3, 0, 1, new JavaSerializer(testConf), new ShuffleWriteMetrics) for (writer <- shuffle3.writers) { writer.write("test3", "value") @@ -98,10 +98,10 @@ class HashShuffleManagerSuite extends FunSuite with LocalSparkContext { writer.commitAndClose() } // check before we register. - checkSegments(shuffle2Segment, shuffleBlockResolver.getBlockData(ShuffleBlockId(1, 2, 0))) + checkSegments(shuffle2Segment, shuffleBlockResolver.getBlockData(ShuffleBlockId(1, 2, 0, 0))) shuffle3.releaseWriters(success = true) - checkSegments(shuffle2Segment, shuffleBlockResolver.getBlockData(ShuffleBlockId(1, 2, 0))) - shuffleBlockResolver.removeShuffle(1) + checkSegments(shuffle2Segment, shuffleBlockResolver.getBlockData(ShuffleBlockId(1, 2, 0, 0))) + shuffleBlockResolver.removeShuffle((1,0)) } def writeToFile(file: File, numBytes: Int) { diff --git a/core/src/test/scala/org/apache/spark/storage/BlockIdSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockIdSuite.scala index b647e8a6728ec..5ecae4eb0cca9 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockIdSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockIdSuite.scala @@ -55,14 +55,16 @@ class BlockIdSuite extends FunSuite { } test("shuffle") { - val id = ShuffleBlockId(1, 2, 3) - assertSame(id, ShuffleBlockId(1, 2, 3)) - assertDifferent(id, ShuffleBlockId(3, 2, 3)) - assert(id.name === "shuffle_1_2_3") + val id = ShuffleBlockId(1, 2, 3, 4) + assertSame(id, ShuffleBlockId(1, 2, 3, 4)) + assertDifferent(id, ShuffleBlockId(3, 2, 3, 4)) + assertDifferent(id, ShuffleBlockId(1, 2, 3, 1)) + assert(id.name === "shuffle_1_2_3_4") assert(id.asRDDId === None) assert(id.shuffleId === 1) assert(id.mapId === 2) assert(id.reduceId === 3) + assert(id.stageAttemptId === 4) assert(id.isShuffle) assertSame(id, BlockId(id.toString)) } diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala index 151955ef7f435..77cd5cdc66857 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala @@ -724,16 +724,16 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfterEach try { conf.set("spark.shuffle.compress", "true") store = makeBlockManager(20000, "exec1") - store.putSingle(ShuffleBlockId(0, 0, 0), new Array[Byte](1000), StorageLevel.MEMORY_ONLY_SER) - assert(store.memoryStore.getSize(ShuffleBlockId(0, 0, 0)) <= 100, + store.putSingle(ShuffleBlockId(0, 0, 0, 0), new Array[Byte](1000), StorageLevel.MEMORY_ONLY_SER) + assert(store.memoryStore.getSize(ShuffleBlockId(0, 0, 0, 0)) <= 100, "shuffle_0_0_0 was not compressed") store.stop() store = null conf.set("spark.shuffle.compress", "false") store = makeBlockManager(20000, "exec2") - store.putSingle(ShuffleBlockId(0, 0, 0), new Array[Byte](10000), StorageLevel.MEMORY_ONLY_SER) - assert(store.memoryStore.getSize(ShuffleBlockId(0, 0, 0)) >= 10000, + store.putSingle(ShuffleBlockId(0, 0, 0, 0), new Array[Byte](10000), StorageLevel.MEMORY_ONLY_SER) + assert(store.memoryStore.getSize(ShuffleBlockId(0, 0, 0, 0)) >= 10000, "shuffle_0_0_0 was compressed") store.stop() store = null diff --git a/core/src/test/scala/org/apache/spark/storage/ShuffleBlockFetcherIteratorSuite.scala b/core/src/test/scala/org/apache/spark/storage/ShuffleBlockFetcherIteratorSuite.scala index 2080c432d77db..afef917bcb034 100644 --- a/core/src/test/scala/org/apache/spark/storage/ShuffleBlockFetcherIteratorSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/ShuffleBlockFetcherIteratorSuite.scala @@ -67,9 +67,9 @@ class ShuffleBlockFetcherIteratorSuite extends FunSuite { // Make sure blockManager.getBlockData would return the blocks val localBlocks = Map[BlockId, ManagedBuffer]( - ShuffleBlockId(0, 0, 0) -> mock(classOf[ManagedBuffer]), - ShuffleBlockId(0, 1, 0) -> mock(classOf[ManagedBuffer]), - ShuffleBlockId(0, 2, 0) -> mock(classOf[ManagedBuffer])) + ShuffleBlockId(0, 0, 0, 0) -> mock(classOf[ManagedBuffer]), + ShuffleBlockId(0, 1, 0, 0) -> mock(classOf[ManagedBuffer]), + ShuffleBlockId(0, 2, 0, 0) -> mock(classOf[ManagedBuffer])) localBlocks.foreach { case (blockId, buf) => doReturn(buf).when(blockManager).getBlockData(meq(blockId)) } @@ -77,8 +77,8 @@ class ShuffleBlockFetcherIteratorSuite extends FunSuite { // Make sure remote blocks would return val remoteBmId = BlockManagerId("test-client-1", "test-client-1", 2) val remoteBlocks = Map[BlockId, ManagedBuffer]( - ShuffleBlockId(0, 3, 0) -> mock(classOf[ManagedBuffer]), - ShuffleBlockId(0, 4, 0) -> mock(classOf[ManagedBuffer]) + ShuffleBlockId(0, 3, 0, 0) -> mock(classOf[ManagedBuffer]), + ShuffleBlockId(0, 4, 0, 0) -> mock(classOf[ManagedBuffer]) ) val transfer = createMockTransfer(remoteBlocks) @@ -126,9 +126,9 @@ class ShuffleBlockFetcherIteratorSuite extends FunSuite { // Make sure remote blocks would return val remoteBmId = BlockManagerId("test-client-1", "test-client-1", 2) val blocks = Map[BlockId, ManagedBuffer]( - ShuffleBlockId(0, 0, 0) -> mock(classOf[ManagedBuffer]), - ShuffleBlockId(0, 1, 0) -> mock(classOf[ManagedBuffer]), - ShuffleBlockId(0, 2, 0) -> mock(classOf[ManagedBuffer]) + ShuffleBlockId(0, 0, 0, 0) -> mock(classOf[ManagedBuffer]), + ShuffleBlockId(0, 1, 0, 0) -> mock(classOf[ManagedBuffer]), + ShuffleBlockId(0, 2, 0, 0) -> mock(classOf[ManagedBuffer]) ) // Semaphore to coordinate event sequence in two different threads. @@ -141,12 +141,12 @@ class ShuffleBlockFetcherIteratorSuite extends FunSuite { future { // Return the first two blocks, and wait till task completion before returning the 3rd one listener.onBlockFetchSuccess( - ShuffleBlockId(0, 0, 0).toString, blocks(ShuffleBlockId(0, 0, 0))) + ShuffleBlockId(0, 0, 0, 0).toString, blocks(ShuffleBlockId(0, 0, 0, 0))) listener.onBlockFetchSuccess( - ShuffleBlockId(0, 1, 0).toString, blocks(ShuffleBlockId(0, 1, 0))) + ShuffleBlockId(0, 1, 0, 0).toString, blocks(ShuffleBlockId(0, 1, 0, 0))) sem.acquire() listener.onBlockFetchSuccess( - ShuffleBlockId(0, 2, 0).toString, blocks(ShuffleBlockId(0, 2, 0))) + ShuffleBlockId(0, 2, 0, 0).toString, blocks(ShuffleBlockId(0, 2, 0, 0))) } } }) @@ -165,20 +165,20 @@ class ShuffleBlockFetcherIteratorSuite extends FunSuite { // Exhaust the first block, and then it should be released. iterator.next()._2.get.foreach(_ => Unit) - verify(blocks(ShuffleBlockId(0, 0, 0)), times(1)).release() + verify(blocks(ShuffleBlockId(0, 0, 0, 0)), times(1)).release() // Get the 2nd block but do not exhaust the iterator val subIter = iterator.next()._2.get // Complete the task; then the 2nd block buffer should be exhausted - verify(blocks(ShuffleBlockId(0, 1, 0)), times(0)).release() + verify(blocks(ShuffleBlockId(0, 1, 0, 0)), times(0)).release() taskContext.markTaskCompleted() - verify(blocks(ShuffleBlockId(0, 1, 0)), times(1)).release() + verify(blocks(ShuffleBlockId(0, 1, 0, 0)), times(1)).release() // The 3rd block should not be retained because the iterator is already in zombie state sem.release() - verify(blocks(ShuffleBlockId(0, 2, 0)), times(0)).retain() - verify(blocks(ShuffleBlockId(0, 2, 0)), times(0)).release() + verify(blocks(ShuffleBlockId(0, 2, 0, 0)), times(0)).retain() + verify(blocks(ShuffleBlockId(0, 2, 0, 0)), times(0)).release() } test("fail all blocks if any of the remote request fails") { @@ -189,9 +189,9 @@ class ShuffleBlockFetcherIteratorSuite extends FunSuite { // Make sure remote blocks would return val remoteBmId = BlockManagerId("test-client-1", "test-client-1", 2) val blocks = Map[BlockId, ManagedBuffer]( - ShuffleBlockId(0, 0, 0) -> mock(classOf[ManagedBuffer]), - ShuffleBlockId(0, 1, 0) -> mock(classOf[ManagedBuffer]), - ShuffleBlockId(0, 2, 0) -> mock(classOf[ManagedBuffer]) + ShuffleBlockId(0, 0, 0, 0) -> mock(classOf[ManagedBuffer]), + ShuffleBlockId(0, 1, 0, 0) -> mock(classOf[ManagedBuffer]), + ShuffleBlockId(0, 2, 0, 0) -> mock(classOf[ManagedBuffer]) ) // Semaphore to coordinate event sequence in two different threads. @@ -204,11 +204,11 @@ class ShuffleBlockFetcherIteratorSuite extends FunSuite { future { // Return the first block, and then fail. listener.onBlockFetchSuccess( - ShuffleBlockId(0, 0, 0).toString, blocks(ShuffleBlockId(0, 0, 0))) + ShuffleBlockId(0, 0, 0, 0).toString, blocks(ShuffleBlockId(0, 0, 0, 0))) listener.onBlockFetchFailure( - ShuffleBlockId(0, 1, 0).toString, new BlockNotFoundException("blah")) + ShuffleBlockId(0, 1, 0, 0).toString, new BlockNotFoundException("blah")) listener.onBlockFetchFailure( - ShuffleBlockId(0, 2, 0).toString, new BlockNotFoundException("blah")) + ShuffleBlockId(0, 2, 0, 0).toString, new BlockNotFoundException("blah")) sem.release() } } diff --git a/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala b/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala index b6f5accef0cef..84aaa089cbdc7 100644 --- a/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala @@ -224,8 +224,10 @@ class UISeleniumSuite extends FunSuite with WebBrowser with Matchers with Before val shuffleId = shuffleHandle.shuffleId val mapId = 0 val reduceId = taskContext.partitionId() + val stageAttemptId = 0 val message = "Simulated fetch failure" - throw new FetchFailedException(bmAddress, shuffleId, mapId, reduceId, message) + throw new FetchFailedException(bmAddress, shuffleId, mapId, reduceId, stageAttemptId, + message) } else { x } diff --git a/core/src/test/scala/org/apache/spark/util/AkkaUtilsSuite.scala b/core/src/test/scala/org/apache/spark/util/AkkaUtilsSuite.scala index bec79fc4dc8f7..b45713bc96a94 100644 --- a/core/src/test/scala/org/apache/spark/util/AkkaUtilsSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/AkkaUtilsSuite.scala @@ -103,7 +103,7 @@ class AkkaUtilsSuite extends FunSuite with LocalSparkContext with ResetSystemPro val size1000 = MapStatus.decompressSize(MapStatus.compressSize(1000L)) masterTracker.registerMapOutput(10, 0, - MapStatus(BlockManagerId("a", "hostA", 1000), Array(1000L))) + MapStatus(BlockManagerId("a", "hostA", 1000), 0, Array(1000L))) masterTracker.incrementEpoch() slaveTracker.updateEpoch(masterTracker.getEpoch) @@ -149,7 +149,7 @@ class AkkaUtilsSuite extends FunSuite with LocalSparkContext with ResetSystemPro val size1000 = MapStatus.decompressSize(MapStatus.compressSize(1000L)) masterTracker.registerMapOutput(10, 0, MapStatus( - BlockManagerId("a", "hostA", 1000), Array(1000L))) + BlockManagerId("a", "hostA", 1000), 0, Array(1000L))) masterTracker.incrementEpoch() slaveTracker.updateEpoch(masterTracker.getEpoch) @@ -228,7 +228,7 @@ class AkkaUtilsSuite extends FunSuite with LocalSparkContext with ResetSystemPro val size1000 = MapStatus.decompressSize(MapStatus.compressSize(1000L)) masterTracker.registerMapOutput(10, 0, - MapStatus(BlockManagerId("a", "hostA", 1000), Array(1000L))) + MapStatus(BlockManagerId("a", "hostA", 1000), 0, Array(1000L))) masterTracker.incrementEpoch() slaveTracker.updateEpoch(masterTracker.getEpoch) @@ -275,7 +275,7 @@ class AkkaUtilsSuite extends FunSuite with LocalSparkContext with ResetSystemPro val size1000 = MapStatus.decompressSize(MapStatus.compressSize(1000L)) masterTracker.registerMapOutput(10, 0, - MapStatus(BlockManagerId("a", "hostA", 1000), Array(1000L))) + MapStatus(BlockManagerId("a", "hostA", 1000), 0, Array(1000L))) masterTracker.incrementEpoch() slaveTracker.updateEpoch(masterTracker.getEpoch) diff --git a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala index 0c5221d10d79d..aca50c6532b93 100644 --- a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala @@ -150,7 +150,7 @@ class JsonProtocolSuite extends FunSuite { // BlockId testBlockId(RDDBlockId(1, 2)) - testBlockId(ShuffleBlockId(1, 2, 3)) + testBlockId(ShuffleBlockId(1, 2, 3, 4)) testBlockId(BroadcastBlockId(1L, "insert_words_of_wisdom_here")) testBlockId(TaskResultBlockId(1L)) testBlockId(StreamBlockId(1, 2L)) From 4d976f4fdf64b1a97a9675e8bf3f5eb4a67da74c Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Tue, 2 Jun 2015 13:18:15 -0500 Subject: [PATCH 21/73] avoid NPE in finally block --- .../org/apache/spark/shuffle/unsafe/UnsafeShuffleWriter.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/java/org/apache/spark/shuffle/unsafe/UnsafeShuffleWriter.java b/core/src/main/java/org/apache/spark/shuffle/unsafe/UnsafeShuffleWriter.java index 1c57d600d217d..a1e278ae3a8b0 100644 --- a/core/src/main/java/org/apache/spark/shuffle/unsafe/UnsafeShuffleWriter.java +++ b/core/src/main/java/org/apache/spark/shuffle/unsafe/UnsafeShuffleWriter.java @@ -150,7 +150,7 @@ public void write(scala.collection.Iterator> records) throws IOEx closeAndWriteOutput(); success = true; } finally { - if (!success) { + if (!success && sorter != null) { sorter.cleanupAfterError(); } } From 2b723fd002fe45be0d6f9e6d87f91fad9a9c0961 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Tue, 2 Jun 2015 13:19:45 -0500 Subject: [PATCH 22/73] use case class for result of mapOutputTracker.getServerStatus --- .../main/scala/org/apache/spark/MapOutputTracker.scala | 9 ++++++--- .../spark/shuffle/hash/BlockStoreShuffleFetcher.scala | 4 ++-- .../src/test/scala/org/apache/spark/ShuffleSuite.scala | 6 +++--- .../org/apache/spark/scheduler/DAGSchedulerSuite.scala | 10 +++++----- 4 files changed, 16 insertions(+), 13 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala index d439c42ce890f..fe79518f08665 100644 --- a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala +++ b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala @@ -36,6 +36,8 @@ private[spark] case class GetMapOutputStatuses(shuffleId: Int) extends MapOutputTrackerMessage private[spark] case object StopMapOutputTracker extends MapOutputTrackerMessage +private[spark] case class MapServerAttemptSize(bmId: BlockManagerId, stageAttempt: Int, size: Long) + /** RpcEndpoint class for MapOutputTrackerMaster */ private[spark] class MapOutputTrackerMasterEndpoint( override val rpcEnv: RpcEnv, tracker: MapOutputTrackerMaster, conf: SparkConf) @@ -127,7 +129,7 @@ private[spark] abstract class MapOutputTracker(conf: SparkConf) extends Logging * Called from executors to get the server URIs and output sizes of the map outputs of * a given shuffle. */ - def getServerStatuses(shuffleId: Int, reduceId: Int): Array[(BlockManagerId, Int, Long)] = { + def getServerStatuses(shuffleId: Int, reduceId: Int): Array[MapServerAttemptSize] = { val statuses = mapStatuses.get(shuffleId).orNull if (statuses == null) { logInfo("Don't have map outputs for shuffle " + shuffleId + ", fetching them") @@ -380,7 +382,7 @@ private[spark] object MapOutputTracker extends Logging { private def convertMapStatuses( shuffleId: Int, reduceId: Int, - statuses: Array[MapStatus]): Array[(BlockManagerId, Int, Long)] = { + statuses: Array[MapStatus]): Array[MapServerAttemptSize] = { assert (statuses != null) statuses.map { status => @@ -390,7 +392,8 @@ private[spark] object MapOutputTracker extends Logging { logError(msg) throw new MetadataFetchFailedException(shuffleId, reduceId, msg) } else { - (status.location, status.stageAttemptId, status.getSizeForBlock(reduceId)) + MapServerAttemptSize( + status.location, status.stageAttemptId, status.getSizeForBlock(reduceId)) } } } diff --git a/core/src/main/scala/org/apache/spark/shuffle/hash/BlockStoreShuffleFetcher.scala b/core/src/main/scala/org/apache/spark/shuffle/hash/BlockStoreShuffleFetcher.scala index c5c934463a7c3..772be3dff232e 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/hash/BlockStoreShuffleFetcher.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/hash/BlockStoreShuffleFetcher.scala @@ -44,7 +44,7 @@ private[hash] object BlockStoreShuffleFetcher extends Logging { shuffleId, reduceId, System.currentTimeMillis - startTime)) val splitsByAddress = new HashMap[BlockManagerId, ArrayBuffer[(Int, Int, Long)]] - for (((address, stageAttempt, size), index) <- statuses.zipWithIndex) { + for ((MapServerAttemptSize(address, stageAttempt, size), index) <- statuses.zipWithIndex) { splitsByAddress.getOrElseUpdate(address, ArrayBuffer()) += ((index, stageAttempt, size)) } @@ -63,7 +63,7 @@ private[hash] object BlockStoreShuffleFetcher extends Logging { case Failure(e) => { blockId match { case ShuffleBlockId(shufId, mapId, _, stageAttempt) => - val address = statuses(mapId.toInt)._1 + val address = statuses(mapId.toInt).bmId throw new FetchFailedException(address, shufId.toInt, mapId.toInt, reduceId, stageAttempt, e) case _ => diff --git a/core/src/test/scala/org/apache/spark/ShuffleSuite.scala b/core/src/test/scala/org/apache/spark/ShuffleSuite.scala index a3bc67726e1b1..e35a894387a8f 100644 --- a/core/src/test/scala/org/apache/spark/ShuffleSuite.scala +++ b/core/src/test/scala/org/apache/spark/ShuffleSuite.scala @@ -67,7 +67,7 @@ abstract class ShuffleSuite extends FunSuite with Matchers with LocalSparkContex // All blocks must have non-zero size (0 until NUM_BLOCKS).foreach { id => val statuses = SparkEnv.get.mapOutputTracker.getServerStatuses(shuffleId, id) - assert(statuses.forall(s => s._2 > 0)) + assert(statuses.forall(s => s.size > 0)) } } @@ -106,7 +106,7 @@ abstract class ShuffleSuite extends FunSuite with Matchers with LocalSparkContex val blockSizes = (0 until NUM_BLOCKS).flatMap { id => val statuses = SparkEnv.get.mapOutputTracker.getServerStatuses(shuffleId, id) - statuses.map(x => x._2) + statuses.map(x => x.size) } val nonEmptyBlocks = blockSizes.filter(x => x > 0) @@ -131,7 +131,7 @@ abstract class ShuffleSuite extends FunSuite with Matchers with LocalSparkContex val blockSizes = (0 until NUM_BLOCKS).flatMap { id => val statuses = SparkEnv.get.mapOutputTracker.getServerStatuses(shuffleId, id) - statuses.map(x => x._2) + statuses.map(x => x.size) } val nonEmptyBlocks = blockSizes.filter(x => x > 0) diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala index 220a9540aa09e..38ad49090a7f4 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -450,7 +450,7 @@ class DAGSchedulerSuite complete(taskSets(0), Seq( (Success, makeMapStatus("hostA", 1)), (Success, makeMapStatus("hostB", 1)))) - assert(mapOutputTracker.getServerStatuses(shuffleId, 0).map(_._1) === + assert(mapOutputTracker.getServerStatuses(shuffleId, 0).map(_.bmId) === Array(makeBlockManagerId("hostA"), makeBlockManagerId("hostB"))) complete(taskSets(1), Seq((Success, 42))) assert(results === Map(0 -> 42)) @@ -477,7 +477,7 @@ class DAGSchedulerSuite // have the 2nd attempt pass complete(taskSets(2), Seq((Success, makeMapStatus("hostA", 1)))) // we can see both result blocks now - assert(mapOutputTracker.getServerStatuses(shuffleId, 0).map(_._1.host) === + assert(mapOutputTracker.getServerStatuses(shuffleId, 0).map(_.bmId.host) === Array("hostA", "hostB")) complete(taskSets(3), Seq((Success, 43))) assert(results === Map(0 -> 42, 1 -> 43)) @@ -494,7 +494,7 @@ class DAGSchedulerSuite (Success, makeMapStatus("hostA", 1)), (Success, makeMapStatus("hostB", 1)))) // The MapOutputTracker should know about both map output locations. - assert(mapOutputTracker.getServerStatuses(shuffleId, 0).map(_._1.host) === + assert(mapOutputTracker.getServerStatuses(shuffleId, 0).map(_.bmId.host) === Array("hostA", "hostB")) // The first result task fails, with a fetch failure for the output from the first mapper. @@ -558,7 +558,7 @@ class DAGSchedulerSuite runEvent(CompletionEvent( newTask, Success, makeMapStatus("hostB", 1), null, createFakeTaskInfo(), null)) - assert(mapOutputTracker.getServerStatuses(shuffleId, 0).map(_._1) === + assert(mapOutputTracker.getServerStatuses(shuffleId, 0).map(_.bmId) === Array(makeBlockManagerId("hostB"), makeBlockManagerId("hostA"))) complete(taskSets(2), Seq((Success, 42), (Success, 43))) assert(results === Map(0 -> 42, 1 -> 43)) @@ -654,7 +654,7 @@ class DAGSchedulerSuite (Success, makeMapStatus("hostB", 1)))) // have hostC complete the resubmitted task complete(taskSets(1), Seq((Success, makeMapStatus("hostC", 1)))) - assert(mapOutputTracker.getServerStatuses(shuffleId, 0).map(_._1) === + assert(mapOutputTracker.getServerStatuses(shuffleId, 0).map(_.bmId) === Array(makeBlockManagerId("hostC"), makeBlockManagerId("hostB"))) complete(taskSets(2), Seq((Success, 42))) assert(results === Map(0 -> 42)) From fd40a93de16be76639cb2c4a89e60a0103e58677 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Tue, 2 Jun 2015 13:19:59 -0500 Subject: [PATCH 23/73] fix tests --- .../apache/spark/shuffle/unsafe/UnsafeShuffleWriterSuite.java | 2 +- .../main/scala/org/apache/spark/tools/StoragePerfTester.scala | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/test/java/org/apache/spark/shuffle/unsafe/UnsafeShuffleWriterSuite.java b/core/src/test/java/org/apache/spark/shuffle/unsafe/UnsafeShuffleWriterSuite.java index a08c90d6473a7..4604e46941d7e 100644 --- a/core/src/test/java/org/apache/spark/shuffle/unsafe/UnsafeShuffleWriterSuite.java +++ b/core/src/test/java/org/apache/spark/shuffle/unsafe/UnsafeShuffleWriterSuite.java @@ -173,7 +173,7 @@ public OutputStream answer(InvocationOnMock invocation) throws Throwable { doAnswer(new Answer() { @Override public Void answer(InvocationOnMock invocationOnMock) throws Throwable { - partitionSizesInMergedFile = (long[]) invocationOnMock.getArguments()[2]; + partitionSizesInMergedFile = (long[]) invocationOnMock.getArguments()[3]; return null; } }).when(shuffleBlockResolver).writeIndexFile(anyInt(), anyInt(), anyInt(), any(long[].class)); 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 baa97616eaff3..ab0e734aa7438 100644 --- a/tools/src/main/scala/org/apache/spark/tools/StoragePerfTester.scala +++ b/tools/src/main/scala/org/apache/spark/tools/StoragePerfTester.scala @@ -60,7 +60,7 @@ object StoragePerfTester { val hashShuffleManager = sc.env.shuffleManager.asInstanceOf[HashShuffleManager] def writeOutputBytes(mapId: Int, total: AtomicLong): Unit = { - val shuffle = hashShuffleManager.shuffleBlockResolver.forMapTask(1, mapId, numOutputSplits, + val shuffle = hashShuffleManager.shuffleBlockResolver.forMapTask(1, mapId, 0, numOutputSplits, new KryoSerializer(sc.conf), new ShuffleWriteMetrics()) val writers = shuffle.writers for (i <- 1 to recordsPerMap) { From 9f01d7ea510f9ed4e42ec5eafb847dfef5f24aed Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Tue, 2 Jun 2015 14:13:23 -0500 Subject: [PATCH 24/73] style --- .../org/apache/spark/shuffle/ShuffleBlockResolver.scala | 3 +-- .../scala/org/apache/spark/shuffle/ShuffleManager.scala | 2 +- .../apache/spark/shuffle/unsafe/UnsafeShuffleManager.scala | 2 +- core/src/main/scala/org/apache/spark/storage/BlockId.scala | 3 ++- .../org/apache/spark/serializer/KryoSerializerSuite.scala | 3 ++- .../apache/spark/shuffle/hash/HashShuffleManagerSuite.scala | 2 +- .../scala/org/apache/spark/storage/BlockManagerSuite.scala | 6 ++++-- 7 files changed, 12 insertions(+), 9 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/shuffle/ShuffleBlockResolver.scala b/core/src/main/scala/org/apache/spark/shuffle/ShuffleBlockResolver.scala index 36a7873859e8f..e65738836c0a9 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/ShuffleBlockResolver.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/ShuffleBlockResolver.scala @@ -17,7 +17,6 @@ package org.apache.spark.shuffle -import java.nio.ByteBuffer import org.apache.spark.network.buffer.ManagedBuffer import org.apache.spark.storage.ShuffleBlockId @@ -29,7 +28,7 @@ private[spark] * implementations when shuffle data is retrieved. */ trait ShuffleBlockResolver { - type ShuffleId = (Int,Int) + type ShuffleId = (Int, Int) /** * Retrieve the data for the specified block. If the data for that block is not available, diff --git a/core/src/main/scala/org/apache/spark/shuffle/ShuffleManager.scala b/core/src/main/scala/org/apache/spark/shuffle/ShuffleManager.scala index 1b7865198de80..cc9c25dcdea79 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/ShuffleManager.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/ShuffleManager.scala @@ -73,7 +73,7 @@ private[spark] trait ShuffleManager { private[this] val shuffleToAttempts = new ConcurrentHashMap[Int, ConcurrentHashMap[Int, Int]]() protected def addShuffleAttempt(shuffleId: Int, stageAttemptId: Int): Unit = { - shuffleToAttempts.putIfAbsent(shuffleId, new ConcurrentHashMap[Int,Int]()) + shuffleToAttempts.putIfAbsent(shuffleId, new ConcurrentHashMap[Int, Int]()) shuffleToAttempts.get(shuffleId).putIfAbsent(stageAttemptId, stageAttemptId) } protected def stageAttemptsForShuffle(shuffleId: Int): Iterable[Int] = { diff --git a/core/src/main/scala/org/apache/spark/shuffle/unsafe/UnsafeShuffleManager.scala b/core/src/main/scala/org/apache/spark/shuffle/unsafe/UnsafeShuffleManager.scala index 4baac3bcb8c9b..2f9ef995e03ed 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/unsafe/UnsafeShuffleManager.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/unsafe/UnsafeShuffleManager.scala @@ -160,7 +160,7 @@ private[spark] class UnsafeShuffleManager(conf: SparkConf) extends ShuffleManage override def getWriter[K, V]( handle: ShuffleHandle, mapId: Int, - stageAttemptId:Int, + stageAttemptId: Int, context: TaskContext): ShuffleWriter[K, V] = { handle match { case unsafeShuffleHandle: UnsafeShuffleHandle[K, V] => 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 bf713521eb807..63c28e43c308d 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockId.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockId.scala @@ -72,7 +72,8 @@ case class ShuffleDataBlockId(shuffleId: Int, mapId: Int, stageAttemptId: Int, r } @DeveloperApi -case class ShuffleIndexBlockId(shuffleId: Int, mapId: Int, stageAttemptId: Int, reduceId: Int) extends BlockId { +case class ShuffleIndexBlockId(shuffleId: Int, mapId: Int, stageAttemptId: Int, reduceId: Int) + extends BlockId { override def name: String = { "shuffle_" + shuffleId + "_" + mapId + "_" + reduceId + "_" + stageAttemptId + ".index" } diff --git a/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala index 24e43bcb62068..bf6fa7f34ce6c 100644 --- a/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala +++ b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala @@ -301,7 +301,8 @@ class KryoSerializerSuite extends SparkFunSuite with SharedSparkContext { val denseBlockSizes = new Array[Long](5000) val sparseBlockSizes = Array[Long](0L, 1L, 0L, 2L) Seq(denseBlockSizes, sparseBlockSizes).foreach { blockSizes => - ser.serialize(HighlyCompressedMapStatus(BlockManagerId("exec-1", "host", 1234), 0, blockSizes)) + ser.serialize( + HighlyCompressedMapStatus(BlockManagerId("exec-1", "host", 1234), 0, blockSizes)) } } diff --git a/core/src/test/scala/org/apache/spark/shuffle/hash/HashShuffleManagerSuite.scala b/core/src/test/scala/org/apache/spark/shuffle/hash/HashShuffleManagerSuite.scala index cb37b898d876a..0b6753b736e66 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/hash/HashShuffleManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/hash/HashShuffleManagerSuite.scala @@ -99,7 +99,7 @@ class HashShuffleManagerSuite extends SparkFunSuite with LocalSparkContext { checkSegments(shuffle2Segment, shuffleBlockResolver.getBlockData(ShuffleBlockId(1, 2, 0, 0))) shuffle3.releaseWriters(success = true) checkSegments(shuffle2Segment, shuffleBlockResolver.getBlockData(ShuffleBlockId(1, 2, 0, 0))) - shuffleBlockResolver.removeShuffle((1,0)) + shuffleBlockResolver.removeShuffle((1, 0)) } def writeToFile(file: File, numBytes: Int) { diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala index d52271baa9eba..6df2d73835e2e 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala @@ -724,7 +724,8 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE try { conf.set("spark.shuffle.compress", "true") store = makeBlockManager(20000, "exec1") - store.putSingle(ShuffleBlockId(0, 0, 0, 0), new Array[Byte](1000), StorageLevel.MEMORY_ONLY_SER) + store.putSingle( + ShuffleBlockId(0, 0, 0, 0), new Array[Byte](1000), StorageLevel.MEMORY_ONLY_SER) assert(store.memoryStore.getSize(ShuffleBlockId(0, 0, 0, 0)) <= 100, "shuffle_0_0_0 was not compressed") store.stop() @@ -732,7 +733,8 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE conf.set("spark.shuffle.compress", "false") store = makeBlockManager(20000, "exec2") - store.putSingle(ShuffleBlockId(0, 0, 0, 0), new Array[Byte](10000), StorageLevel.MEMORY_ONLY_SER) + store.putSingle( + ShuffleBlockId(0, 0, 0, 0), new Array[Byte](10000), StorageLevel.MEMORY_ONLY_SER) assert(store.memoryStore.getSize(ShuffleBlockId(0, 0, 0, 0)) >= 10000, "shuffle_0_0_0 was compressed") store.stop() From fae9c0c604dd55ee3d0fefdadb40b632554e4b85 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Tue, 2 Jun 2015 14:22:07 -0500 Subject: [PATCH 25/73] style --- core/src/main/scala/org/apache/spark/scheduler/MapStatus.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/MapStatus.scala b/core/src/main/scala/org/apache/spark/scheduler/MapStatus.scala index a5cb9d3d39e8c..5e562c65c351d 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/MapStatus.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/MapStatus.scala @@ -106,7 +106,7 @@ private[spark] class CompressedMapStatus( override def location: BlockManagerId = loc - override def stageAttemptId = _stageAttemptId + override def stageAttemptId: Int = _stageAttemptId override def getSizeForBlock(reduceId: Int): Long = { MapStatus.decompressSize(compressedSizes(reduceId)) From 06daceb2e027d64b975b29c2dac7385942fa8101 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Tue, 2 Jun 2015 16:56:55 -0500 Subject: [PATCH 26/73] make ContextCleanerSuite pass ... though maybe the test is pointless --- .../scala/org/apache/spark/shuffle/ShuffleManager.scala | 6 +++++- .../test/scala/org/apache/spark/ContextCleanerSuite.scala | 3 +++ 2 files changed, 8 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/shuffle/ShuffleManager.scala b/core/src/main/scala/org/apache/spark/shuffle/ShuffleManager.scala index cc9c25dcdea79..2123e19a68d64 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/ShuffleManager.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/ShuffleManager.scala @@ -78,6 +78,10 @@ private[spark] trait ShuffleManager { } protected def stageAttemptsForShuffle(shuffleId: Int): Iterable[Int] = { val attempts = shuffleToAttempts.get(shuffleId) - attempts.values.asScala + if (attempts == null) { + Iterable[Int]() + } else { + attempts.values().asScala + } } } diff --git a/core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala b/core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala index c6449549df52b..0b85fbe216fb3 100644 --- a/core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala +++ b/core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala @@ -398,6 +398,9 @@ class SortShuffleContextCleanerSuite extends ContextCleanerSuiteBase(classOf[Sor runGC() postGCTester.assertCleanup() + // TODO I'm not convinced this is actually testing shuffle cleanup completely -- add more + // tests (for hash, shuffle, and unsafe) before we commit this + // Make sure the broadcasted task closure no longer exists after GC. val taskClosureBroadcastId = broadcastIds.max + 1 assert(sc.env.blockManager.master.getMatchingBlockIds({ From cd16ee869e6374931b6dfbb90f33044f4579b7a7 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Tue, 2 Jun 2015 17:06:48 -0500 Subject: [PATCH 27/73] fix tests --- .../scala/org/apache/spark/MapOutputTrackerSuite.scala | 7 ++++--- .../test/scala/org/apache/spark/util/AkkaUtilsSuite.scala | 8 ++++---- 2 files changed, 8 insertions(+), 7 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala b/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala index df78a79871f60..4d2510ff9137a 100644 --- a/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala +++ b/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala @@ -56,8 +56,9 @@ class MapOutputTrackerSuite extends SparkFunSuite { tracker.registerMapOutput(10, 1, MapStatus(BlockManagerId("b", "hostB", 1000), 0, Array(10000L, 1000L))) val statuses = tracker.getServerStatuses(10, 0) - assert(statuses.toSeq === Seq((BlockManagerId("a", "hostA", 1000), size1000), - (BlockManagerId("b", "hostB", 1000), size10000))) + assert(statuses.toSeq === Seq( + MapServerAttemptSize(BlockManagerId("a", "hostA", 1000), 0, size1000), + MapServerAttemptSize(BlockManagerId("b", "hostB", 1000), 0, size10000))) tracker.stop() rpcEnv.shutdown() } @@ -134,7 +135,7 @@ class MapOutputTrackerSuite extends SparkFunSuite { masterTracker.incrementEpoch() slaveTracker.updateEpoch(masterTracker.getEpoch) assert(slaveTracker.getServerStatuses(10, 0).toSeq === - Seq((BlockManagerId("a", "hostA", 1000), size1000))) + Seq(MapServerAttemptSize(BlockManagerId("a", "hostA", 1000), 0, size1000))) masterTracker.unregisterMapOutput(10, 0, BlockManagerId("a", "hostA", 1000)) masterTracker.incrementEpoch() diff --git a/core/src/test/scala/org/apache/spark/util/AkkaUtilsSuite.scala b/core/src/test/scala/org/apache/spark/util/AkkaUtilsSuite.scala index 87071bf7b0c2c..2dd414037311a 100644 --- a/core/src/test/scala/org/apache/spark/util/AkkaUtilsSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/AkkaUtilsSuite.scala @@ -108,7 +108,7 @@ class AkkaUtilsSuite extends SparkFunSuite with LocalSparkContext with ResetSyst // this should succeed since security off assert(slaveTracker.getServerStatuses(10, 0).toSeq === - Seq((BlockManagerId("a", "hostA", 1000), size1000))) + Seq(MapServerAttemptSize(BlockManagerId("a", "hostA", 1000), 0, size1000))) rpcEnv.shutdown() slaveRpcEnv.shutdown() @@ -154,7 +154,7 @@ class AkkaUtilsSuite extends SparkFunSuite with LocalSparkContext with ResetSyst // this should succeed since security on and passwords match assert(slaveTracker.getServerStatuses(10, 0).toSeq === - Seq((BlockManagerId("a", "hostA", 1000), size1000))) + Seq(MapServerAttemptSize(BlockManagerId("a", "hostA", 1000), 0, size1000))) rpcEnv.shutdown() slaveRpcEnv.shutdown() @@ -233,7 +233,7 @@ class AkkaUtilsSuite extends SparkFunSuite with LocalSparkContext with ResetSyst // this should succeed since security off assert(slaveTracker.getServerStatuses(10, 0).toSeq === - Seq((BlockManagerId("a", "hostA", 1000), size1000))) + Seq(MapServerAttemptSize(BlockManagerId("a", "hostA", 1000), 0, size1000))) rpcEnv.shutdown() slaveRpcEnv.shutdown() @@ -279,7 +279,7 @@ class AkkaUtilsSuite extends SparkFunSuite with LocalSparkContext with ResetSyst slaveTracker.updateEpoch(masterTracker.getEpoch) assert(slaveTracker.getServerStatuses(10, 0).toSeq === - Seq((BlockManagerId("a", "hostA", 1000), size1000))) + Seq(MapServerAttemptSize(BlockManagerId("a", "hostA", 1000), 0, size1000))) rpcEnv.shutdown() slaveRpcEnv.shutdown() From 2006de892d2c46b84bf9f3eb9a3371f9689ea50a Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Tue, 2 Jun 2015 17:35:30 -0500 Subject: [PATCH 28/73] ExternalShuffleService use stageAttemptId in shuffle block id --- .../apache/spark/ExternalShuffleServiceSuite.scala | 3 +++ .../shuffle/ExternalShuffleBlockResolver.java | 13 +++++++------ 2 files changed, 10 insertions(+), 6 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/ExternalShuffleServiceSuite.scala b/core/src/test/scala/org/apache/spark/ExternalShuffleServiceSuite.scala index bac6fdbcdc976..eaedc43aef4b3 100644 --- a/core/src/test/scala/org/apache/spark/ExternalShuffleServiceSuite.scala +++ b/core/src/test/scala/org/apache/spark/ExternalShuffleServiceSuite.scala @@ -70,5 +70,8 @@ class ExternalShuffleServiceSuite extends ShuffleSuite with BeforeAndAfterAll { rdd.count() } e.getMessage should include ("Fetch failure will not retry stage due to testing config") + + // TODO make sure we have some tests with stage retry (maybe just inherited from ShuffleSuite) + } } diff --git a/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java b/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java index dd08e24cade23..1a09df5649f77 100644 --- a/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java +++ b/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java @@ -91,7 +91,7 @@ public void registerExecutor( */ public ManagedBuffer getBlockData(String appId, String execId, String blockId) { String[] blockIdParts = blockId.split("_"); - if (blockIdParts.length < 4) { + if (blockIdParts.length != 5) { throw new IllegalArgumentException("Unexpected block id format: " + blockId); } else if (!blockIdParts[0].equals("shuffle")) { throw new IllegalArgumentException("Expected shuffle block id, got: " + blockId); @@ -99,6 +99,7 @@ public ManagedBuffer getBlockData(String appId, String execId, String blockId) { int shuffleId = Integer.parseInt(blockIdParts[1]); int mapId = Integer.parseInt(blockIdParts[2]); int reduceId = Integer.parseInt(blockIdParts[3]); + int stageAttemptId = Integer.parseInt(blockIdParts[4]); ExecutorShuffleInfo executor = executors.get(new AppExecId(appId, execId)); if (executor == null) { @@ -109,7 +110,7 @@ public ManagedBuffer getBlockData(String appId, String execId, String blockId) { if ("org.apache.spark.shuffle.hash.HashShuffleManager".equals(executor.shuffleManager)) { return getHashBasedShuffleBlockData(executor, blockId); } else if ("org.apache.spark.shuffle.sort.SortShuffleManager".equals(executor.shuffleManager)) { - return getSortBasedShuffleBlockData(executor, shuffleId, mapId, reduceId); + return getSortBasedShuffleBlockData(executor, shuffleId, mapId, reduceId, stageAttemptId); } else { throw new UnsupportedOperationException( "Unsupported shuffle manager: " + executor.shuffleManager); @@ -182,9 +183,10 @@ private ManagedBuffer getHashBasedShuffleBlockData(ExecutorShuffleInfo executor, * and the block id format is from ShuffleDataBlockId and ShuffleIndexBlockId. */ private ManagedBuffer getSortBasedShuffleBlockData( - ExecutorShuffleInfo executor, int shuffleId, int mapId, int reduceId) { + ExecutorShuffleInfo executor, int shuffleId, int mapId, int reduceId, int stageAttemptId) { + String baseFileName = "shuffle_" + shuffleId + "_" + mapId + "_0_" + stageAttemptId; File indexFile = getFile(executor.localDirs, executor.subDirsPerLocalDir, - "shuffle_" + shuffleId + "_" + mapId + "_0.index"); + baseFileName + ".index"); DataInputStream in = null; try { @@ -194,8 +196,7 @@ private ManagedBuffer getSortBasedShuffleBlockData( long nextOffset = in.readLong(); return new FileSegmentManagedBuffer( conf, - getFile(executor.localDirs, executor.subDirsPerLocalDir, - "shuffle_" + shuffleId + "_" + mapId + "_0.data"), + getFile(executor.localDirs, executor.subDirsPerLocalDir, baseFileName + ".data"), offset, nextOffset - offset); } catch (IOException e) { From e905f6d24cded32ce8d5849194b01881e5b8f15c Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Tue, 2 Jun 2015 17:35:40 -0500 Subject: [PATCH 29/73] better error msg (just for debugging tests) --- .../main/scala/org/apache/spark/scheduler/DAGScheduler.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index f60f45a4046fe..fbeac2626e9a0 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -1138,7 +1138,8 @@ class DAGScheduler( } if (disallowStageRetryForTest) { - abortStage(failedStage, "Fetch failure will not retry stage due to testing config") + abortStage(failedStage, + s"Fetch failure will not retry stage due to testing config. Failure = $failureMessage") } else if (failedStages.isEmpty) { // Don't schedule an event to resubmit failed stages if failed isn't empty, because // in that case the event will already have been scheduled. From 86e651c12d70724b6fa11165427918fc04434135 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Tue, 2 Jun 2015 19:45:15 -0500 Subject: [PATCH 30/73] fix ExternalShuffleService tests for new block id format (need to add more tests) --- .../ExternalShuffleBlockResolverSuite.java | 29 ++++++++---- .../shuffle/ExternalShuffleCleanupSuite.java | 4 +- .../ExternalShuffleIntegrationSuite.java | 44 ++++++++++--------- .../shuffle/TestShuffleDataContext.java | 10 +++-- 4 files changed, 52 insertions(+), 35 deletions(-) diff --git a/network/shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolverSuite.java b/network/shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolverSuite.java index d02f4f0fdb682..e7a82bce3240c 100644 --- a/network/shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolverSuite.java +++ b/network/shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolverSuite.java @@ -47,9 +47,9 @@ public static void beforeAll() throws IOException { dataContext.create(); // Write some sort and hash data. - dataContext.insertSortShuffleData(0, 0, + dataContext.insertSortShuffleData(0, 0, 0, new byte[][] { sortBlock0.getBytes(), sortBlock1.getBytes() } ); - dataContext.insertHashShuffleData(1, 0, + dataContext.insertHashShuffleData(1, 0, 0, new byte[][] { hashBlock0.getBytes(), hashBlock1.getBytes() } ); } @@ -63,7 +63,7 @@ public void testBadRequests() { ExternalShuffleBlockResolver resolver = new ExternalShuffleBlockResolver(conf); // Unregistered executor try { - resolver.getBlockData("app0", "exec1", "shuffle_1_1_0"); + resolver.getBlockData("app0", "exec1", "shuffle_1_1_0_0"); fail("Should have failed"); } catch (RuntimeException e) { assertTrue("Bad error message: " + e, e.getMessage().contains("not registered")); @@ -72,7 +72,7 @@ public void testBadRequests() { // Invalid shuffle manager resolver.registerExecutor("app0", "exec2", dataContext.createExecutorInfo("foobar")); try { - resolver.getBlockData("app0", "exec2", "shuffle_1_1_0"); + resolver.getBlockData("app0", "exec2", "shuffle_1_1_0_0"); fail("Should have failed"); } catch (UnsupportedOperationException e) { // pass @@ -82,11 +82,20 @@ public void testBadRequests() { resolver.registerExecutor("app0", "exec3", dataContext.createExecutorInfo("org.apache.spark.shuffle.sort.SortShuffleManager")); try { - resolver.getBlockData("app0", "exec3", "shuffle_1_1_0"); + resolver.getBlockData("app0", "exec3", "shuffle_1_1_0_0"); fail("Should have failed"); } catch (Exception e) { // pass } + + // no stageAttemptId + try { + resolver.getBlockData("app0", "exec1", "shuffle_1_1_0"); + fail("Should have failed"); + } catch (RuntimeException e) { + assertTrue("Bad error message: " + e, e.getMessage().contains("Unexpected block id format")); + } + } @Test @@ -96,16 +105,18 @@ public void testSortShuffleBlocks() throws IOException { dataContext.createExecutorInfo("org.apache.spark.shuffle.sort.SortShuffleManager")); InputStream block0Stream = - resolver.getBlockData("app0", "exec0", "shuffle_0_0_0").createInputStream(); + resolver.getBlockData("app0", "exec0", "shuffle_0_0_0_0").createInputStream(); String block0 = CharStreams.toString(new InputStreamReader(block0Stream)); block0Stream.close(); assertEquals(sortBlock0, block0); InputStream block1Stream = - resolver.getBlockData("app0", "exec0", "shuffle_0_0_1").createInputStream(); + resolver.getBlockData("app0", "exec0", "shuffle_0_0_1_0").createInputStream(); String block1 = CharStreams.toString(new InputStreamReader(block1Stream)); block1Stream.close(); assertEquals(sortBlock1, block1); + + // TODO test reading from a different stage attempt } @Test @@ -115,13 +126,13 @@ public void testHashShuffleBlocks() throws IOException { dataContext.createExecutorInfo("org.apache.spark.shuffle.hash.HashShuffleManager")); InputStream block0Stream = - resolver.getBlockData("app0", "exec0", "shuffle_1_0_0").createInputStream(); + resolver.getBlockData("app0", "exec0", "shuffle_1_0_0_0").createInputStream(); String block0 = CharStreams.toString(new InputStreamReader(block0Stream)); block0Stream.close(); assertEquals(hashBlock0, block0); InputStream block1Stream = - resolver.getBlockData("app0", "exec0", "shuffle_1_0_1").createInputStream(); + resolver.getBlockData("app0", "exec0", "shuffle_1_0_1_0").createInputStream(); String block1 = CharStreams.toString(new InputStreamReader(block1Stream)); block1Stream.close(); assertEquals(hashBlock1, block1); diff --git a/network/shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleCleanupSuite.java b/network/shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleCleanupSuite.java index d9d9c1bf2f17a..678fba7d9faa8 100644 --- a/network/shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleCleanupSuite.java +++ b/network/shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleCleanupSuite.java @@ -139,10 +139,10 @@ private TestShuffleDataContext createSomeData() throws IOException { TestShuffleDataContext dataContext = new TestShuffleDataContext(10, 5); dataContext.create(); - dataContext.insertSortShuffleData(rand.nextInt(1000), rand.nextInt(1000), + dataContext.insertSortShuffleData(rand.nextInt(1000), rand.nextInt(1000), rand.nextInt(1000), new byte[][] { "ABC".getBytes(), "DEF".getBytes() } ); dataContext.insertHashShuffleData(rand.nextInt(1000), rand.nextInt(1000) + 1000, - new byte[][] { "GHI".getBytes(), "JKLMNOPQRSTUVWXYZ".getBytes() } ); + rand.nextInt(1000), new byte[][] { "GHI".getBytes(), "JKLMNOPQRSTUVWXYZ".getBytes() } ); return dataContext; } } diff --git a/network/shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleIntegrationSuite.java b/network/shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleIntegrationSuite.java index 39aa49911d9cb..21f9e9e26a929 100644 --- a/network/shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleIntegrationSuite.java +++ b/network/shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleIntegrationSuite.java @@ -85,11 +85,13 @@ public static void beforeAll() throws IOException { dataContext0 = new TestShuffleDataContext(2, 5); dataContext0.create(); - dataContext0.insertSortShuffleData(0, 0, exec0Blocks); + dataContext0.insertSortShuffleData(0, 0, 0, exec0Blocks); dataContext1 = new TestShuffleDataContext(6, 2); dataContext1.create(); - dataContext1.insertHashShuffleData(1, 0, exec1Blocks); + dataContext1.insertHashShuffleData(1, 0, 0, exec1Blocks); + + // TODO tests w/ different stage attempts conf = new TransportConf(new SystemPropertyConfigProvider()); handler = new ExternalShuffleBlockHandler(conf); @@ -173,19 +175,21 @@ public void onBlockFetchFailure(String blockId, Throwable exception) { @Test public void testFetchOneSort() throws Exception { registerExecutor("exec-0", dataContext0.createExecutorInfo(SORT_MANAGER)); - FetchResult exec0Fetch = fetchBlocks("exec-0", new String[] { "shuffle_0_0_0" }); - assertEquals(Sets.newHashSet("shuffle_0_0_0"), exec0Fetch.successBlocks); + FetchResult exec0Fetch = fetchBlocks("exec-0", new String[] { "shuffle_0_0_0_0" }); + assertEquals(Sets.newHashSet("shuffle_0_0_0_0"), exec0Fetch.successBlocks); assertTrue(exec0Fetch.failedBlocks.isEmpty()); assertBufferListsEqual(exec0Fetch.buffers, Lists.newArrayList(exec0Blocks[0])); exec0Fetch.releaseBuffers(); + + //TODO test fetch w/ non-zero stage attempt id } @Test public void testFetchThreeSort() throws Exception { registerExecutor("exec-0", dataContext0.createExecutorInfo(SORT_MANAGER)); FetchResult exec0Fetch = fetchBlocks("exec-0", - new String[] { "shuffle_0_0_0", "shuffle_0_0_1", "shuffle_0_0_2" }); - assertEquals(Sets.newHashSet("shuffle_0_0_0", "shuffle_0_0_1", "shuffle_0_0_2"), + new String[] { "shuffle_0_0_0_0", "shuffle_0_0_1_0", "shuffle_0_0_2_0" }); + assertEquals(Sets.newHashSet("shuffle_0_0_0_0", "shuffle_0_0_1_0", "shuffle_0_0_2_0"), exec0Fetch.successBlocks); assertTrue(exec0Fetch.failedBlocks.isEmpty()); assertBufferListsEqual(exec0Fetch.buffers, Lists.newArrayList(exec0Blocks)); @@ -196,8 +200,8 @@ public void testFetchThreeSort() throws Exception { public void testFetchHash() throws Exception { registerExecutor("exec-1", dataContext1.createExecutorInfo(HASH_MANAGER)); FetchResult execFetch = fetchBlocks("exec-1", - new String[] { "shuffle_1_0_0", "shuffle_1_0_1" }); - assertEquals(Sets.newHashSet("shuffle_1_0_0", "shuffle_1_0_1"), execFetch.successBlocks); + new String[] { "shuffle_1_0_0_0", "shuffle_1_0_1_0" }); + assertEquals(Sets.newHashSet("shuffle_1_0_0_0", "shuffle_1_0_1_0"), execFetch.successBlocks); assertTrue(execFetch.failedBlocks.isEmpty()); assertBufferListsEqual(execFetch.buffers, Lists.newArrayList(exec1Blocks)); execFetch.releaseBuffers(); @@ -207,18 +211,18 @@ public void testFetchHash() throws Exception { public void testFetchWrongShuffle() throws Exception { registerExecutor("exec-1", dataContext1.createExecutorInfo(SORT_MANAGER /* wrong manager */)); FetchResult execFetch = fetchBlocks("exec-1", - new String[] { "shuffle_1_0_0", "shuffle_1_0_1" }); + new String[] { "shuffle_1_0_0_0", "shuffle_1_0_1_0" }); assertTrue(execFetch.successBlocks.isEmpty()); - assertEquals(Sets.newHashSet("shuffle_1_0_0", "shuffle_1_0_1"), execFetch.failedBlocks); + assertEquals(Sets.newHashSet("shuffle_1_0_0_0", "shuffle_1_0_1_0"), execFetch.failedBlocks); } @Test public void testFetchInvalidShuffle() throws Exception { registerExecutor("exec-1", dataContext1.createExecutorInfo("unknown sort manager")); FetchResult execFetch = fetchBlocks("exec-1", - new String[] { "shuffle_1_0_0" }); + new String[] { "shuffle_1_0_0_0" }); assertTrue(execFetch.successBlocks.isEmpty()); - assertEquals(Sets.newHashSet("shuffle_1_0_0"), execFetch.failedBlocks); + assertEquals(Sets.newHashSet("shuffle_1_0_0_0"), execFetch.failedBlocks); } @Test @@ -234,28 +238,28 @@ public void testFetchWrongBlockId() throws Exception { public void testFetchNonexistent() throws Exception { registerExecutor("exec-0", dataContext0.createExecutorInfo(SORT_MANAGER)); FetchResult execFetch = fetchBlocks("exec-0", - new String[] { "shuffle_2_0_0" }); + new String[] { "shuffle_2_0_0_0" }); assertTrue(execFetch.successBlocks.isEmpty()); - assertEquals(Sets.newHashSet("shuffle_2_0_0"), execFetch.failedBlocks); + assertEquals(Sets.newHashSet("shuffle_2_0_0_0"), execFetch.failedBlocks); } @Test public void testFetchWrongExecutor() throws Exception { registerExecutor("exec-0", dataContext0.createExecutorInfo(SORT_MANAGER)); FetchResult execFetch = fetchBlocks("exec-0", - new String[] { "shuffle_0_0_0" /* right */, "shuffle_1_0_0" /* wrong */ }); + new String[] { "shuffle_0_0_0_0" /* right */, "shuffle_1_0_0_0" /* wrong */ }); // Both still fail, as we start by checking for all block. assertTrue(execFetch.successBlocks.isEmpty()); - assertEquals(Sets.newHashSet("shuffle_0_0_0", "shuffle_1_0_0"), execFetch.failedBlocks); + assertEquals(Sets.newHashSet("shuffle_0_0_0_0", "shuffle_1_0_0_0"), execFetch.failedBlocks); } @Test public void testFetchUnregisteredExecutor() throws Exception { registerExecutor("exec-0", dataContext0.createExecutorInfo(SORT_MANAGER)); FetchResult execFetch = fetchBlocks("exec-2", - new String[] { "shuffle_0_0_0", "shuffle_1_0_0" }); + new String[] { "shuffle_0_0_0_0", "shuffle_1_0_0_0" }); assertTrue(execFetch.successBlocks.isEmpty()); - assertEquals(Sets.newHashSet("shuffle_0_0_0", "shuffle_1_0_0"), execFetch.failedBlocks); + assertEquals(Sets.newHashSet("shuffle_0_0_0_0", "shuffle_1_0_0_0"), execFetch.failedBlocks); } @Test @@ -264,9 +268,9 @@ public void testFetchNoServer() throws Exception { try { registerExecutor("exec-0", dataContext0.createExecutorInfo(SORT_MANAGER)); FetchResult execFetch = fetchBlocks("exec-0", - new String[]{"shuffle_1_0_0", "shuffle_1_0_1"}, 1 /* port */); + new String[]{"shuffle_1_0_0_0", "shuffle_1_0_1_0"}, 1 /* port */); assertTrue(execFetch.successBlocks.isEmpty()); - assertEquals(Sets.newHashSet("shuffle_1_0_0", "shuffle_1_0_1"), execFetch.failedBlocks); + assertEquals(Sets.newHashSet("shuffle_1_0_0_0", "shuffle_1_0_1_0"), execFetch.failedBlocks); } finally { System.clearProperty("spark.shuffle.io.maxRetries"); } diff --git a/network/shuffle/src/test/java/org/apache/spark/network/shuffle/TestShuffleDataContext.java b/network/shuffle/src/test/java/org/apache/spark/network/shuffle/TestShuffleDataContext.java index 3fdde054ab6c7..e98765c1d4876 100644 --- a/network/shuffle/src/test/java/org/apache/spark/network/shuffle/TestShuffleDataContext.java +++ b/network/shuffle/src/test/java/org/apache/spark/network/shuffle/TestShuffleDataContext.java @@ -57,8 +57,9 @@ public void cleanup() { } /** Creates reducer blocks in a sort-based data format within our local dirs. */ - public void insertSortShuffleData(int shuffleId, int mapId, byte[][] blocks) throws IOException { - String blockId = "shuffle_" + shuffleId + "_" + mapId + "_0"; + public void insertSortShuffleData(int shuffleId, int mapId, int stageAttemptId, + byte[][] blocks) throws IOException { + String blockId = "shuffle_" + shuffleId + "_" + mapId + "_0_" + stageAttemptId; OutputStream dataStream = new FileOutputStream( ExternalShuffleBlockResolver.getFile(localDirs, subDirsPerLocalDir, blockId + ".data")); @@ -78,9 +79,10 @@ public void insertSortShuffleData(int shuffleId, int mapId, byte[][] blocks) thr } /** Creates reducer blocks in a hash-based data format within our local dirs. */ - public void insertHashShuffleData(int shuffleId, int mapId, byte[][] blocks) throws IOException { + public void insertHashShuffleData(int shuffleId, int mapId, int stageAttemptId, + byte[][] blocks) throws IOException { for (int i = 0; i < blocks.length; i ++) { - String blockId = "shuffle_" + shuffleId + "_" + mapId + "_" + i; + String blockId = "shuffle_" + shuffleId + "_" + mapId + "_" + i + "_" + stageAttemptId; Files.write(blocks[i], ExternalShuffleBlockResolver.getFile(localDirs, subDirsPerLocalDir, blockId)); } From fdcc92d67943546c6536dfc895151af10fac384a Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Wed, 3 Jun 2015 16:05:44 -0500 Subject: [PATCH 31/73] ignore mima false positive --- project/MimaExcludes.scala | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index 34371c9659423..8fd359e28b49f 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -46,7 +46,10 @@ object MimaExcludes { "org.apache.spark.api.java.JavaRDDLike.partitioner"), // Mima false positive (was a private[spark] class) ProblemFilters.exclude[MissingClassProblem]( - "org.apache.spark.util.collection.PairIterator") + "org.apache.spark.util.collection.PairIterator"), + // false positive, this is a @Private class + ProblemFilters.exclude[MissingMethodProblem]( + "org.apache.spark.shuffle.unsafe.UnsafeShuffleWriter") ) case v if v.startsWith("1.4") => Seq( From 66d5bf569521741a36dcc90dcc7b3e9bff9272e5 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Wed, 3 Jun 2015 16:36:30 -0500 Subject: [PATCH 32/73] fix mima exclude --- project/MimaExcludes.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index 8fd359e28b49f..134baf492abfb 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -49,7 +49,7 @@ object MimaExcludes { "org.apache.spark.util.collection.PairIterator"), // false positive, this is a @Private class ProblemFilters.exclude[MissingMethodProblem]( - "org.apache.spark.shuffle.unsafe.UnsafeShuffleWriter") + "org.apache.spark.shuffle.unsafe.UnsafeShuffleWriter.this") ) case v if v.startsWith("1.4") => Seq( From 289576d1507a5e37560b6cf00c0182392277500c Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Thu, 4 Jun 2015 12:08:54 -0500 Subject: [PATCH 33/73] undo some dagscheduler related changes which will go in another pr --- .../apache/spark/scheduler/DAGScheduler.scala | 49 +++---------------- .../spark/scheduler/ShuffleMapStage.scala | 11 ----- .../spark/scheduler/TaskSchedulerImpl.scala | 3 +- 3 files changed, 8 insertions(+), 55 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index fbeac2626e9a0..950e6b837d9ab 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -833,10 +833,6 @@ class DAGScheduler( logDebug("submitMissingTasks(" + stage + ")") // Get our pending tasks and remember them in our pendingTasks entry stage.pendingTasks.clear() - stage match { - case smt: ShuffleMapStage => smt.clearPartitionComputeCount() - case _ => - } // First figure out the indexes of partition ids to compute. @@ -981,7 +977,6 @@ class DAGScheduler( val stageId = task.stageId val taskType = Utils.getFormattedClassName(task) - // REVIEWERS: does this need special handling for multiple completions of the same task? outputCommitCoordinator.taskCompleted(stageId, task.partitionId, event.taskInfo.attempt, event.reason) @@ -1039,31 +1034,15 @@ class DAGScheduler( case smt: ShuffleMapTask => val shuffleStage = stage.asInstanceOf[ShuffleMapStage] - val computeCount = shuffleStage.incComputeCount(smt.partitionId) updateAccumulators(event) val status = event.result.asInstanceOf[MapStatus] val execId = status.location.executorId logDebug("ShuffleMapTask finished on " + execId) - if (computeCount > 1) { - // REVIEWERS: do I need to worry about speculation here, when multiple completion - // events are normal? - - // REVIEWERS: is this really only a problem on a ShuffleMapTask?? does it also cause - // problems for ResultTask? - - // This can happen when a retry runs a task, but there was a lingering task from an - // earlier attempt which also finished. The results might be OK, or they might not. - // To be safe, we'll retry the task, and do it in yet another attempt, to avoid more - // task output clobbering. - logInfo(s"Multiple completion events for task $task. Results may be corrupt," + - s" assuming task needs to be rerun.") - shuffleStage.removeOutputLoc(task.partitionId) - } else if (failedEpoch.contains(execId) && smt.epoch <= failedEpoch(execId)) { - logInfo(s"Ignoring possibly bogus $smt completion from executor $execId") + if (failedEpoch.contains(execId) && smt.epoch <= failedEpoch(execId)) { + logInfo("Ignoring possibly bogus ShuffleMapTask completion from " + execId) } else { shuffleStage.addOutputLoc(smt.partitionId, status) } - if (runningStages.contains(shuffleStage) && shuffleStage.pendingTasks.isEmpty) { markStageAsFinished(shuffleStage) logInfo("looking for newly runnable stages") @@ -1127,19 +1106,13 @@ class DAGScheduler( // multiple tasks running concurrently on different executors). In that case, it is possible // the fetch failure has already been handled by the scheduler. if (runningStages.contains(failedStage)) { - if (failedStage.attemptId - 1 > task.stageAttemptId) { - logInfo(s"Ignoring fetch failure from $task as it's from $failedStage attempt" + - s" ${task.stageAttemptId}, which has already failed") - } else { - logInfo(s"Marking $failedStage (${failedStage.name}) as failed " + - s"due to a fetch failure from $mapStage (${mapStage.name})") - markStageAsFinished(failedStage, Some(failureMessage)) - } + logInfo(s"Marking $failedStage (${failedStage.name}) as failed " + + s"due to a fetch failure from $mapStage (${mapStage.name})") + markStageAsFinished(failedStage, Some(failureMessage)) } if (disallowStageRetryForTest) { - abortStage(failedStage, - s"Fetch failure will not retry stage due to testing config. Failure = $failureMessage") + abortStage(failedStage, "Fetch failure will not retry stage due to testing config") } else if (failedStages.isEmpty) { // Don't schedule an event to resubmit failed stages if failed isn't empty, because // in that case the event will already have been scheduled. @@ -1158,16 +1131,6 @@ class DAGScheduler( mapOutputTracker.unregisterMapOutput(shuffleId, mapId, bmAddress) } - // We also have to mark this map output as unavailable. Its possible that a *later* attempt - // has finished this task in the meantime, but when this task fails, it might end up - // deleting the mapOutput from the earlier successful attempt. - failedStage match { - case smt: ShuffleMapStage => - smt.incComputeCount(reduceId) - smt.removeOutputLoc(reduceId) - case _ => - } - // TODO: mark the executor as failed only if there were lots of fetch failures on it if (bmAddress != null) { handleExecutorLost(bmAddress.executorId, fetchFailed = true, Some(task.epoch)) diff --git a/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapStage.scala b/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapStage.scala index 12357ccbfce25..67a69d9c7f848 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapStage.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapStage.scala @@ -45,17 +45,6 @@ private[spark] class ShuffleMapStage( val outputLocs = Array.fill[List[MapStatus]](numPartitions)(Nil) - private val partitionComputeCount = HashMap[Int, Int]() - - def incComputeCount(partition: Int): Int = { - partitionComputeCount(partition) = partitionComputeCount.getOrElse(partition, 0) + 1 - partitionComputeCount(partition) - } - - def clearPartitionComputeCount(): Unit = { - partitionComputeCount.clear() - } - def addOutputLoc(partition: Int, status: MapStatus): Unit = { val prevList = outputLocs(partition) outputLocs(partition) = status :: prevList diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala index 60173e21b64a8..d3b37e9e43011 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -166,7 +166,8 @@ private[spark] class TaskSchedulerImpl( val taskSetsPerStage = activeTaskSets.values.filterNot(_.isZombie).groupBy(_.stageId) taskSetsPerStage.foreach { case (stage, taskSets) => if (taskSets.size > 1) { - throw new SparkIllegalStateException("more than one active taskSet for stage " + stage) + logWarning("more than one active taskSet for stage " + stage) +// throw new SparkIllegalStateException("more than one active taskSet for stage " + stage) } } schedulableBuilder.addTaskSetManager(manager, manager.taskSet.properties) From 9a06fe2adac2eb7afd87c86bda5b5e9979e38fae Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Thu, 4 Jun 2015 12:27:51 -0500 Subject: [PATCH 34/73] removing some unnecessary changes --- .../org/apache/spark/SparkException.scala | 10 --- .../spark/scheduler/DAGSchedulerSuite.scala | 77 +------------------ 2 files changed, 2 insertions(+), 85 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkException.scala b/core/src/main/scala/org/apache/spark/SparkException.scala index b7c2386fd7d87..2ebd7a7151a59 100644 --- a/core/src/main/scala/org/apache/spark/SparkException.scala +++ b/core/src/main/scala/org/apache/spark/SparkException.scala @@ -30,13 +30,3 @@ class SparkException(message: String, cause: Throwable) */ private[spark] class SparkDriverExecutionException(cause: Throwable) extends SparkException("Execution error", cause) - -/** - * Exception indicating an error internal to Spark -- it is in an inconsistent state, not due - * to any error by the user - */ -class SparkIllegalStateException(message: String, cause: Throwable) - extends SparkException(message, cause) { - - def this(message: String) = this(message, null) -} diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala index 620ab553aff46..5186949bcf896 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -28,7 +28,6 @@ import org.scalatest.time.SpanSugar._ import org.apache.spark._ import org.apache.spark.rdd.RDD import org.apache.spark.scheduler.SchedulingMode.SchedulingMode -import org.apache.spark.shuffle.FetchFailedException import org.apache.spark.storage.{BlockId, BlockManagerId, BlockManagerMaster} import org.apache.spark.util.CallSite import org.apache.spark.executor.TaskMetrics @@ -563,7 +562,7 @@ class DAGSchedulerSuite // should be ignored for being too old runEvent(CompletionEvent( taskSet.tasks(0), Success, makeMapStatus("hostA", 1), null, createFakeTaskInfo(), null)) - // its a non-failed host, but we can't be sure if the results were clobbered + // should work because it's a non-failed host runEvent(CompletionEvent( taskSet.tasks(0), Success, makeMapStatus("hostB", 1), null, createFakeTaskInfo(), null)) // should be ignored for being too old @@ -573,21 +572,9 @@ class DAGSchedulerSuite taskSet.tasks(1).epoch = newEpoch runEvent(CompletionEvent( taskSet.tasks(1), Success, makeMapStatus("hostA", 1), null, createFakeTaskInfo(), null)) - - // now we should have a new taskSet for stage 0, which has us retry partition 0 - assert(taskSets.size === 2) - val newTaskSet = taskSets(1) - assert(newTaskSet.stageId === 0) - assert(newTaskSet.attempt === 1) - assert(newTaskSet.tasks.size === 1) - val newTask = newTaskSet.tasks(0) - assert(newTask.epoch === newEpoch + 1) - runEvent(CompletionEvent( - newTask, Success, makeMapStatus("hostB", 1), null, createFakeTaskInfo(), null)) - assert(mapOutputTracker.getServerStatuses(shuffleId, 0).map(_.bmId) === Array(makeBlockManagerId("hostB"), makeBlockManagerId("hostA"))) - complete(taskSets(2), Seq((Success, 42), (Success, 43))) + complete(taskSets(1), Seq((Success, 42), (Success, 43))) assert(results === Map(0 -> 42, 1 -> 43)) assertDataStructuresEmpty() } @@ -813,66 +800,6 @@ class DAGSchedulerSuite assertDataStructuresEmpty() } - ignore("no concurrent retries for stage attempts (SPARK-7308)") { - // see SPARK-7308 for a detailed description of the conditions this is trying to recreate. - // note that this is somewhat convoluted for a test case, but isn't actually very unusual - // under a real workload. Note that we only fail the first attempt of stage 2, but that - // could be enough to cause havoc. - - val conf = new SparkConf().set("spark.executor.memory", "100m") - val clusterSc = new SparkContext("local-cluster[5,4,100]", "test-cluster", conf) - val bms = ArrayBuffer[BlockManagerId]() - val stageFailureCount = HashMap[Int, Int]() - clusterSc.addSparkListener(new SparkListener { - override def onBlockManagerAdded(bmAdded: SparkListenerBlockManagerAdded): Unit = { - bms += bmAdded.blockManagerId - } - - override def onStageCompleted(stageCompleted: SparkListenerStageCompleted): Unit = { - if (stageCompleted.stageInfo.failureReason.isDefined) { - val stage = stageCompleted.stageInfo.stageId - stageFailureCount(stage) = stageFailureCount.getOrElse(stage, 0) + 1 - println("stage " + stage + " failed: " + stageFailureCount(stage)) - } - } - }) - try { - val rawData = clusterSc.parallelize(1 to 1e6.toInt, 20).map { x => (x % 100) -> x }.cache() - rawData.count() - val aBm = bms(0) - val shuffled = rawData.groupByKey(100).mapPartitionsWithIndex { case (idx, itr) => - // we want one failure quickly, and more failures after stage 0 has finished its - // second attempt - if (TaskContext.get().asInstanceOf[TaskContextImpl].stageAttemptId == 0) { - if (idx == 0) { - throw new FetchFailedException(aBm, 0, 0, idx, 0, - cause = new RuntimeException("simulated fetch failure")) - } else if (idx > 0 && math.random < 0.2) { - Thread.sleep(5000) - throw new FetchFailedException(aBm, 0, 0, idx, 0, - cause = new RuntimeException("simulated fetch failure")) - } else { - // want to make sure plenty of these finish after task 0 fails, and some even finish - // after the previous stage is retried and this stage retry is started - Thread.sleep((500 + math.random * 5000).toLong) - } - } - itr.map { x => ((x._1 + 5) % 100) -> x._2 } - } - val shuffledAgain = shuffled.flatMap { case (k, vs) => vs.map(k -> _) }.groupByKey(100) - val data = shuffledAgain.mapPartitions { itr => itr.flatMap(_._2) }.cache().collect() - val count = data.size - assert(count === 1e6.toInt) - assert(data.toSet === (1 to 1e6.toInt).toSet) - // we should only get one failure from stage 2, everything else should be fine - assert(stageFailureCount(2) === 1) - assert(stageFailureCount.getOrElse(1, 0) === 0) - assert(stageFailureCount.getOrElse(3, 0) == 0) - } finally { - clusterSc.stop() - } - } - /** * Assert that the supplied TaskSet has exactly the given hosts as its preferred locations. * Note that this checks only the host and not the executor ID. From 9befe5160b9d3876d1883953685da522f3f5cf21 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Thu, 4 Jun 2015 12:31:59 -0500 Subject: [PATCH 35/73] fault tolerance test in its own suite, since its more of an integration test --- .../DAGSchedulerFailureRecoverySuite.scala | 122 ++++++++++++++++++ 1 file changed, 122 insertions(+) create mode 100644 core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerFailureRecoverySuite.scala diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerFailureRecoverySuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerFailureRecoverySuite.scala new file mode 100644 index 0000000000000..efe279a4cb1ef --- /dev/null +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerFailureRecoverySuite.scala @@ -0,0 +1,122 @@ +/* +* 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.scheduler + +import java.util.Date + +import org.scalatest.FunSuite + +import scala.collection.mutable.{ArrayBuffer,HashMap} + +import org.apache.spark.shuffle.FetchFailedException +import org.apache.spark.storage.BlockManagerId +import org.apache.spark._ + +class DAGSchedulerFailureRecoverySuite extends FunSuite with Logging { + + ignore("no concurrent retries for stage attempts (SPARK-7308)") { + // see SPARK-7308 for a detailed description of the conditions this is trying to recreate. + // note that this is somewhat convoluted for a test case, but isn't actually very unusual + // under a real workload. Note that we only fail the first attempt of stage 2, but that + // could be enough to cause havoc. + + var overallBroadcastFailures = 0 + + (0 until 100).foreach { idx => + println(new Date() + "\ttrial " + idx) + logInfo(new Date() + "\ttrial " + idx) + + val conf = new SparkConf().set("spark.executor.memory", "100m") + val clusterSc = new SparkContext("local-cluster[5,4,100]", "test-cluster", conf) + val bms = ArrayBuffer[BlockManagerId]() + val stageFailureCount = HashMap[Int, Int]() + var broadcastFailures = 0 + clusterSc.addSparkListener(new SparkListener { + override def onBlockManagerAdded(bmAdded: SparkListenerBlockManagerAdded): Unit = { + bms += bmAdded.blockManagerId + } + + override def onStageCompleted(stageCompleted: SparkListenerStageCompleted): Unit = { + if (stageCompleted.stageInfo.failureReason.isDefined) { + val stage = stageCompleted.stageInfo.stageId + stageFailureCount(stage) = stageFailureCount.getOrElse(stage, 0) + 1 + val reason = stageCompleted.stageInfo.failureReason.get + println("stage " + stage + " failed: " + stageFailureCount(stage)) + if (reason.contains("Failed to get broadcast")) { + broadcastFailures += 1 + } + } + } + }) + try { + val rawData = clusterSc.parallelize(1 to 1e6.toInt, 20).map { x => (x % 100) -> x }.cache() + rawData.count() + val aBm = bms(0) + val shuffled = rawData.groupByKey(100).mapPartitionsWithIndex { case (idx, itr) => + // we want one failure quickly, and more failures after stage 0 has finished its + // second attempt + val stageAttemptId = TaskContext.get().asInstanceOf[TaskContextImpl].stageAttemptId + if (stageAttemptId == 0) { + if (idx == 0) { + throw new FetchFailedException(aBm, 0, 0, idx, stageAttemptId, + cause = new RuntimeException("simulated fetch failure")) + } else if (idx > 0 && math.random < 0.2) { + Thread.sleep(5000) + throw new FetchFailedException(aBm, 0, 0, idx, stageAttemptId, + cause = new RuntimeException("simulated fetch failure")) + } else { + // want to make sure plenty of these finish after task 0 fails, and some even finish + // after the previous stage is retried and this stage retry is started + Thread.sleep((500 + math.random * 5000).toLong) + } + } + itr.map { x => ((x._1 + 5) % 100) -> x._2 } + } + val shuffledAgain = shuffled.flatMap { case (k, vs) => vs.map(k -> _) }.groupByKey(100) + try { + val data = shuffledAgain.mapPartitions { itr => itr.flatMap(_._2) }.cache().collect() + val count = data.size + assert(count === 1e6.toInt) + assert(data.toSet === (1 to 1e6.toInt).toSet) + + // we should only get one failure from stage 2, everything else should be fine + // However, one submissions can result in multiple failures (SPARK-8103), so for + // now this has to have very weak checks. Once there is a fix for SPARK-8103, these + // checks should be made tighter + + assert(stageFailureCount.getOrElse(1, 0) === 0) + assert(stageFailureCount.getOrElse(2, 0) <= 10) // should be 1 failure + assert(stageFailureCount.getOrElse(3, 0) <= 10) // should be 0 failures + } catch { + case se: SparkException => + if (se.getMessage.contains("Failed to get broadcast_")) { + overallBroadcastFailures += 1 + println("this attempt failed from a broadcast failure, ignoring") + logInfo("broadcast failure: ", se) + } else { + throw se + } + } + } finally { + clusterSc.stop() + } + } + println("total broadcast failures = " + overallBroadcastFailures) + assert(overallBroadcastFailures < 10) + } + +} From 87d7ddddc9beeb03948bc4d357fad31de7b7f60a Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Thu, 4 Jun 2015 12:35:28 -0500 Subject: [PATCH 36/73] remove more unnecessary changes --- .../org/apache/spark/scheduler/ShuffleMapStage.scala | 9 --------- 1 file changed, 9 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapStage.scala b/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapStage.scala index 67a69d9c7f848..66c75f325fcde 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapStage.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapStage.scala @@ -17,8 +17,6 @@ package org.apache.spark.scheduler -import scala.collection.mutable.HashMap - import org.apache.spark.ShuffleDependency import org.apache.spark.rdd.RDD import org.apache.spark.storage.BlockManagerId @@ -53,13 +51,6 @@ private[spark] class ShuffleMapStage( } } - def removeOutputLoc(partition: Int): Unit = { - if (outputLocs(partition) != Nil) { - outputLocs(partition) = Nil - numAvailableOutputs -= 1 - } - } - def removeOutputLoc(partition: Int, bmAddress: BlockManagerId): Unit = { val prevList = outputLocs(partition) val newList = prevList.filterNot(_.location == bmAddress) From 1072a44f1bc8dcc8e4be0dd52255f88385866c36 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Thu, 4 Jun 2015 12:41:33 -0500 Subject: [PATCH 37/73] ShuffleId --> ShuffleIdAndAttempt (and a case class) --- .../spark/shuffle/FileShuffleBlockResolver.scala | 15 ++++++++------- .../spark/shuffle/ShuffleBlockResolver.scala | 4 ++-- .../spark/shuffle/hash/HashShuffleManager.scala | 2 +- .../shuffle/hash/HashShuffleManagerSuite.scala | 6 +++--- 4 files changed, 14 insertions(+), 13 deletions(-) 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 63dbb8ddf4655..690d54b8fc197 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/FileShuffleBlockResolver.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/FileShuffleBlockResolver.scala @@ -95,7 +95,7 @@ private[spark] class FileShuffleBlockResolver(conf: SparkConf) val completedMapTasks = new ConcurrentLinkedQueue[Int]() } - private val shuffleStates = new TimeStampedHashMap[ShuffleId, ShuffleState] + private val shuffleStates = new TimeStampedHashMap[ShuffleIdAndAttempt, ShuffleState] private val metadataCleaner = new MetadataCleaner(MetadataCleanerType.SHUFFLE_BLOCK_MANAGER, this.cleanup, conf) @@ -107,8 +107,8 @@ private[spark] class FileShuffleBlockResolver(conf: SparkConf) def forMapTask(shuffleId: Int, mapId: Int, stageAttemptId: Int, numBuckets: Int, serializer: Serializer, writeMetrics: ShuffleWriteMetrics): ShuffleWriterGroup = { new ShuffleWriterGroup { - shuffleStates.putIfAbsent((shuffleId, stageAttemptId), new ShuffleState(numBuckets)) - private val shuffleState = shuffleStates((shuffleId, stageAttemptId)) + shuffleStates.putIfAbsent(ShuffleIdAndAttempt(shuffleId, stageAttemptId), new ShuffleState(numBuckets)) + private val shuffleState = shuffleStates(ShuffleIdAndAttempt(shuffleId, stageAttemptId)) private var fileGroup: ShuffleFileGroup = null val openStartTime = System.nanoTime @@ -179,7 +179,7 @@ private[spark] class FileShuffleBlockResolver(conf: SparkConf) override def getBlockData(blockId: ShuffleBlockId): ManagedBuffer = { if (consolidateShuffleFiles) { // Search all file groups associated with this shuffle. - val shuffleState = shuffleStates((blockId.shuffleId, blockId.stageAttemptId)) + val shuffleState = shuffleStates(ShuffleIdAndAttempt(blockId.shuffleId, blockId.stageAttemptId)) val iter = shuffleState.allFileGroups.iterator while (iter.hasNext) { val segmentOpt = iter.next.getFileSegmentFor(blockId.mapId, blockId.reduceId) @@ -197,7 +197,7 @@ private[spark] class FileShuffleBlockResolver(conf: SparkConf) } /** Remove all the blocks / files and metadata related to a particular shuffle. */ - def removeShuffle(shuffleId: ShuffleId): Boolean = { + def removeShuffle(shuffleId: ShuffleIdAndAttempt): Boolean = { // Do not change the ordering of this, if shuffleStates should be removed only // after the corresponding shuffle blocks have been removed val cleaned = removeShuffleBlocks(shuffleId) @@ -206,7 +206,7 @@ private[spark] class FileShuffleBlockResolver(conf: SparkConf) } /** Remove all the blocks / files related to a particular shuffle. */ - private def removeShuffleBlocks(shuffleId: ShuffleId): Boolean = { + private def removeShuffleBlocks(shuffleId: ShuffleIdAndAttempt): Boolean = { shuffleStates.get(shuffleId) match { case Some(state) => if (consolidateShuffleFiles) { @@ -215,7 +215,8 @@ private[spark] class FileShuffleBlockResolver(conf: SparkConf) } } else { for (mapId <- state.completedMapTasks; reduceId <- 0 until state.numBuckets) { - val blockId = new ShuffleBlockId(shuffleId._1, mapId, reduceId, shuffleId._2) + val blockId = new ShuffleBlockId(shuffleId.shuffleId, mapId, reduceId, + shuffleId.stageAttemptId) blockManager.diskBlockManager.getFile(blockId).delete() } } diff --git a/core/src/main/scala/org/apache/spark/shuffle/ShuffleBlockResolver.scala b/core/src/main/scala/org/apache/spark/shuffle/ShuffleBlockResolver.scala index e65738836c0a9..51c30abbbf1ef 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/ShuffleBlockResolver.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/ShuffleBlockResolver.scala @@ -28,8 +28,6 @@ private[spark] * implementations when shuffle data is retrieved. */ trait ShuffleBlockResolver { - type ShuffleId = (Int, Int) - /** * Retrieve the data for the specified block. If the data for that block is not available, * throws an unspecified exception. @@ -38,3 +36,5 @@ trait ShuffleBlockResolver { def stop(): Unit } + +private[spark] case class ShuffleIdAndAttempt(shuffleId: Int, stageAttemptId: Int) diff --git a/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleManager.scala b/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleManager.scala index 7896d8b77062f..d83539546fd1d 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleManager.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleManager.scala @@ -63,7 +63,7 @@ private[spark] class HashShuffleManager(conf: SparkConf) extends ShuffleManager /** Remove a shuffle's metadata from the ShuffleManager. */ override def unregisterShuffle(shuffleId: Int): Boolean = { stageAttemptsForShuffle(shuffleId).forall { stageAttemptId => - shuffleBlockResolver.removeShuffle(shuffleId, stageAttemptId) + shuffleBlockResolver.removeShuffle(ShuffleIdAndAttempt(shuffleId, stageAttemptId)) } } diff --git a/core/src/test/scala/org/apache/spark/shuffle/hash/HashShuffleManagerSuite.scala b/core/src/test/scala/org/apache/spark/shuffle/hash/HashShuffleManagerSuite.scala index 0b6753b736e66..3a13df3e4a03b 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/hash/HashShuffleManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/hash/HashShuffleManagerSuite.scala @@ -25,8 +25,8 @@ import org.apache.spark.{LocalSparkContext, SparkConf, SparkContext, SparkEnv, S import org.apache.spark.executor.ShuffleWriteMetrics import org.apache.spark.network.buffer.{FileSegmentManagedBuffer, ManagedBuffer} import org.apache.spark.serializer.JavaSerializer -import org.apache.spark.shuffle.FileShuffleBlockResolver -import org.apache.spark.storage.{ShuffleBlockId, FileSegment} +import org.apache.spark.shuffle.{FileShuffleBlockResolver, ShuffleIdAndAttempt} +import org.apache.spark.storage.{FileSegment, ShuffleBlockId} class HashShuffleManagerSuite extends SparkFunSuite with LocalSparkContext { private val testConf = new SparkConf(false) @@ -99,7 +99,7 @@ class HashShuffleManagerSuite extends SparkFunSuite with LocalSparkContext { checkSegments(shuffle2Segment, shuffleBlockResolver.getBlockData(ShuffleBlockId(1, 2, 0, 0))) shuffle3.releaseWriters(success = true) checkSegments(shuffle2Segment, shuffleBlockResolver.getBlockData(ShuffleBlockId(1, 2, 0, 0))) - shuffleBlockResolver.removeShuffle((1, 0)) + shuffleBlockResolver.removeShuffle(ShuffleIdAndAttempt(1, 0)) } def writeToFile(file: File, numBytes: Int) { From 89a93aea09f53cb60079d49929c8c48192a58685 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Thu, 4 Jun 2015 12:46:55 -0500 Subject: [PATCH 38/73] remove more unnecessary stuff --- .../org/apache/spark/shuffle/sort/SortShuffleWriter.scala | 3 --- 1 file changed, 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 d4a5b8b4a7a55..e943adf9346f7 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 @@ -77,9 +77,6 @@ private[spark] class SortShuffleWriter[K, V, C]( // 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, stageAttemptId) - // Because we append to the data file, we need the index file to know the current size of the - // data file as a starting point - val initialFileLength = outputFile.length() val blockId = ShuffleBlockId(dep.shuffleId, mapId, IndexShuffleBlockResolver.NOOP_REDUCE_ID, stageAttemptId) val partitionLengths = sorter.writePartitionedFile(blockId, context, outputFile) From ece31bab0a43e4ad4c2c333dfd0ff3d35ac747c9 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Thu, 4 Jun 2015 13:37:24 -0500 Subject: [PATCH 39/73] style --- .../org/apache/spark/shuffle/FileShuffleBlockResolver.scala | 6 ++++-- .../spark/scheduler/DAGSchedulerFailureRecoverySuite.scala | 6 ++---- 2 files changed, 6 insertions(+), 6 deletions(-) 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 690d54b8fc197..d1464d3e4ea9c 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/FileShuffleBlockResolver.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/FileShuffleBlockResolver.scala @@ -107,7 +107,8 @@ private[spark] class FileShuffleBlockResolver(conf: SparkConf) def forMapTask(shuffleId: Int, mapId: Int, stageAttemptId: Int, numBuckets: Int, serializer: Serializer, writeMetrics: ShuffleWriteMetrics): ShuffleWriterGroup = { new ShuffleWriterGroup { - shuffleStates.putIfAbsent(ShuffleIdAndAttempt(shuffleId, stageAttemptId), new ShuffleState(numBuckets)) + shuffleStates.putIfAbsent( + ShuffleIdAndAttempt(shuffleId, stageAttemptId), new ShuffleState(numBuckets)) private val shuffleState = shuffleStates(ShuffleIdAndAttempt(shuffleId, stageAttemptId)) private var fileGroup: ShuffleFileGroup = null @@ -179,7 +180,8 @@ private[spark] class FileShuffleBlockResolver(conf: SparkConf) override def getBlockData(blockId: ShuffleBlockId): ManagedBuffer = { if (consolidateShuffleFiles) { // Search all file groups associated with this shuffle. - val shuffleState = shuffleStates(ShuffleIdAndAttempt(blockId.shuffleId, blockId.stageAttemptId)) + val shuffleState = shuffleStates( + ShuffleIdAndAttempt(blockId.shuffleId, blockId.stageAttemptId)) val iter = shuffleState.allFileGroups.iterator while (iter.hasNext) { val segmentOpt = iter.next.getFileSegmentFor(blockId.mapId, blockId.reduceId) diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerFailureRecoverySuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerFailureRecoverySuite.scala index efe279a4cb1ef..3e748d1b8bcbe 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerFailureRecoverySuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerFailureRecoverySuite.scala @@ -18,15 +18,13 @@ package org.apache.spark.scheduler import java.util.Date -import org.scalatest.FunSuite - -import scala.collection.mutable.{ArrayBuffer,HashMap} +import scala.collection.mutable.{ArrayBuffer, HashMap} import org.apache.spark.shuffle.FetchFailedException import org.apache.spark.storage.BlockManagerId import org.apache.spark._ -class DAGSchedulerFailureRecoverySuite extends FunSuite with Logging { +class DAGSchedulerFailureRecoverySuite extends SparkFunSuite with Logging { ignore("no concurrent retries for stage attempts (SPARK-7308)") { // see SPARK-7308 for a detailed description of the conditions this is trying to recreate. From de62da03dc7912ffd7f59f47bcba00ffc2ac3589 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Thu, 4 Jun 2015 13:52:10 -0500 Subject: [PATCH 40/73] explanation of the broadcast failure handling --- .../spark/scheduler/DAGSchedulerFailureRecoverySuite.scala | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerFailureRecoverySuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerFailureRecoverySuite.scala index 3e748d1b8bcbe..7988ddbeb69c7 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerFailureRecoverySuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerFailureRecoverySuite.scala @@ -26,12 +26,16 @@ import org.apache.spark._ class DAGSchedulerFailureRecoverySuite extends SparkFunSuite with Logging { - ignore("no concurrent retries for stage attempts (SPARK-7308)") { + test("no concurrent retries for stage attempts (SPARK-7308)") { // see SPARK-7308 for a detailed description of the conditions this is trying to recreate. // note that this is somewhat convoluted for a test case, but isn't actually very unusual // under a real workload. Note that we only fail the first attempt of stage 2, but that // could be enough to cause havoc. + + // even when we address the main issues we're trying to fix here, we still see some occasional + // failures from broadcast -- maybe the same as SPARK-5812 or SPARK-5594. So just allow + // some extra slop for it var overallBroadcastFailures = 0 (0 until 100).foreach { idx => From a7f2d9a09ebf93df5a7c8a79079b96d6a09c6b27 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Fri, 5 Jun 2015 07:54:27 -0500 Subject: [PATCH 41/73] make sure we dont drop the driver block manager (prevents the broadcast failures) --- .../DAGSchedulerFailureRecoverySuite.scala | 57 +++++++------------ 1 file changed, 20 insertions(+), 37 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerFailureRecoverySuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerFailureRecoverySuite.scala index 7988ddbeb69c7..4862577f340b5 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerFailureRecoverySuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerFailureRecoverySuite.scala @@ -32,12 +32,6 @@ class DAGSchedulerFailureRecoverySuite extends SparkFunSuite with Logging { // under a real workload. Note that we only fail the first attempt of stage 2, but that // could be enough to cause havoc. - - // even when we address the main issues we're trying to fix here, we still see some occasional - // failures from broadcast -- maybe the same as SPARK-5812 or SPARK-5594. So just allow - // some extra slop for it - var overallBroadcastFailures = 0 - (0 until 100).foreach { idx => println(new Date() + "\ttrial " + idx) logInfo(new Date() + "\ttrial " + idx) @@ -46,7 +40,6 @@ class DAGSchedulerFailureRecoverySuite extends SparkFunSuite with Logging { val clusterSc = new SparkContext("local-cluster[5,4,100]", "test-cluster", conf) val bms = ArrayBuffer[BlockManagerId]() val stageFailureCount = HashMap[Int, Int]() - var broadcastFailures = 0 clusterSc.addSparkListener(new SparkListener { override def onBlockManagerAdded(bmAdded: SparkListenerBlockManagerAdded): Unit = { bms += bmAdded.blockManagerId @@ -58,27 +51,28 @@ class DAGSchedulerFailureRecoverySuite extends SparkFunSuite with Logging { stageFailureCount(stage) = stageFailureCount.getOrElse(stage, 0) + 1 val reason = stageCompleted.stageInfo.failureReason.get println("stage " + stage + " failed: " + stageFailureCount(stage)) - if (reason.contains("Failed to get broadcast")) { - broadcastFailures += 1 - } } } }) try { val rawData = clusterSc.parallelize(1 to 1e6.toInt, 20).map { x => (x % 100) -> x }.cache() rawData.count() - val aBm = bms(0) + + // choose any executor block manager for the fetch failures. Just can't be driver + // to avoid broadcast failures + val someBlockManager = bms.filter{!_.isDriver}(0) + val shuffled = rawData.groupByKey(100).mapPartitionsWithIndex { case (idx, itr) => // we want one failure quickly, and more failures after stage 0 has finished its // second attempt val stageAttemptId = TaskContext.get().asInstanceOf[TaskContextImpl].stageAttemptId if (stageAttemptId == 0) { if (idx == 0) { - throw new FetchFailedException(aBm, 0, 0, idx, stageAttemptId, + throw new FetchFailedException(someBlockManager, 0, 0, idx, stageAttemptId, cause = new RuntimeException("simulated fetch failure")) } else if (idx > 0 && math.random < 0.2) { Thread.sleep(5000) - throw new FetchFailedException(aBm, 0, 0, idx, stageAttemptId, + throw new FetchFailedException(someBlockManager, 0, 0, idx, stageAttemptId, cause = new RuntimeException("simulated fetch failure")) } else { // want to make sure plenty of these finish after task 0 fails, and some even finish @@ -89,36 +83,25 @@ class DAGSchedulerFailureRecoverySuite extends SparkFunSuite with Logging { itr.map { x => ((x._1 + 5) % 100) -> x._2 } } val shuffledAgain = shuffled.flatMap { case (k, vs) => vs.map(k -> _) }.groupByKey(100) - try { - val data = shuffledAgain.mapPartitions { itr => itr.flatMap(_._2) }.cache().collect() - val count = data.size - assert(count === 1e6.toInt) - assert(data.toSet === (1 to 1e6.toInt).toSet) + val data = shuffledAgain.mapPartitions { itr => itr.flatMap(_._2) }.cache().collect() + val count = data.size + assert(count === 1e6.toInt) + assert(data.toSet === (1 to 1e6.toInt).toSet) - // we should only get one failure from stage 2, everything else should be fine - // However, one submissions can result in multiple failures (SPARK-8103), so for - // now this has to have very weak checks. Once there is a fix for SPARK-8103, these - // checks should be made tighter + // we should only get one failure from stage 2, everything else should be fine + // However, one submissions can result in multiple failures (SPARK-8103), so for + // now this has to have very weak checks. Once there is a fix for SPARK-8103, these + // checks should be made tighter - assert(stageFailureCount.getOrElse(1, 0) === 0) - assert(stageFailureCount.getOrElse(2, 0) <= 10) // should be 1 failure - assert(stageFailureCount.getOrElse(3, 0) <= 10) // should be 0 failures - } catch { - case se: SparkException => - if (se.getMessage.contains("Failed to get broadcast_")) { - overallBroadcastFailures += 1 - println("this attempt failed from a broadcast failure, ignoring") - logInfo("broadcast failure: ", se) - } else { - throw se - } - } + assert(stageFailureCount.getOrElse(1, 0) === 0) + assert(stageFailureCount.getOrElse(2, 0) <= 10) // should be 1 failure + assert(stageFailureCount.getOrElse(3, 0) <= 10) // should be 0 failures } finally { clusterSc.stop() } } - println("total broadcast failures = " + overallBroadcastFailures) - assert(overallBroadcastFailures < 10) } + + } From 4bfbf9405444d79041da1e2bff5ef11ef37fa4df Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Fri, 5 Jun 2015 07:55:43 -0500 Subject: [PATCH 42/73] oops, ignore test again for jenkins! --- .../spark/scheduler/DAGSchedulerFailureRecoverySuite.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerFailureRecoverySuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerFailureRecoverySuite.scala index 4862577f340b5..0e2b2c922fb70 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerFailureRecoverySuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerFailureRecoverySuite.scala @@ -26,7 +26,7 @@ import org.apache.spark._ class DAGSchedulerFailureRecoverySuite extends SparkFunSuite with Logging { - test("no concurrent retries for stage attempts (SPARK-7308)") { + ignore("no concurrent retries for stage attempts (SPARK-7308)") { // see SPARK-7308 for a detailed description of the conditions this is trying to recreate. // note that this is somewhat convoluted for a test case, but isn't actually very unusual // under a real workload. Note that we only fail the first attempt of stage 2, but that From f9a1a319af134251962b75472bf72076e9329408 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Fri, 5 Jun 2015 14:10:46 -0500 Subject: [PATCH 43/73] comment tweak --- .../spark/scheduler/DAGSchedulerFailureRecoverySuite.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerFailureRecoverySuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerFailureRecoverySuite.scala index 0e2b2c922fb70..5085304165252 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerFailureRecoverySuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerFailureRecoverySuite.scala @@ -29,7 +29,7 @@ class DAGSchedulerFailureRecoverySuite extends SparkFunSuite with Logging { ignore("no concurrent retries for stage attempts (SPARK-7308)") { // see SPARK-7308 for a detailed description of the conditions this is trying to recreate. // note that this is somewhat convoluted for a test case, but isn't actually very unusual - // under a real workload. Note that we only fail the first attempt of stage 2, but that + // under a real workload. We only fail the first attempt of stage 2, but that // could be enough to cause havoc. (0 until 100).foreach { idx => From b762e22b36e54331f8c47146d8788fc57e01bf17 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Fri, 5 Jun 2015 14:25:13 -0500 Subject: [PATCH 44/73] fix mima --- project/MimaExcludes.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index 11ae5eec9ab50..68bbcde0e3b11 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -49,7 +49,7 @@ object MimaExcludes { "org.apache.spark.util.collection.PairIterator"), // false positive, this is a @Private class ProblemFilters.exclude[MissingMethodProblem]( - "org.apache.spark.shuffle.unsafe.UnsafeShuffleWriter.this") + "org.apache.spark.shuffle.unsafe.UnsafeShuffleWriter.this"), // SQL execution is considered private. excludePackage("org.apache.spark.sql.execution") ) From 8bbda629b75ef9cded1d562ec5075eb920938330 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Fri, 5 Jun 2015 16:49:39 -0500 Subject: [PATCH 45/73] more tests around multiple attempts of ShuffleMapTasks --- .../shuffle/unsafe/UnsafeShuffleWriter.java | 6 +- .../shuffle/FileShuffleBlockResolver.scala | 7 ++ .../shuffle/IndexShuffleBlockResolver.scala | 7 +- .../apache/spark/shuffle/ShuffleManager.scala | 9 +++ .../shuffle/hash/HashShuffleManager.scala | 13 ++++ .../shuffle/sort/SortShuffleManager.scala | 13 +++- .../shuffle/unsafe/UnsafeShuffleManager.scala | 9 +++ .../apache/spark/ContextCleanerSuite.scala | 3 - .../spark/ExternalShuffleServiceSuite.scala | 4 +- .../org/apache/spark/HashShuffleSuite.scala | 2 + .../org/apache/spark/ShuffleNettySuite.scala | 2 + .../org/apache/spark/SortShuffleSuite.scala | 2 + .../DAGSchedulerFailureRecoverySuite.scala | 4 ++ .../spark/{ => shuffle}/ShuffleSuite.scala | 72 +++++++++++++++++-- .../shuffle/unsafe/UnsafeShuffleSuite.scala | 16 ++++- .../shuffle/ExternalShuffleBlockResolver.java | 4 +- .../ExternalShuffleBlockResolverSuite.java | 54 +++++++------- .../ExternalShuffleIntegrationSuite.java | 41 +++++++++-- 18 files changed, 217 insertions(+), 51 deletions(-) rename core/src/test/scala/org/apache/spark/{ => shuffle}/ShuffleSuite.scala (79%) diff --git a/core/src/main/java/org/apache/spark/shuffle/unsafe/UnsafeShuffleWriter.java b/core/src/main/java/org/apache/spark/shuffle/unsafe/UnsafeShuffleWriter.java index a1e278ae3a8b0..a7913f0dd07cd 100644 --- a/core/src/main/java/org/apache/spark/shuffle/unsafe/UnsafeShuffleWriter.java +++ b/core/src/main/java/org/apache/spark/shuffle/unsafe/UnsafeShuffleWriter.java @@ -90,6 +90,7 @@ private static final class MyByteArrayOutputStream extends ByteArrayOutputStream private MyByteArrayOutputStream serBuffer; private SerializationStream serOutputStream; + private final boolean allowSpillMove; /** * Are we in the process of stopping? Because map tasks can call stop() with success = true @@ -128,6 +129,7 @@ public UnsafeShuffleWriter( taskContext.taskMetrics().shuffleWriteMetrics_$eq(Option.apply(writeMetrics)); this.taskContext = taskContext; this.sparkConf = sparkConf; + this.allowSpillMove = sparkConf.getBoolean("spark.shuffle.unsafe.testing.allowSpillMove", true); this.transferToEnabled = sparkConf.getBoolean("spark.file.transferTo", true); open(); } @@ -232,7 +234,7 @@ private long[] mergeSpills(SpillInfo[] spills) throws IOException { if (spills.length == 0) { new FileOutputStream(outputFile).close(); // Create an empty file return new long[partitioner.numPartitions()]; - } else if (spills.length == 1) { + } else if (spills.length == 1 && allowSpillMove) { // 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); @@ -297,7 +299,7 @@ private long[] mergeSpillsWithFileStream( SpillInfo[] spills, File outputFile, @Nullable CompressionCodec compressionCodec) throws IOException { - assert (spills.length >= 2); + assert (spills.length >= 2 || !allowSpillMove); final int numPartitions = partitioner.numPartitions(); final long[] partitionLengths = new long[numPartitions]; final InputStream[] spillInputStreams = new FileInputStream[spills.length]; 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 d1464d3e4ea9c..a552d54dd8561 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/FileShuffleBlockResolver.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/FileShuffleBlockResolver.scala @@ -23,6 +23,8 @@ import java.util.concurrent.atomic.AtomicInteger import scala.collection.JavaConversions._ +import com.google.common.annotations.VisibleForTesting + import org.apache.spark.{Logging, SparkConf, SparkEnv} import org.apache.spark.executor.ShuffleWriteMetrics import org.apache.spark.network.buffer.{FileSegmentManagedBuffer, ManagedBuffer} @@ -206,6 +208,11 @@ private[spark] class FileShuffleBlockResolver(conf: SparkConf) shuffleStates.remove(shuffleId) cleaned } + + @VisibleForTesting + private[shuffle] def getShuffleFiles(blockId: ShuffleBlockId): Seq[File] = { + Seq(blockManager.diskBlockManager.getFile(blockId)) + } /** Remove all the blocks / files related to a particular shuffle. */ private def removeShuffleBlocks(shuffleId: ShuffleIdAndAttempt): Boolean = { 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 7bfde919002ae..3230747a928fc 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala @@ -19,16 +19,16 @@ package org.apache.spark.shuffle import java.io._ +import com.google.common.annotations.VisibleForTesting import com.google.common.io.ByteStreams import org.apache.spark.{SparkConf, SparkEnv} import org.apache.spark.network.buffer.{FileSegmentManagedBuffer, ManagedBuffer} import org.apache.spark.network.netty.SparkTransportConf +import org.apache.spark.shuffle.IndexShuffleBlockResolver.NOOP_REDUCE_ID import org.apache.spark.storage._ import org.apache.spark.util.Utils -import IndexShuffleBlockResolver.NOOP_REDUCE_ID - /** * Create and maintain the shuffle blocks' mapping between logic block and physical file location. * Data of shuffle blocks from the same map task are stored in a single consolidated data file. @@ -51,7 +51,8 @@ private[spark] class IndexShuffleBlockResolver(conf: SparkConf) extends ShuffleB stageAttemptId)) } - private def getIndexFile(shuffleId: Int, mapId: Int, stageAttemptId: Int): File = { + @VisibleForTesting + private[shuffle] def getIndexFile(shuffleId: Int, mapId: Int, stageAttemptId: Int): File = { blockManager.diskBlockManager.getFile(ShuffleIndexBlockId(shuffleId, mapId, NOOP_REDUCE_ID, stageAttemptId)) } diff --git a/core/src/main/scala/org/apache/spark/shuffle/ShuffleManager.scala b/core/src/main/scala/org/apache/spark/shuffle/ShuffleManager.scala index 2123e19a68d64..86e28030ecc10 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/ShuffleManager.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/ShuffleManager.scala @@ -17,6 +17,7 @@ package org.apache.spark.shuffle +import java.io.File import java.util.concurrent.ConcurrentHashMap import scala.collection.JavaConverters._ @@ -57,6 +58,14 @@ private[spark] trait ShuffleManager { endPartition: Int, context: TaskContext): ShuffleReader[K, C] + + private[shuffle] def getShuffleFiles( + handle: ShuffleHandle, + mapId: Int, + reduceId: Int, + stageAttemptId: Int): Seq[File] + + /** * Remove a shuffle's metadata from the ShuffleManager. * @return true if the metadata removed successfully, otherwise false. diff --git a/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleManager.scala b/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleManager.scala index d83539546fd1d..48f29b4c2cae2 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleManager.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleManager.scala @@ -17,8 +17,11 @@ package org.apache.spark.shuffle.hash +import java.io.File + import org.apache.spark._ import org.apache.spark.shuffle._ +import org.apache.spark.storage.ShuffleBlockId /** * A ShuffleManager using hashing, that creates one output file per reduce partition on each @@ -67,6 +70,16 @@ private[spark] class HashShuffleManager(conf: SparkConf) extends ShuffleManager } } + private[shuffle] override def getShuffleFiles( + handle: ShuffleHandle, + mapId: Int, + reduceId: Int, + stageAttemptId: Int): Seq[File] = { + val blockId = ShuffleBlockId(handle.shuffleId, mapId, reduceId, stageAttemptId) + fileShuffleBlockResolver.getShuffleFiles(blockId) + } + + override def shuffleBlockResolver: FileShuffleBlockResolver = { fileShuffleBlockResolver } diff --git a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleManager.scala b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleManager.scala index 99ea4f744bb0b..ff7fc877d3051 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleManager.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleManager.scala @@ -17,6 +17,7 @@ package org.apache.spark.shuffle.sort +import java.io.File import java.util.concurrent.ConcurrentHashMap import org.apache.spark.{SparkConf, TaskContext, ShuffleDependency} @@ -89,5 +90,15 @@ private[spark] class SortShuffleManager(conf: SparkConf) extends ShuffleManager override def stop(): Unit = { shuffleBlockResolver.stop() } -} + private[shuffle] override def getShuffleFiles( + handle: ShuffleHandle, + mapId: Int, + reduceId: Int, + stageAttemptId: Int): Seq[File] = { + Seq( + indexShuffleBlockResolver.getDataFile(handle.shuffleId, mapId, stageAttemptId), + indexShuffleBlockResolver.getIndexFile(handle.shuffleId, mapId, stageAttemptId) + ) + } +} diff --git a/core/src/main/scala/org/apache/spark/shuffle/unsafe/UnsafeShuffleManager.scala b/core/src/main/scala/org/apache/spark/shuffle/unsafe/UnsafeShuffleManager.scala index 2f9ef995e03ed..15089c6e2ad95 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/unsafe/UnsafeShuffleManager.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/unsafe/UnsafeShuffleManager.scala @@ -17,6 +17,7 @@ package org.apache.spark.shuffle.unsafe +import java.io.File import java.util.Collections import java.util.concurrent.ConcurrentHashMap @@ -200,6 +201,14 @@ private[spark] class UnsafeShuffleManager(conf: SparkConf) extends ShuffleManage } } + private[shuffle] override def getShuffleFiles( + handle: ShuffleHandle, + mapId: Int, + reduceId: Int, + stageAttemptId: Int): Seq[File] = { + sortShuffleManager.getShuffleFiles(handle, mapId, reduceId, stageAttemptId) + } + override val shuffleBlockResolver: IndexShuffleBlockResolver = { sortShuffleManager.shuffleBlockResolver } diff --git a/core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala b/core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala index 0b85fbe216fb3..c6449549df52b 100644 --- a/core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala +++ b/core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala @@ -398,9 +398,6 @@ class SortShuffleContextCleanerSuite extends ContextCleanerSuiteBase(classOf[Sor runGC() postGCTester.assertCleanup() - // TODO I'm not convinced this is actually testing shuffle cleanup completely -- add more - // tests (for hash, shuffle, and unsafe) before we commit this - // Make sure the broadcasted task closure no longer exists after GC. val taskClosureBroadcastId = broadcastIds.max + 1 assert(sc.env.blockManager.master.getMatchingBlockIds({ diff --git a/core/src/test/scala/org/apache/spark/ExternalShuffleServiceSuite.scala b/core/src/test/scala/org/apache/spark/ExternalShuffleServiceSuite.scala index 067fd197bc665..6ebfbdc316208 100644 --- a/core/src/test/scala/org/apache/spark/ExternalShuffleServiceSuite.scala +++ b/core/src/test/scala/org/apache/spark/ExternalShuffleServiceSuite.scala @@ -23,6 +23,7 @@ import org.apache.spark.network.TransportContext import org.apache.spark.network.netty.SparkTransportConf import org.apache.spark.network.server.TransportServer import org.apache.spark.network.shuffle.{ExternalShuffleBlockHandler, ExternalShuffleClient} +import org.apache.spark.shuffle.ShuffleSuite /** * This suite creates an external shuffle server and routes all shuffle fetches through it. @@ -78,8 +79,5 @@ class ExternalShuffleServiceSuite extends ShuffleSuite with BeforeAndAfterAll { rdd.count() } e.getMessage should include ("Fetch failure will not retry stage due to testing config") - - // TODO make sure we have some tests with stage retry (maybe just inherited from ShuffleSuite) - } } diff --git a/core/src/test/scala/org/apache/spark/HashShuffleSuite.scala b/core/src/test/scala/org/apache/spark/HashShuffleSuite.scala index 19180e88ebe0a..700f6f7bc8888 100644 --- a/core/src/test/scala/org/apache/spark/HashShuffleSuite.scala +++ b/core/src/test/scala/org/apache/spark/HashShuffleSuite.scala @@ -19,6 +19,8 @@ package org.apache.spark import org.scalatest.BeforeAndAfterAll +import org.apache.spark.shuffle.ShuffleSuite + class HashShuffleSuite extends ShuffleSuite with BeforeAndAfterAll { // This test suite should run all tests in ShuffleSuite with hash-based shuffle. diff --git a/core/src/test/scala/org/apache/spark/ShuffleNettySuite.scala b/core/src/test/scala/org/apache/spark/ShuffleNettySuite.scala index d78c99c2e1e06..58ab6bcdb1b74 100644 --- a/core/src/test/scala/org/apache/spark/ShuffleNettySuite.scala +++ b/core/src/test/scala/org/apache/spark/ShuffleNettySuite.scala @@ -19,6 +19,8 @@ package org.apache.spark import org.scalatest.BeforeAndAfterAll +import org.apache.spark.shuffle.ShuffleSuite + class ShuffleNettySuite extends ShuffleSuite with BeforeAndAfterAll { // This test suite should run all tests in ShuffleSuite with Netty shuffle mode. diff --git a/core/src/test/scala/org/apache/spark/SortShuffleSuite.scala b/core/src/test/scala/org/apache/spark/SortShuffleSuite.scala index 63358172ea1f4..5bb0d577cc089 100644 --- a/core/src/test/scala/org/apache/spark/SortShuffleSuite.scala +++ b/core/src/test/scala/org/apache/spark/SortShuffleSuite.scala @@ -19,6 +19,8 @@ package org.apache.spark import org.scalatest.BeforeAndAfterAll +import org.apache.spark.shuffle.ShuffleSuite + class SortShuffleSuite extends ShuffleSuite with BeforeAndAfterAll { // This test suite should run all tests in ShuffleSuite with sort-based shuffle. diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerFailureRecoverySuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerFailureRecoverySuite.scala index 5085304165252..276a4ad62b181 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerFailureRecoverySuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerFailureRecoverySuite.scala @@ -26,6 +26,10 @@ import org.apache.spark._ class DAGSchedulerFailureRecoverySuite extends SparkFunSuite with Logging { + // TODO we should run this with a matrix of configurations: different shufflers, + // external shuffle service, etc. But that is really pushing the question of how to run + // such a long test ... + ignore("no concurrent retries for stage attempts (SPARK-7308)") { // see SPARK-7308 for a detailed description of the conditions this is trying to recreate. // note that this is somewhat convoluted for a test case, but isn't actually very unusual diff --git a/core/src/test/scala/org/apache/spark/ShuffleSuite.scala b/core/src/test/scala/org/apache/spark/shuffle/ShuffleSuite.scala similarity index 79% rename from core/src/test/scala/org/apache/spark/ShuffleSuite.scala rename to core/src/test/scala/org/apache/spark/shuffle/ShuffleSuite.scala index 2fc2d07d4cbb9..ee032072f2b9d 100644 --- a/core/src/test/scala/org/apache/spark/ShuffleSuite.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/ShuffleSuite.scala @@ -15,16 +15,20 @@ * limitations under the License. */ -package org.apache.spark +package org.apache.spark.shuffle import org.scalatest.Matchers -import org.apache.spark.ShuffleSuite.NonJavaSerializableClass +import org.apache.spark._ +import org.apache.spark.executor.TaskMetrics import org.apache.spark.rdd.{CoGroupedRDD, OrderedRDDFunctions, RDD, ShuffledRDD, SubtractedRDD} -import org.apache.spark.scheduler.{SparkListener, SparkListenerTaskEnd} +import org.apache.spark.scheduler.{MyRDD, SparkListener, SparkListenerTaskEnd} import org.apache.spark.serializer.KryoSerializer -import org.apache.spark.storage.{ShuffleDataBlockId, ShuffleBlockId} +import org.apache.spark.shuffle.ShuffleSuite.NonJavaSerializableClass +import org.apache.spark.storage.{ShuffleBlockId, ShuffleDataBlockId} +import org.apache.spark.unsafe.memory.TaskMemoryManager import org.apache.spark.util.MutablePair +import org.apache.sparktest.TestTags.ActiveTag abstract class ShuffleSuite extends SparkFunSuite with Matchers with LocalSparkContext { @@ -315,6 +319,66 @@ abstract class ShuffleSuite extends SparkFunSuite with Matchers with LocalSparkC assert(metrics.bytesWritten === metrics.byresRead) assert(metrics.bytesWritten > 0) } + + def multipleAttemptConfs: Seq[(String, SparkConf)] = Seq("basic" -> conf) + + multipleAttemptConfs.foreach { case (name, multipleAttemptConf) => + test("multiple attempts for one task: conf = " + name, ActiveTag) { + sc = new SparkContext("local", "test", multipleAttemptConf) + val mapTrackerMaster = sc.env.mapOutputTracker.asInstanceOf[MapOutputTrackerMaster] + val manager = sc.env.shuffleManager + val taskMemoryManager = new TaskMemoryManager(sc.env.executorMemoryManager) + 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, 0, + new TaskContextImpl(0, 0, 0L, 0, taskMemoryManager, false, stageAttemptId = 0, + taskMetrics = new TaskMetrics)) + val data = (1 to 10).map { x => x -> x} + writer1.write(data.iterator) + val mapOutput = writer1.stop(true) + mapOutput.foreach { mapStatus => mapTrackerMaster.registerMapOutputs(0, Array(mapStatus))} + val reader1 = manager.getReader[Int, Int](shuffleHandle, 0, 1, + new TaskContextImpl(1, 0, 2L, 0, taskMemoryManager, false, taskMetrics = new TaskMetrics)) + reader1.read().toIndexedSeq should be (data.toIndexedSeq) + + + // 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, 1, + new TaskContextImpl(0, 0, 1L, 0, taskMemoryManager, false, stageAttemptId = 1, + taskMetrics = new TaskMetrics)) + val data2 = (11 to 20).map { x => x -> x} + writer2.write(data2.iterator) + val mapOutput2 = writer2.stop(true) + // registeringMapOutputs always blows away all previous outputs, so we won't ever find the + // previous output anymore + mapOutput2.foreach { mapStatus => mapTrackerMaster.registerMapOutputs(0, Array(mapStatus))} + + val reader2 = manager.getReader[Int, Int](shuffleHandle, 0, 1, + new TaskContextImpl(1, 0, 2L, 0, taskMemoryManager, false, taskMetrics = new TaskMetrics)) + reader2.read().toIndexedSeq should be(data2.toIndexedSeq) + + + // make sure that when the shuffle gets unregistered, we cleanup from all attempts + val shuffleFiles1 = manager.getShuffleFiles(shuffleHandle, 0, 0, 0) + val shuffleFiles2 = manager.getShuffleFiles(shuffleHandle, 0, 0, 1) + // we are relying on getShuffleFiles to be accurate. We can't be positive its correct, but + // at least this makes sure they are returning something which seems plausible + assert(shuffleFiles1.nonEmpty) + assert(shuffleFiles2.nonEmpty) + assert(shuffleFiles1.toSet.intersect(shuffleFiles2.toSet).isEmpty) + val shuffleFiles = shuffleFiles1 ++ shuffleFiles2 + shuffleFiles.foreach { file => assert(file.exists()) } + + // now unregister, and check all the files were deleted + manager.unregisterShuffle(0) + shuffleFiles.foreach { file => assert(!file.exists()) } + } + } } object ShuffleSuite { diff --git a/core/src/test/scala/org/apache/spark/shuffle/unsafe/UnsafeShuffleSuite.scala b/core/src/test/scala/org/apache/spark/shuffle/unsafe/UnsafeShuffleSuite.scala index 6351539e91e97..dcc52d7532ed3 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/unsafe/UnsafeShuffleSuite.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/unsafe/UnsafeShuffleSuite.scala @@ -25,9 +25,10 @@ import org.apache.commons.io.FileUtils import org.apache.commons.io.filefilter.TrueFileFilter import org.scalatest.BeforeAndAfterAll -import org.apache.spark.{HashPartitioner, ShuffleDependency, SparkContext, ShuffleSuite} +import org.apache.spark._ import org.apache.spark.rdd.ShuffledRDD import org.apache.spark.serializer.{JavaSerializer, KryoSerializer} +import org.apache.spark.shuffle.ShuffleSuite import org.apache.spark.util.Utils class UnsafeShuffleSuite extends ShuffleSuite with BeforeAndAfterAll { @@ -41,6 +42,19 @@ class UnsafeShuffleSuite extends ShuffleSuite with BeforeAndAfterAll { conf.set("spark.shuffle.memoryFraction", "0.5") } + override def multipleAttemptConfs: Seq[(String, SparkConf)] = { + val kryoAndNoSpillMove = conf.clone() + .set("spark.serializer", "org.apache.spark.serializer.KryoSerializer") + .set("spark.shuffle.unsafe.testing.allowSpillMove", "false") + val noCompression = kryoAndNoSpillMove.clone() + .set("spark.shuffle.compress", "false") + .set("spark.file.transferTo","false") + Seq( + "slow merge path" -> kryoAndNoSpillMove, + "filestream based fast merge" -> noCompression + ) + } + test("UnsafeShuffleManager properly cleans up files for shuffles that use the new shuffle path") { val tmpDir = Utils.createTempDir() try { diff --git a/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java b/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java index 1a09df5649f77..5fe770f14121b 100644 --- a/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java +++ b/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java @@ -86,8 +86,8 @@ public void registerExecutor( /** * Obtains a FileSegmentManagedBuffer from a shuffle block id. We expect the blockId has the - * format "shuffle_ShuffleId_MapId_ReduceId" (from ShuffleBlockId), and additionally make - * assumptions about how the hash and sort based shuffles store their data. + * format "shuffle_ShuffleId_MapId_ReduceId_StageAttemptId" (from ShuffleBlockId), and + * additionally make assumptions about how the hash and sort based shuffles store their data. */ public ManagedBuffer getBlockData(String appId, String execId, String blockId) { String[] blockIdParts = blockId.split("_"); diff --git a/network/shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolverSuite.java b/network/shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolverSuite.java index e7a82bce3240c..eacb33171da2d 100644 --- a/network/shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolverSuite.java +++ b/network/shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolverSuite.java @@ -34,9 +34,16 @@ public class ExternalShuffleBlockResolverSuite { static String sortBlock0 = "Hello!"; static String sortBlock1 = "World!"; + static String sortBlock0_1 = "supercali"; + static String sortBlock1_1 = "fragilistic"; + static String hashBlock0 = "Elementary"; static String hashBlock1 = "Tabular"; + static String hashBlock0_1 = "expiali"; + static String hashBlock1_1 = "docious"; + + static TestShuffleDataContext dataContext; static TransportConf conf = new TransportConf(new SystemPropertyConfigProvider()); @@ -49,8 +56,12 @@ public static void beforeAll() throws IOException { // Write some sort and hash data. dataContext.insertSortShuffleData(0, 0, 0, new byte[][] { sortBlock0.getBytes(), sortBlock1.getBytes() } ); + dataContext.insertSortShuffleData(0, 0, 1, + new byte[][] { sortBlock0_1.getBytes(), sortBlock1_1.getBytes() } ); dataContext.insertHashShuffleData(1, 0, 0, - new byte[][] { hashBlock0.getBytes(), hashBlock1.getBytes() } ); + new byte[][]{hashBlock0.getBytes(), hashBlock1.getBytes()}); + dataContext.insertHashShuffleData(1, 0, 1, + new byte[][] { hashBlock0_1.getBytes(), hashBlock1_1.getBytes() } ); } @AfterClass @@ -104,19 +115,10 @@ public void testSortShuffleBlocks() throws IOException { resolver.registerExecutor("app0", "exec0", dataContext.createExecutorInfo("org.apache.spark.shuffle.sort.SortShuffleManager")); - InputStream block0Stream = - resolver.getBlockData("app0", "exec0", "shuffle_0_0_0_0").createInputStream(); - String block0 = CharStreams.toString(new InputStreamReader(block0Stream)); - block0Stream.close(); - assertEquals(sortBlock0, block0); - - InputStream block1Stream = - resolver.getBlockData("app0", "exec0", "shuffle_0_0_1_0").createInputStream(); - String block1 = CharStreams.toString(new InputStreamReader(block1Stream)); - block1Stream.close(); - assertEquals(sortBlock1, block1); - - // TODO test reading from a different stage attempt + testReadBlockData(resolver, "shuffle_0_0_0_0", sortBlock0); + testReadBlockData(resolver, "shuffle_0_0_1_0", sortBlock1); + testReadBlockData(resolver, "shuffle_0_0_0_1", sortBlock0_1); + testReadBlockData(resolver, "shuffle_0_0_1_1", sortBlock1_1); } @Test @@ -125,16 +127,18 @@ public void testHashShuffleBlocks() throws IOException { resolver.registerExecutor("app0", "exec0", dataContext.createExecutorInfo("org.apache.spark.shuffle.hash.HashShuffleManager")); - InputStream block0Stream = - resolver.getBlockData("app0", "exec0", "shuffle_1_0_0_0").createInputStream(); - String block0 = CharStreams.toString(new InputStreamReader(block0Stream)); - block0Stream.close(); - assertEquals(hashBlock0, block0); - - InputStream block1Stream = - resolver.getBlockData("app0", "exec0", "shuffle_1_0_1_0").createInputStream(); - String block1 = CharStreams.toString(new InputStreamReader(block1Stream)); - block1Stream.close(); - assertEquals(hashBlock1, block1); + testReadBlockData(resolver, "shuffle_1_0_0_0", hashBlock0); + testReadBlockData(resolver, "shuffle_1_0_1_0", hashBlock1); + testReadBlockData(resolver, "shuffle_1_0_0_1", hashBlock0_1); + testReadBlockData(resolver, "shuffle_1_0_1_1", hashBlock1_1); + } + + private void testReadBlockData(ExternalShuffleBlockResolver resolver, String blockId, + String expected) throws IOException { + InputStream blockStream = + resolver.getBlockData("app0", "exec0", blockId).createInputStream(); + String block0 = CharStreams.toString(new InputStreamReader(blockStream)); + blockStream.close(); + assertEquals(expected, block0); } } diff --git a/network/shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleIntegrationSuite.java b/network/shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleIntegrationSuite.java index 21f9e9e26a929..3ad5300ad83fd 100644 --- a/network/shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleIntegrationSuite.java +++ b/network/shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleIntegrationSuite.java @@ -72,26 +72,39 @@ public class ExternalShuffleIntegrationSuite { new byte[54321], }; + static byte[][] exec0BlocksAttempt1 = new byte[][] { + new byte[789] + }; + + static byte[][] exec1BlocksAttempt1 = new byte[][] { + new byte[345], + new byte[678] + }; + + + @BeforeClass public static void beforeAll() throws IOException { Random rand = new Random(); - for (byte[] block : exec0Blocks) { - rand.nextBytes(block); - } - for (byte[] block: exec1Blocks) { - rand.nextBytes(block); + byte[][][] allBlocks = new byte[][][]{ + exec0Blocks, exec1Blocks, exec0BlocksAttempt1, exec1BlocksAttempt1}; + for (byte[][] blockGroup: allBlocks) { + for (byte[] block : blockGroup) { + rand.nextBytes(block); + } } dataContext0 = new TestShuffleDataContext(2, 5); dataContext0.create(); dataContext0.insertSortShuffleData(0, 0, 0, exec0Blocks); + dataContext0.insertSortShuffleData(0, 0, 1, exec0BlocksAttempt1); dataContext1 = new TestShuffleDataContext(6, 2); dataContext1.create(); dataContext1.insertHashShuffleData(1, 0, 0, exec1Blocks); + dataContext1.insertHashShuffleData(1, 0, 1, exec1BlocksAttempt1); - // TODO tests w/ different stage attempts conf = new TransportConf(new SystemPropertyConfigProvider()); handler = new ExternalShuffleBlockHandler(conf); @@ -181,7 +194,13 @@ public void testFetchOneSort() throws Exception { assertBufferListsEqual(exec0Fetch.buffers, Lists.newArrayList(exec0Blocks[0])); exec0Fetch.releaseBuffers(); - //TODO test fetch w/ non-zero stage attempt id + + FetchResult exec0Fetch1 = fetchBlocks("exec-0", new String[] { "shuffle_0_0_0_1" }); + assertEquals(Sets.newHashSet("shuffle_0_0_0_1"), exec0Fetch1.successBlocks); + assertTrue(exec0Fetch1.failedBlocks.isEmpty()); + assertBufferListsEqual(exec0Fetch1.buffers, Lists.newArrayList(exec0BlocksAttempt1[0])); + exec0Fetch1.releaseBuffers(); + } @Test @@ -205,6 +224,14 @@ public void testFetchHash() throws Exception { assertTrue(execFetch.failedBlocks.isEmpty()); assertBufferListsEqual(execFetch.buffers, Lists.newArrayList(exec1Blocks)); execFetch.releaseBuffers(); + + + FetchResult exec1Fetch1 = fetchBlocks("exec-1", + new String[] { "shuffle_1_0_0_1", "shuffle_1_0_1_1" }); + assertEquals(Sets.newHashSet("shuffle_1_0_0_1", "shuffle_1_0_1_1"), exec1Fetch1.successBlocks); + assertTrue(exec1Fetch1.failedBlocks.isEmpty()); + assertBufferListsEqual(exec1Fetch1.buffers, Lists.newArrayList(exec1BlocksAttempt1)); + exec1Fetch1.releaseBuffers(); } @Test From ff1870ad7cbd48ff626e96921b7d01f672eb9c3c Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Fri, 5 Jun 2015 16:54:47 -0500 Subject: [PATCH 46/73] style --- .../org/apache/spark/shuffle/FileShuffleBlockResolver.scala | 2 +- .../org/apache/spark/shuffle/unsafe/UnsafeShuffleSuite.scala | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) 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 a552d54dd8561..5bbbd7210acdc 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/FileShuffleBlockResolver.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/FileShuffleBlockResolver.scala @@ -208,7 +208,7 @@ private[spark] class FileShuffleBlockResolver(conf: SparkConf) shuffleStates.remove(shuffleId) cleaned } - + @VisibleForTesting private[shuffle] def getShuffleFiles(blockId: ShuffleBlockId): Seq[File] = { Seq(blockManager.diskBlockManager.getFile(blockId)) diff --git a/core/src/test/scala/org/apache/spark/shuffle/unsafe/UnsafeShuffleSuite.scala b/core/src/test/scala/org/apache/spark/shuffle/unsafe/UnsafeShuffleSuite.scala index dcc52d7532ed3..942818e7f0f96 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/unsafe/UnsafeShuffleSuite.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/unsafe/UnsafeShuffleSuite.scala @@ -48,7 +48,7 @@ class UnsafeShuffleSuite extends ShuffleSuite with BeforeAndAfterAll { .set("spark.shuffle.unsafe.testing.allowSpillMove", "false") val noCompression = kryoAndNoSpillMove.clone() .set("spark.shuffle.compress", "false") - .set("spark.file.transferTo","false") + .set("spark.file.transferTo", "false") Seq( "slow merge path" -> kryoAndNoSpillMove, "filestream based fast merge" -> noCompression From e2daa0507618b441ed2673ee2ca5344bbd83fc92 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Fri, 5 Jun 2015 17:07:19 -0500 Subject: [PATCH 47/73] test transferTo; remove accidental addition --- .../scala/org/apache/spark/shuffle/ShuffleSuite.scala | 3 +-- .../apache/spark/shuffle/unsafe/UnsafeShuffleSuite.scala | 9 +++++++-- 2 files changed, 8 insertions(+), 4 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/shuffle/ShuffleSuite.scala b/core/src/test/scala/org/apache/spark/shuffle/ShuffleSuite.scala index ee032072f2b9d..4ddeb2e2a2229 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/ShuffleSuite.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/ShuffleSuite.scala @@ -28,7 +28,6 @@ import org.apache.spark.shuffle.ShuffleSuite.NonJavaSerializableClass import org.apache.spark.storage.{ShuffleBlockId, ShuffleDataBlockId} import org.apache.spark.unsafe.memory.TaskMemoryManager import org.apache.spark.util.MutablePair -import org.apache.sparktest.TestTags.ActiveTag abstract class ShuffleSuite extends SparkFunSuite with Matchers with LocalSparkContext { @@ -323,7 +322,7 @@ abstract class ShuffleSuite extends SparkFunSuite with Matchers with LocalSparkC def multipleAttemptConfs: Seq[(String, SparkConf)] = Seq("basic" -> conf) multipleAttemptConfs.foreach { case (name, multipleAttemptConf) => - test("multiple attempts for one task: conf = " + name, ActiveTag) { + test("multiple attempts for one task: conf = " + name) { sc = new SparkContext("local", "test", multipleAttemptConf) val mapTrackerMaster = sc.env.mapOutputTracker.asInstanceOf[MapOutputTrackerMaster] val manager = sc.env.shuffleManager diff --git a/core/src/test/scala/org/apache/spark/shuffle/unsafe/UnsafeShuffleSuite.scala b/core/src/test/scala/org/apache/spark/shuffle/unsafe/UnsafeShuffleSuite.scala index 942818e7f0f96..c095efd3b6c9a 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/unsafe/UnsafeShuffleSuite.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/unsafe/UnsafeShuffleSuite.scala @@ -43,15 +43,20 @@ class UnsafeShuffleSuite extends ShuffleSuite with BeforeAndAfterAll { } override def multipleAttemptConfs: Seq[(String, SparkConf)] = { + // unsafe shuffle has a few different code paths, based on various configs. We want to + // make sure we stress multiple attempts under all variants val kryoAndNoSpillMove = conf.clone() .set("spark.serializer", "org.apache.spark.serializer.KryoSerializer") .set("spark.shuffle.unsafe.testing.allowSpillMove", "false") - val noCompression = kryoAndNoSpillMove.clone() + val noCompressionFileMerge = kryoAndNoSpillMove.clone() .set("spark.shuffle.compress", "false") .set("spark.file.transferTo", "false") + val noCompressionTransferTo = noCompressionFileMerge.clone() + .set("spark.file.transferTo", "true") Seq( "slow merge path" -> kryoAndNoSpillMove, - "filestream based fast merge" -> noCompression + "filestream based fast merge" -> noCompressionFileMerge, + "transferTo based fast merge" -> noCompressionTransferTo ) } From 54948a821b6d5267375029aea8ea54db843e4194 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Fri, 5 Jun 2015 20:11:03 -0500 Subject: [PATCH 48/73] have both attempts write the shuffle data simultaneously --- .../apache/spark/shuffle/ShuffleSuite.scala | 71 +++++++++++++++---- 1 file changed, 59 insertions(+), 12 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/shuffle/ShuffleSuite.scala b/core/src/test/scala/org/apache/spark/shuffle/ShuffleSuite.scala index 4ddeb2e2a2229..d68955d3b2ad1 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/ShuffleSuite.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/ShuffleSuite.scala @@ -17,6 +17,8 @@ package org.apache.spark.shuffle +import java.util.concurrent._ + import org.scalatest.Matchers import org.apache.spark._ @@ -335,14 +337,7 @@ abstract class ShuffleSuite extends SparkFunSuite with Matchers with LocalSparkC val writer1 = manager.getWriter[Int, Int](shuffleHandle, 0, 0, new TaskContextImpl(0, 0, 0L, 0, taskMemoryManager, false, stageAttemptId = 0, taskMetrics = new TaskMetrics)) - val data = (1 to 10).map { x => x -> x} - writer1.write(data.iterator) - val mapOutput = writer1.stop(true) - mapOutput.foreach { mapStatus => mapTrackerMaster.registerMapOutputs(0, Array(mapStatus))} - val reader1 = manager.getReader[Int, Int](shuffleHandle, 0, 1, - new TaskContextImpl(1, 0, 2L, 0, taskMemoryManager, false, taskMetrics = new TaskMetrics)) - reader1.read().toIndexedSeq should be (data.toIndexedSeq) - + 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 @@ -351,10 +346,23 @@ abstract class ShuffleSuite extends SparkFunSuite with Matchers with LocalSparkC new TaskContextImpl(0, 0, 1L, 0, taskMemoryManager, false, stageAttemptId = 1, taskMetrics = new TaskMetrics)) val data2 = (11 to 20).map { x => x -> x} - writer2.write(data2.iterator) - val mapOutput2 = writer2.stop(true) - // registeringMapOutputs always blows away all previous outputs, so we won't ever find the - // previous output anymore + + // interleave writes of both attempts -- we want to test that both attempts can occur + // simultaneously, and everything is still OK + val interleaver = new InterleavingIterator( + data1, {iter: Iterator[(Int,Int)] => writer1.write(iter); writer1.stop(true)}, + data2, {iter: Iterator[(Int,Int)] => writer2.write(iter); writer2.stop(true)}) + val (mapOutput1, mapOutput2) = interleaver.run() + + + // register the output from attempt 1, and try to read it + mapOutput1.foreach { mapStatus => mapTrackerMaster.registerMapOutputs(0, Array(mapStatus))} + val reader1 = manager.getReader[Int, Int](shuffleHandle, 0, 1, + new TaskContextImpl(1, 0, 2L, 0, taskMemoryManager, false, taskMetrics = new TaskMetrics)) + reader1.read().toIndexedSeq should be (data1.toIndexedSeq) + + // now for attempt 2 (registeringMapOutputs always blows away all previous outputs, so we + // won't find the output for attempt 1) mapOutput2.foreach { mapStatus => mapTrackerMaster.registerMapOutputs(0, Array(mapStatus))} val reader2 = manager.getReader[Int, Int](shuffleHandle, 0, 1, @@ -378,8 +386,47 @@ abstract class ShuffleSuite extends SparkFunSuite with Matchers with LocalSparkC shuffleFiles.foreach { file => assert(!file.exists()) } } } + } +class InterleavingIterator[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 { def mergeCombineException(x: Int, y: Int): Int = { From c2312212a720af0e28e9cd0382a449908173e98b Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Fri, 5 Jun 2015 20:26:45 -0500 Subject: [PATCH 49/73] style --- .../scala/org/apache/spark/shuffle/ShuffleSuite.scala | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/shuffle/ShuffleSuite.scala b/core/src/test/scala/org/apache/spark/shuffle/ShuffleSuite.scala index d68955d3b2ad1..bb71fd8c9ab92 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/ShuffleSuite.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/ShuffleSuite.scala @@ -350,8 +350,8 @@ abstract class ShuffleSuite extends SparkFunSuite with Matchers with LocalSparkC // interleave writes of both attempts -- we want to test that both attempts can occur // simultaneously, and everything is still OK val interleaver = new InterleavingIterator( - data1, {iter: Iterator[(Int,Int)] => writer1.write(iter); writer1.stop(true)}, - data2, {iter: Iterator[(Int,Int)] => writer2.write(iter); writer2.stop(true)}) + data1, {iter: Iterator[(Int, Int)] => writer1.write(iter); writer1.stop(true)}, + data2, {iter: Iterator[(Int, Int)] => writer2.write(iter); writer2.stop(true)}) val (mapOutput1, mapOutput2) = interleaver.run() @@ -416,13 +416,13 @@ class InterleavingIterator[T, R]( val e: ExecutorService = Executors.newFixedThreadPool(2) - def run(): (R,R) = { + 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) + (r1, r2) } } From 52eba21b26f707cdec8a686b45ea72dd56216e1e Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Mon, 8 Jun 2015 07:27:33 -0500 Subject: [PATCH 50/73] comments, names, style --- .../org/apache/spark/MapOutputTracker.scala | 13 ++++++--- .../spark/scheduler/TaskSchedulerImpl.scala | 7 ----- .../apache/spark/shuffle/ShuffleManager.scala | 28 +++++++++++++++++-- .../hash/BlockStoreShuffleFetcher.scala | 2 +- .../shuffle/hash/HashShuffleManager.scala | 1 - .../apache/spark/MapOutputTrackerSuite.scala | 6 ++-- .../apache/spark/shuffle/ShuffleSuite.scala | 2 ++ .../apache/spark/util/AkkaUtilsSuite.scala | 8 +++--- 8 files changed, 44 insertions(+), 23 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala index fe79518f08665..2aaf48c526df2 100644 --- a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala +++ b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala @@ -36,7 +36,12 @@ private[spark] case class GetMapOutputStatuses(shuffleId: Int) extends MapOutputTrackerMessage private[spark] case object StopMapOutputTracker extends MapOutputTrackerMessage -private[spark] case class MapServerAttemptSize(bmId: BlockManagerId, stageAttempt: Int, size: Long) +/** + * Metadata used on the shuffle-read side to know where to find the data from the map side. + * This corresponds to one map partition and one reduce partition, though those partitions + * are not explicitly represented -- users must know from context. + */ +private[spark] case class ServerAttemptSize(bmId: BlockManagerId, stageAttempt: Int, size: Long) /** RpcEndpoint class for MapOutputTrackerMaster */ private[spark] class MapOutputTrackerMasterEndpoint( @@ -129,7 +134,7 @@ private[spark] abstract class MapOutputTracker(conf: SparkConf) extends Logging * Called from executors to get the server URIs and output sizes of the map outputs of * a given shuffle. */ - def getServerStatuses(shuffleId: Int, reduceId: Int): Array[MapServerAttemptSize] = { + def getServerStatuses(shuffleId: Int, reduceId: Int): Array[ServerAttemptSize] = { val statuses = mapStatuses.get(shuffleId).orNull if (statuses == null) { logInfo("Don't have map outputs for shuffle " + shuffleId + ", fetching them") @@ -382,7 +387,7 @@ private[spark] object MapOutputTracker extends Logging { private def convertMapStatuses( shuffleId: Int, reduceId: Int, - statuses: Array[MapStatus]): Array[MapServerAttemptSize] = { + statuses: Array[MapStatus]): Array[ServerAttemptSize] = { assert (statuses != null) statuses.map { status => @@ -392,7 +397,7 @@ private[spark] object MapOutputTracker extends Logging { logError(msg) throw new MetadataFetchFailedException(shuffleId, reduceId, msg) } else { - MapServerAttemptSize( + ServerAttemptSize( status.location, status.stageAttemptId, status.getSizeForBlock(reduceId)) } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala index d3b37e9e43011..ed3dde0fc3055 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -163,13 +163,6 @@ private[spark] class TaskSchedulerImpl( this.synchronized { val manager = createTaskSetManager(taskSet, maxTaskFailures) activeTaskSets(taskSet.id) = manager - val taskSetsPerStage = activeTaskSets.values.filterNot(_.isZombie).groupBy(_.stageId) - taskSetsPerStage.foreach { case (stage, taskSets) => - if (taskSets.size > 1) { - logWarning("more than one active taskSet for stage " + stage) -// throw new SparkIllegalStateException("more than one active taskSet for stage " + stage) - } - } schedulableBuilder.addTaskSetManager(manager, manager.taskSet.properties) if (!isLocal && !hasReceivedTask) { diff --git a/core/src/main/scala/org/apache/spark/shuffle/ShuffleManager.scala b/core/src/main/scala/org/apache/spark/shuffle/ShuffleManager.scala index 86e28030ecc10..c176d3346a355 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/ShuffleManager.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/ShuffleManager.scala @@ -22,7 +22,9 @@ import java.util.concurrent.ConcurrentHashMap import scala.collection.JavaConverters._ -import org.apache.spark.{TaskContext, ShuffleDependency} +import com.google.common.annotations.VisibleForTesting + +import org.apache.spark.{ShuffleDependency, TaskContext} /** * Pluggable interface for shuffle systems. A ShuffleManager is created in SparkEnv on the driver @@ -59,6 +61,13 @@ private[spark] trait ShuffleManager { context: TaskContext): ShuffleReader[K, C] + /** + * Get all the files associated with the given shuffle. + * + * This method exists just so that general shuffle tests can make sure shuffle files are cleaned + * up correctly. + */ + @VisibleForTesting private[shuffle] def getShuffleFiles( handle: ShuffleHandle, mapId: Int, @@ -81,12 +90,25 @@ private[spark] trait ShuffleManager { def stop(): Unit private[this] val shuffleToAttempts = new ConcurrentHashMap[Int, ConcurrentHashMap[Int, Int]]() + + /** + * Register a stage attempt for the given shuffle, so we can clean up all attempts when + * the shuffle is unregistered + */ protected def addShuffleAttempt(shuffleId: Int, stageAttemptId: Int): Unit = { shuffleToAttempts.putIfAbsent(shuffleId, new ConcurrentHashMap[Int, Int]()) shuffleToAttempts.get(shuffleId).putIfAbsent(stageAttemptId, stageAttemptId) } - protected def stageAttemptsForShuffle(shuffleId: Int): Iterable[Int] = { - val attempts = shuffleToAttempts.get(shuffleId) + + /** + * Get all stage attempts a shuffle, so they can all be cleaned up. + * + * Calling this also cleans up internal state which tracks attempts for each shuffle, so calling + * this again for the same shuffleId will always yield an empty Iterable. + */ + @VisibleForTesting + private[shuffle] def stageAttemptsForShuffle(shuffleId: Int): Iterable[Int] = { + val attempts = shuffleToAttempts.remove(shuffleId) if (attempts == null) { Iterable[Int]() } else { diff --git a/core/src/main/scala/org/apache/spark/shuffle/hash/BlockStoreShuffleFetcher.scala b/core/src/main/scala/org/apache/spark/shuffle/hash/BlockStoreShuffleFetcher.scala index 3050eb61f5def..a086a445af2d7 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/hash/BlockStoreShuffleFetcher.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/hash/BlockStoreShuffleFetcher.scala @@ -44,7 +44,7 @@ private[hash] object BlockStoreShuffleFetcher extends Logging { shuffleId, reduceId, System.currentTimeMillis - startTime)) val splitsByAddress = new HashMap[BlockManagerId, ArrayBuffer[(Int, Int, Long)]] - for ((MapServerAttemptSize(address, stageAttempt, size), index) <- statuses.zipWithIndex) { + for ((ServerAttemptSize(address, stageAttempt, size), index) <- statuses.zipWithIndex) { splitsByAddress.getOrElseUpdate(address, ArrayBuffer()) += ((index, stageAttempt, size)) } diff --git a/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleManager.scala b/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleManager.scala index 48f29b4c2cae2..47af86ed10d48 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleManager.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleManager.scala @@ -79,7 +79,6 @@ private[spark] class HashShuffleManager(conf: SparkConf) extends ShuffleManager fileShuffleBlockResolver.getShuffleFiles(blockId) } - override def shuffleBlockResolver: FileShuffleBlockResolver = { fileShuffleBlockResolver } diff --git a/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala b/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala index 4d2510ff9137a..39633e26e641d 100644 --- a/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala +++ b/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala @@ -57,8 +57,8 @@ class MapOutputTrackerSuite extends SparkFunSuite { Array(10000L, 1000L))) val statuses = tracker.getServerStatuses(10, 0) assert(statuses.toSeq === Seq( - MapServerAttemptSize(BlockManagerId("a", "hostA", 1000), 0, size1000), - MapServerAttemptSize(BlockManagerId("b", "hostB", 1000), 0, size10000))) + ServerAttemptSize(BlockManagerId("a", "hostA", 1000), 0, size1000), + ServerAttemptSize(BlockManagerId("b", "hostB", 1000), 0, size10000))) tracker.stop() rpcEnv.shutdown() } @@ -135,7 +135,7 @@ class MapOutputTrackerSuite extends SparkFunSuite { masterTracker.incrementEpoch() slaveTracker.updateEpoch(masterTracker.getEpoch) assert(slaveTracker.getServerStatuses(10, 0).toSeq === - Seq(MapServerAttemptSize(BlockManagerId("a", "hostA", 1000), 0, size1000))) + Seq(ServerAttemptSize(BlockManagerId("a", "hostA", 1000), 0, size1000))) masterTracker.unregisterMapOutput(10, 0, BlockManagerId("a", "hostA", 1000)) masterTracker.incrementEpoch() diff --git a/core/src/test/scala/org/apache/spark/shuffle/ShuffleSuite.scala b/core/src/test/scala/org/apache/spark/shuffle/ShuffleSuite.scala index bb71fd8c9ab92..6e6065eb5a5ec 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/ShuffleSuite.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/ShuffleSuite.scala @@ -384,6 +384,8 @@ abstract class ShuffleSuite extends SparkFunSuite with Matchers with LocalSparkC // now unregister, and check all the files were deleted manager.unregisterShuffle(0) shuffleFiles.foreach { file => assert(!file.exists()) } + // also make sure shuffleToAttempts gets cleanded up + manager.stageAttemptsForShuffle(0).size should be (0) } } diff --git a/core/src/test/scala/org/apache/spark/util/AkkaUtilsSuite.scala b/core/src/test/scala/org/apache/spark/util/AkkaUtilsSuite.scala index 2dd414037311a..b2d417d21e9b3 100644 --- a/core/src/test/scala/org/apache/spark/util/AkkaUtilsSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/AkkaUtilsSuite.scala @@ -108,7 +108,7 @@ class AkkaUtilsSuite extends SparkFunSuite with LocalSparkContext with ResetSyst // this should succeed since security off assert(slaveTracker.getServerStatuses(10, 0).toSeq === - Seq(MapServerAttemptSize(BlockManagerId("a", "hostA", 1000), 0, size1000))) + Seq(ServerAttemptSize(BlockManagerId("a", "hostA", 1000), 0, size1000))) rpcEnv.shutdown() slaveRpcEnv.shutdown() @@ -154,7 +154,7 @@ class AkkaUtilsSuite extends SparkFunSuite with LocalSparkContext with ResetSyst // this should succeed since security on and passwords match assert(slaveTracker.getServerStatuses(10, 0).toSeq === - Seq(MapServerAttemptSize(BlockManagerId("a", "hostA", 1000), 0, size1000))) + Seq(ServerAttemptSize(BlockManagerId("a", "hostA", 1000), 0, size1000))) rpcEnv.shutdown() slaveRpcEnv.shutdown() @@ -233,7 +233,7 @@ class AkkaUtilsSuite extends SparkFunSuite with LocalSparkContext with ResetSyst // this should succeed since security off assert(slaveTracker.getServerStatuses(10, 0).toSeq === - Seq(MapServerAttemptSize(BlockManagerId("a", "hostA", 1000), 0, size1000))) + Seq(ServerAttemptSize(BlockManagerId("a", "hostA", 1000), 0, size1000))) rpcEnv.shutdown() slaveRpcEnv.shutdown() @@ -279,7 +279,7 @@ class AkkaUtilsSuite extends SparkFunSuite with LocalSparkContext with ResetSyst slaveTracker.updateEpoch(masterTracker.getEpoch) assert(slaveTracker.getServerStatuses(10, 0).toSeq === - Seq(MapServerAttemptSize(BlockManagerId("a", "hostA", 1000), 0, size1000))) + Seq(ServerAttemptSize(BlockManagerId("a", "hostA", 1000), 0, size1000))) rpcEnv.shutdown() slaveRpcEnv.shutdown() From 3b4159bb927b6ced615eb97408918ec88be65d8b Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Tue, 9 Jun 2015 09:45:41 -0500 Subject: [PATCH 51/73] use CopyOnWriteArraySet for stageAttempts per shuffleId, its a more appropriate concurrent collection --- .../org/apache/spark/shuffle/ShuffleManager.scala | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/shuffle/ShuffleManager.scala b/core/src/main/scala/org/apache/spark/shuffle/ShuffleManager.scala index c176d3346a355..2c730dffff849 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/ShuffleManager.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/ShuffleManager.scala @@ -18,7 +18,7 @@ package org.apache.spark.shuffle import java.io.File -import java.util.concurrent.ConcurrentHashMap +import java.util.concurrent.{CopyOnWriteArraySet, ConcurrentHashMap} import scala.collection.JavaConverters._ @@ -89,15 +89,15 @@ private[spark] trait ShuffleManager { /** Shut down this ShuffleManager. */ def stop(): Unit - private[this] val shuffleToAttempts = new ConcurrentHashMap[Int, ConcurrentHashMap[Int, Int]]() + private[this] val shuffleToAttempts = new ConcurrentHashMap[Int, CopyOnWriteArraySet[Int]]() /** * Register a stage attempt for the given shuffle, so we can clean up all attempts when * the shuffle is unregistered */ protected def addShuffleAttempt(shuffleId: Int, stageAttemptId: Int): Unit = { - shuffleToAttempts.putIfAbsent(shuffleId, new ConcurrentHashMap[Int, Int]()) - shuffleToAttempts.get(shuffleId).putIfAbsent(stageAttemptId, stageAttemptId) + shuffleToAttempts.putIfAbsent(shuffleId, new CopyOnWriteArraySet[Int]()) + shuffleToAttempts.get(shuffleId).add(stageAttemptId) } /** @@ -112,7 +112,7 @@ private[spark] trait ShuffleManager { if (attempts == null) { Iterable[Int]() } else { - attempts.values().asScala + attempts.asScala } } } From f1d5c1ccd185d4d6593d2cca98fe303f74f9fc43 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Mon, 29 Jun 2015 15:55:50 -0500 Subject: [PATCH 52/73] fix merge --- .../apache/spark/shuffle/hash/HashShuffleReaderSuite.scala | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/shuffle/hash/HashShuffleReaderSuite.scala b/core/src/test/scala/org/apache/spark/shuffle/hash/HashShuffleReaderSuite.scala index 28ca68698e3dc..555a3a79cfa28 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/hash/HashShuffleReaderSuite.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/hash/HashShuffleReaderSuite.scala @@ -104,7 +104,7 @@ class HashShuffleReaderSuite extends SparkFunSuite with LocalSparkContext { // Setup the blockManager mock so the buffer gets returned when the shuffle code tries to // fetch shuffle data. - val shuffleBlockId = ShuffleBlockId(shuffleId, mapId, reduceId) + val shuffleBlockId = ShuffleBlockId(shuffleId, mapId, reduceId, 0) when(blockManager.getBlockData(shuffleBlockId)).thenReturn(managedBuffer) when(blockManager.wrapForCompression(meq(shuffleBlockId), isA(classOf[InputStream]))) .thenAnswer(dummyCompressionFunction) @@ -117,8 +117,8 @@ class HashShuffleReaderSuite extends SparkFunSuite with LocalSparkContext { val mapOutputTracker = mock(classOf[MapOutputTracker]) // Test a scenario where all data is local, just to avoid creating a bunch of additional mocks // for the code to read data over the network. - val statuses: Array[(BlockManagerId, Long)] = - Array.fill(numMaps)((localBlockManagerId, byteOutputStream.size().toLong)) + val statuses: Array[ServerAttemptSize] = + Array.fill(numMaps)(ServerAttemptSize(localBlockManagerId, 0, byteOutputStream.size().toLong)) when(mapOutputTracker.getServerStatuses(shuffleId, reduceId)).thenReturn(statuses) // Create a mocked shuffle handle to pass into HashShuffleReader. From bcdbf5475d0cb6f9d70c90a097f35167b04b6a09 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Wed, 1 Jul 2015 12:00:30 -0500 Subject: [PATCH 53/73] actually use UnsafeShuffle in UnsafeShuffleSuite -- update mergeSpillsWithTransferTo now that its tested --- .../org/apache/spark/shuffle/unsafe/UnsafeShuffleWriter.java | 2 +- .../org/apache/spark/shuffle/unsafe/UnsafeShuffleSuite.scala | 1 + 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/core/src/main/java/org/apache/spark/shuffle/unsafe/UnsafeShuffleWriter.java b/core/src/main/java/org/apache/spark/shuffle/unsafe/UnsafeShuffleWriter.java index fd5b750cc8e75..cf169ed1ad775 100644 --- a/core/src/main/java/org/apache/spark/shuffle/unsafe/UnsafeShuffleWriter.java +++ b/core/src/main/java/org/apache/spark/shuffle/unsafe/UnsafeShuffleWriter.java @@ -367,7 +367,7 @@ private long[] mergeSpillsWithFileStream( * @return the partition lengths in the merged file. */ private long[] mergeSpillsWithTransferTo(SpillInfo[] spills, File outputFile) throws IOException { - assert (spills.length >= 2); + assert (spills.length >= 2 || !allowSpillMove); final int numPartitions = partitioner.numPartitions(); final long[] partitionLengths = new long[numPartitions]; final FileChannel[] spillInputChannels = new FileChannel[spills.length]; diff --git a/core/src/test/scala/org/apache/spark/shuffle/unsafe/UnsafeShuffleSuite.scala b/core/src/test/scala/org/apache/spark/shuffle/unsafe/UnsafeShuffleSuite.scala index c095efd3b6c9a..f6841bd1cef0f 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/unsafe/UnsafeShuffleSuite.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/unsafe/UnsafeShuffleSuite.scala @@ -46,6 +46,7 @@ class UnsafeShuffleSuite extends ShuffleSuite with BeforeAndAfterAll { // unsafe shuffle has a few different code paths, based on various configs. We want to // make sure we stress multiple attempts under all variants val kryoAndNoSpillMove = conf.clone() + .set("spark.shuffle.manager", "tungsten-sort") .set("spark.serializer", "org.apache.spark.serializer.KryoSerializer") .set("spark.shuffle.unsafe.testing.allowSpillMove", "false") val noCompressionFileMerge = kryoAndNoSpillMove.clone() From b996802d0402841ef1f9859ec97c8ef46d325f51 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Wed, 1 Jul 2015 12:27:34 -0500 Subject: [PATCH 54/73] try to clarify getWriter / clearStageAttemptsForShuffle; style --- .../org/apache/spark/shuffle/ShuffleManager.scala | 13 +++++++------ .../spark/shuffle/hash/HashShuffleManager.scala | 2 +- .../spark/shuffle/sort/SortShuffleManager.scala | 2 +- .../spark/shuffle/unsafe/UnsafeShuffleManager.scala | 2 +- .../scala/org/apache/spark/storage/BlockId.scala | 6 +++--- .../org/apache/spark/shuffle/ShuffleSuite.scala | 2 +- 6 files changed, 14 insertions(+), 13 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/shuffle/ShuffleManager.scala b/core/src/main/scala/org/apache/spark/shuffle/ShuffleManager.scala index 2c730dffff849..72e5b624646cf 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/ShuffleManager.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/ShuffleManager.scala @@ -43,7 +43,10 @@ private[spark] trait ShuffleManager { numMaps: Int, dependency: ShuffleDependency[K, V, C]): ShuffleHandle - /** Get a writer for a given partition. Called on executors by map tasks. */ + /** Get a writer for a given partition. Called on executors by map tasks. + * Implementations should call [[addShuffleAttempt]] to update internal state, so we can track + * all attempts for each shuffle. + * */ def getWriter[K, V]( handle: ShuffleHandle, mapId: Int, @@ -101,13 +104,11 @@ private[spark] trait ShuffleManager { } /** - * Get all stage attempts a shuffle, so they can all be cleaned up. - * - * Calling this also cleans up internal state which tracks attempts for each shuffle, so calling - * this again for the same shuffleId will always yield an empty Iterable. + * Clear internal state which tracks attempts for each shuffle, and return the set of attempts + * so implementations can perform extra cleanup on each attempt (eg., delete shuffle files) */ @VisibleForTesting - private[shuffle] def stageAttemptsForShuffle(shuffleId: Int): Iterable[Int] = { + private[shuffle] def clearStageAttemptsForShuffle(shuffleId: Int): Iterable[Int] = { val attempts = shuffleToAttempts.remove(shuffleId) if (attempts == null) { Iterable[Int]() diff --git a/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleManager.scala b/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleManager.scala index 47af86ed10d48..545a38751e85d 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleManager.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleManager.scala @@ -65,7 +65,7 @@ private[spark] class HashShuffleManager(conf: SparkConf) extends ShuffleManager /** Remove a shuffle's metadata from the ShuffleManager. */ override def unregisterShuffle(shuffleId: Int): Boolean = { - stageAttemptsForShuffle(shuffleId).forall { stageAttemptId => + clearStageAttemptsForShuffle(shuffleId).forall { stageAttemptId => shuffleBlockResolver.removeShuffle(ShuffleIdAndAttempt(shuffleId, stageAttemptId)) } } diff --git a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleManager.scala b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleManager.scala index ff7fc877d3051..876b11f9ceda3 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleManager.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleManager.scala @@ -72,7 +72,7 @@ private[spark] class SortShuffleManager(conf: SparkConf) extends ShuffleManager override def unregisterShuffle(shuffleId: Int): Boolean = { if (shuffleMapNumber.containsKey(shuffleId)) { val numMaps = shuffleMapNumber.remove(shuffleId) - val attempts = stageAttemptsForShuffle(shuffleId) + val attempts = clearStageAttemptsForShuffle(shuffleId) (0 until numMaps).map{ mapId => attempts.foreach { stageAttemptId => shuffleBlockResolver.removeDataByMap(shuffleId, mapId, stageAttemptId) diff --git a/core/src/main/scala/org/apache/spark/shuffle/unsafe/UnsafeShuffleManager.scala b/core/src/main/scala/org/apache/spark/shuffle/unsafe/UnsafeShuffleManager.scala index b351107db1863..67cb4f88a5853 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/unsafe/UnsafeShuffleManager.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/unsafe/UnsafeShuffleManager.scala @@ -187,7 +187,7 @@ private[spark] class UnsafeShuffleManager(conf: SparkConf) extends ShuffleManage sortShuffleManager.unregisterShuffle(shuffleId) } else { Option(numMapsForShufflesThatUsedNewPath.remove(shuffleId)).foreach { numMaps => - val attempts = stageAttemptsForShuffle(shuffleId) + val attempts = clearStageAttemptsForShuffle(shuffleId) (0 until numMaps).foreach { mapId => attempts.foreach{ stageAttemptId => shuffleBlockResolver.removeDataByMap(shuffleId, mapId, stageAttemptId) 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 63c28e43c308d..201847853e4b2 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockId.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockId.scala @@ -59,7 +59,7 @@ case class RDDBlockId(rddId: Int, splitIndex: Int) extends BlockId { case class ShuffleBlockId(shuffleId: Int, mapId: Int, reduceId: Int, stageAttemptId: Int) extends BlockId { override def name: String = { - "shuffle_" + shuffleId + "_" + mapId + "_" + reduceId + "_" + stageAttemptId + s"shuffle_${shuffleId}_${mapId}_${reduceId}_$stageAttemptId" } } @@ -67,7 +67,7 @@ case class ShuffleBlockId(shuffleId: Int, mapId: Int, reduceId: Int, stageAttemp case class ShuffleDataBlockId(shuffleId: Int, mapId: Int, stageAttemptId: Int, reduceId: Int) extends BlockId { override def name: String = { - "shuffle_" + shuffleId + "_" + mapId + "_" + reduceId + "_" + stageAttemptId + ".data" + s"shuffle_${shuffleId}_${mapId}_${reduceId}_$stageAttemptId.data" } } @@ -75,7 +75,7 @@ case class ShuffleDataBlockId(shuffleId: Int, mapId: Int, stageAttemptId: Int, r case class ShuffleIndexBlockId(shuffleId: Int, mapId: Int, stageAttemptId: Int, reduceId: Int) extends BlockId { override def name: String = { - "shuffle_" + shuffleId + "_" + mapId + "_" + reduceId + "_" + stageAttemptId + ".index" + s"shuffle_${shuffleId}_${mapId}_${reduceId}_$stageAttemptId.index" } } diff --git a/core/src/test/scala/org/apache/spark/shuffle/ShuffleSuite.scala b/core/src/test/scala/org/apache/spark/shuffle/ShuffleSuite.scala index 6e6065eb5a5ec..156eeeea9f587 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/ShuffleSuite.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/ShuffleSuite.scala @@ -385,7 +385,7 @@ abstract class ShuffleSuite extends SparkFunSuite with Matchers with LocalSparkC manager.unregisterShuffle(0) shuffleFiles.foreach { file => assert(!file.exists()) } // also make sure shuffleToAttempts gets cleanded up - manager.stageAttemptsForShuffle(0).size should be (0) + manager.clearStageAttemptsForShuffle(0).size should be (0) } } From c29fa57185bbf4e79d18dc8871cedb23db32c448 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Wed, 1 Jul 2015 12:29:27 -0500 Subject: [PATCH 55/73] rename & small tweaks to use of ShuffleIdAndAttempt --- .../shuffle/FileShuffleBlockResolver.scala | 35 ++++++++++--------- .../shuffle/hash/HashShuffleWriter.scala | 5 +-- .../hash/HashShuffleManagerSuite.scala | 12 +++---- .../spark/tools/StoragePerfTester.scala | 5 +-- 4 files changed, 30 insertions(+), 27 deletions(-) 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 5bbbd7210acdc..bf20277830440 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/FileShuffleBlockResolver.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/FileShuffleBlockResolver.scala @@ -106,12 +106,11 @@ private[spark] class FileShuffleBlockResolver(conf: SparkConf) * Get a ShuffleWriterGroup for the given map task, which will register it as complete * when the writers are closed successfully */ - def forMapTask(shuffleId: Int, mapId: Int, stageAttemptId: Int, numBuckets: Int, + def forMapTask(shuffleAndAttempt: ShuffleIdAndAttempt, mapId: Int, numBuckets: Int, serializer: Serializer, writeMetrics: ShuffleWriteMetrics): ShuffleWriterGroup = { new ShuffleWriterGroup { - shuffleStates.putIfAbsent( - ShuffleIdAndAttempt(shuffleId, stageAttemptId), new ShuffleState(numBuckets)) - private val shuffleState = shuffleStates(ShuffleIdAndAttempt(shuffleId, stageAttemptId)) + shuffleStates.putIfAbsent(shuffleAndAttempt, new ShuffleState(numBuckets)) + private val shuffleState = shuffleStates(shuffleAndAttempt) private var fileGroup: ShuffleFileGroup = null val openStartTime = System.nanoTime @@ -119,13 +118,15 @@ private[spark] class FileShuffleBlockResolver(conf: SparkConf) val writers: Array[BlockObjectWriter] = if (consolidateShuffleFiles) { fileGroup = getUnusedFileGroup() Array.tabulate[BlockObjectWriter](numBuckets) { bucketId => - val blockId = ShuffleBlockId(shuffleId, mapId, bucketId, stageAttemptId) + val blockId = ShuffleBlockId(shuffleAndAttempt.shuffleId, mapId, bucketId, + shuffleAndAttempt.stageAttemptId) blockManager.getDiskWriter(blockId, fileGroup(bucketId), serializerInstance, bufferSize, writeMetrics) } } else { Array.tabulate[BlockObjectWriter](numBuckets) { bucketId => - val blockId = ShuffleBlockId(shuffleId, mapId, bucketId, stageAttemptId) + val blockId = ShuffleBlockId(shuffleAndAttempt.shuffleId, mapId, bucketId, + shuffleAndAttempt.stageAttemptId) val blockFile = blockManager.diskBlockManager.getFile(blockId) // Because of previous failures, the shuffle file may already exist on this machine. // If so, remove it. @@ -165,10 +166,10 @@ private[spark] class FileShuffleBlockResolver(conf: SparkConf) private def newFileGroup(): ShuffleFileGroup = { val fileId = shuffleState.nextFileId.getAndIncrement() val files = Array.tabulate[File](numBuckets) { bucketId => - val filename = physicalFileName(shuffleId, bucketId, fileId) + val filename = physicalFileName(shuffleAndAttempt.shuffleId, bucketId, fileId) blockManager.diskBlockManager.getFile(filename) } - val fileGroup = new ShuffleFileGroup(shuffleId, fileId, files) + val fileGroup = new ShuffleFileGroup(shuffleAndAttempt.shuffleId, fileId, files) shuffleState.allFileGroups.add(fileGroup) fileGroup } @@ -201,11 +202,11 @@ private[spark] class FileShuffleBlockResolver(conf: SparkConf) } /** Remove all the blocks / files and metadata related to a particular shuffle. */ - def removeShuffle(shuffleId: ShuffleIdAndAttempt): Boolean = { + def removeShuffle(shuffleAndAttempt: ShuffleIdAndAttempt): Boolean = { // Do not change the ordering of this, if shuffleStates should be removed only // after the corresponding shuffle blocks have been removed - val cleaned = removeShuffleBlocks(shuffleId) - shuffleStates.remove(shuffleId) + val cleaned = removeShuffleBlocks(shuffleAndAttempt) + shuffleStates.remove(shuffleAndAttempt) cleaned } @@ -215,8 +216,8 @@ private[spark] class FileShuffleBlockResolver(conf: SparkConf) } /** Remove all the blocks / files related to a particular shuffle. */ - private def removeShuffleBlocks(shuffleId: ShuffleIdAndAttempt): Boolean = { - shuffleStates.get(shuffleId) match { + private def removeShuffleBlocks(shuffleAndAttempt: ShuffleIdAndAttempt): Boolean = { + shuffleStates.get(shuffleAndAttempt) match { case Some(state) => if (consolidateShuffleFiles) { for (fileGroup <- state.allFileGroups; file <- fileGroup.files) { @@ -224,15 +225,15 @@ private[spark] class FileShuffleBlockResolver(conf: SparkConf) } } else { for (mapId <- state.completedMapTasks; reduceId <- 0 until state.numBuckets) { - val blockId = new ShuffleBlockId(shuffleId.shuffleId, mapId, reduceId, - shuffleId.stageAttemptId) + val blockId = new ShuffleBlockId(shuffleAndAttempt.shuffleId, mapId, reduceId, + shuffleAndAttempt.stageAttemptId) blockManager.diskBlockManager.getFile(blockId).delete() } } - logInfo("Deleted all files for shuffle " + shuffleId) + logInfo("Deleted all files for shuffle " + shuffleAndAttempt) true case None => - logInfo("Could not find files for shuffle " + shuffleId + " for deleting") + logInfo("Could not find files for shuffle " + shuffleAndAttempt + " for deleting") false } } 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 cb0ed4e32007e..ce712c69d6721 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 @@ -46,8 +46,9 @@ private[spark] class HashShuffleWriter[K, V]( private val blockManager = SparkEnv.get.blockManager private val ser = Serializer.getSerializer(dep.serializer.getOrElse(null)) - private val shuffle = shuffleBlockResolver.forMapTask(dep.shuffleId, mapId, stageAttemptId, - numOutputSplits, ser, writeMetrics) + private val shuffleAndAttempt = ShuffleIdAndAttempt(dep.shuffleId, stageAttemptId) + private val shuffle = shuffleBlockResolver.forMapTask(shuffleAndAttempt, mapId, numOutputSplits, + ser, writeMetrics) /** Write a bunch of records to this task's output */ override def write(records: Iterator[Product2[K, V]]): Unit = { diff --git a/core/src/test/scala/org/apache/spark/shuffle/hash/HashShuffleManagerSuite.scala b/core/src/test/scala/org/apache/spark/shuffle/hash/HashShuffleManagerSuite.scala index 3a13df3e4a03b..a9726172af824 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/hash/HashShuffleManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/hash/HashShuffleManagerSuite.scala @@ -54,8 +54,8 @@ class HashShuffleManagerSuite extends SparkFunSuite with LocalSparkContext { val shuffleBlockResolver = SparkEnv.get.shuffleManager.shuffleBlockResolver.asInstanceOf[FileShuffleBlockResolver] - val shuffle1 = shuffleBlockResolver.forMapTask(1, 1, 0, 1, new JavaSerializer(conf), - new ShuffleWriteMetrics) + val shuffle1 = shuffleBlockResolver.forMapTask(ShuffleIdAndAttempt(1, 0), 1, 1, + new JavaSerializer(conf),new ShuffleWriteMetrics) for (writer <- shuffle1.writers) { writer.write("test1", "value") writer.write("test2", "value") @@ -67,8 +67,8 @@ class HashShuffleManagerSuite extends SparkFunSuite with LocalSparkContext { val shuffle1Segment = shuffle1.writers(0).fileSegment() shuffle1.releaseWriters(success = true) - val shuffle2 = shuffleBlockResolver.forMapTask(1, 2, 0, 1, new JavaSerializer(conf), - new ShuffleWriteMetrics) + val shuffle2 = shuffleBlockResolver.forMapTask(ShuffleIdAndAttempt(1, 0), 2, 1, + new JavaSerializer(conf), new ShuffleWriteMetrics) for (writer <- shuffle2.writers) { writer.write("test3", "value") @@ -86,8 +86,8 @@ class HashShuffleManagerSuite extends SparkFunSuite with LocalSparkContext { // of block based on remaining data in file : which could mess things up when there is // concurrent read and writes happening to the same shuffle group. - val shuffle3 = shuffleBlockResolver.forMapTask(1, 3, 0, 1, new JavaSerializer(testConf), - new ShuffleWriteMetrics) + val shuffle3 = shuffleBlockResolver.forMapTask(ShuffleIdAndAttempt(1, 0), 3, 1, + new JavaSerializer(testConf), new ShuffleWriteMetrics) for (writer <- shuffle3.writers) { writer.write("test3", "value") writer.write("test4", "value") 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 ab0e734aa7438..68eb8704ff925 100644 --- a/tools/src/main/scala/org/apache/spark/tools/StoragePerfTester.scala +++ b/tools/src/main/scala/org/apache/spark/tools/StoragePerfTester.scala @@ -21,6 +21,7 @@ import java.util.concurrent.{CountDownLatch, Executors} import java.util.concurrent.atomic.AtomicLong import org.apache.spark.executor.ShuffleWriteMetrics +import org.apache.spark.shuffle.ShuffleIdAndAttempt import org.apache.spark.{SparkConf, SparkContext} import org.apache.spark.serializer.KryoSerializer import org.apache.spark.shuffle.hash.HashShuffleManager @@ -60,8 +61,8 @@ object StoragePerfTester { val hashShuffleManager = sc.env.shuffleManager.asInstanceOf[HashShuffleManager] def writeOutputBytes(mapId: Int, total: AtomicLong): Unit = { - val shuffle = hashShuffleManager.shuffleBlockResolver.forMapTask(1, mapId, 0, numOutputSplits, - new KryoSerializer(sc.conf), new ShuffleWriteMetrics()) + val shuffle = hashShuffleManager.shuffleBlockResolver.forMapTask(ShuffleIdAndAttempt(1, 0), + mapId, numOutputSplits, new KryoSerializer(sc.conf), new ShuffleWriteMetrics()) val writers = shuffle.writers for (i <- 1 to recordsPerMap) { writers(i % numOutputSplits).write(writeKey, writeValue) From d56f8d818ca5209578fa367dd1eb501ae6f57e05 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Wed, 1 Jul 2015 12:36:51 -0500 Subject: [PATCH 56/73] numNonEmptyBlocks is only needed in constructor, doesnt need to be a member --- core/src/main/scala/org/apache/spark/scheduler/MapStatus.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/MapStatus.scala b/core/src/main/scala/org/apache/spark/scheduler/MapStatus.scala index 5e562c65c351d..7b8fecb6720ad 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/MapStatus.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/MapStatus.scala @@ -141,7 +141,7 @@ private[spark] class CompressedMapStatus( private[spark] class HighlyCompressedMapStatus private ( private[this] var loc: BlockManagerId, private[this] var _stageAttemptId: Int, - private[this] var numNonEmptyBlocks: Int, + numNonEmptyBlocks: Int, private[this] var emptyBlocks: RoaringBitmap, private[this] var avgSize: Long) extends MapStatus with Externalizable { From 55a9bb1d8617424e059e7de052ecaca154e4ab44 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Wed, 1 Jul 2015 13:09:36 -0500 Subject: [PATCH 57/73] style --- .../org/apache/spark/shuffle/hash/HashShuffleManagerSuite.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/test/scala/org/apache/spark/shuffle/hash/HashShuffleManagerSuite.scala b/core/src/test/scala/org/apache/spark/shuffle/hash/HashShuffleManagerSuite.scala index a9726172af824..29aeb2842306d 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/hash/HashShuffleManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/hash/HashShuffleManagerSuite.scala @@ -55,7 +55,7 @@ class HashShuffleManagerSuite extends SparkFunSuite with LocalSparkContext { SparkEnv.get.shuffleManager.shuffleBlockResolver.asInstanceOf[FileShuffleBlockResolver] val shuffle1 = shuffleBlockResolver.forMapTask(ShuffleIdAndAttempt(1, 0), 1, 1, - new JavaSerializer(conf),new ShuffleWriteMetrics) + new JavaSerializer(conf), new ShuffleWriteMetrics) for (writer <- shuffle1.writers) { writer.write("test1", "value") writer.write("test2", "value") From 9d1189ff1452bef2cd2fb83f5e44093c6a65b0b1 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Thu, 16 Jul 2015 11:09:55 -0500 Subject: [PATCH 58/73] fix merge --- .../main/scala/org/apache/spark/scheduler/DAGScheduler.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index 18865eb93d22b..2a2bb540bc924 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -914,7 +914,7 @@ class DAGScheduler( partitionsToCompute.map { id => val locs = getPreferredLocs(stage.rdd, id) val part = stage.rdd.partitions(id) - new ShuffleMapTask(stage.id, stage.attemptId, taskBinary, part, locs) + new ShuffleMapTask(stage.id, stage.latestInfo.attemptId, taskBinary, part, locs) } case stage: ResultStage => @@ -923,7 +923,7 @@ class DAGScheduler( val p: Int = job.partitions(id) val part = stage.rdd.partitions(p) val locs = getPreferredLocs(stage.rdd, p) - new ResultTask(stage.id, stage.attemptId, taskBinary, part, locs, id) + new ResultTask(stage.id, stage.latestInfo.attemptId, taskBinary, part, locs, id) } } } catch { From c297c78a3ecc9cefd919b4576e963f04f3e44f64 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Thu, 16 Jul 2015 11:26:06 -0500 Subject: [PATCH 59/73] get rid of printlns --- .../spark/scheduler/DAGSchedulerFailureRecoverySuite.scala | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerFailureRecoverySuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerFailureRecoverySuite.scala index 276a4ad62b181..69a3f5652ec00 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerFailureRecoverySuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerFailureRecoverySuite.scala @@ -37,7 +37,6 @@ class DAGSchedulerFailureRecoverySuite extends SparkFunSuite with Logging { // could be enough to cause havoc. (0 until 100).foreach { idx => - println(new Date() + "\ttrial " + idx) logInfo(new Date() + "\ttrial " + idx) val conf = new SparkConf().set("spark.executor.memory", "100m") @@ -54,7 +53,7 @@ class DAGSchedulerFailureRecoverySuite extends SparkFunSuite with Logging { val stage = stageCompleted.stageInfo.stageId stageFailureCount(stage) = stageFailureCount.getOrElse(stage, 0) + 1 val reason = stageCompleted.stageInfo.failureReason.get - println("stage " + stage + " failed: " + stageFailureCount(stage)) + logInfo("stage " + stage + " failed: " + stageFailureCount(stage)) } } }) From 78d9614df2d8dd1cafc54274b2603549902bcd01 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Wed, 22 Jul 2015 17:44:40 -0500 Subject: [PATCH 60/73] oops, fix merge --- .../apache/spark/util/AkkaUtilsSuite.scala | 21 +++---------------- 1 file changed, 3 insertions(+), 18 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/util/AkkaUtilsSuite.scala b/core/src/test/scala/org/apache/spark/util/AkkaUtilsSuite.scala index 03046b21bc3f4..d40fb6f69b0dd 100644 --- a/core/src/test/scala/org/apache/spark/util/AkkaUtilsSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/AkkaUtilsSuite.scala @@ -156,14 +156,9 @@ class AkkaUtilsSuite extends SparkFunSuite with LocalSparkContext with ResetSyst slaveTracker.updateEpoch(masterTracker.getEpoch) // this should succeed since security on and passwords match -<<<<<<< HEAD - assert(slaveTracker.getServerStatuses(10, 0).toSeq === - Seq(ServerAttemptSize(BlockManagerId("a", "hostA", 1000), 0, size1000))) -======= assert(slaveTracker.getMapSizesByExecutorId(10, 0) === Seq((BlockManagerId("a", "hostA", 1000), - ArrayBuffer((ShuffleBlockId(10, 0, 0), size1000))))) ->>>>>>> master + ArrayBuffer((ShuffleBlockId(10, 0, 0, 0), size1000))))) rpcEnv.shutdown() slaveRpcEnv.shutdown() @@ -241,13 +236,8 @@ class AkkaUtilsSuite extends SparkFunSuite with LocalSparkContext with ResetSyst slaveTracker.updateEpoch(masterTracker.getEpoch) // this should succeed since security off -<<<<<<< HEAD - assert(slaveTracker.getServerStatuses(10, 0).toSeq === - Seq(ServerAttemptSize(BlockManagerId("a", "hostA", 1000), 0, size1000))) -======= assert(slaveTracker.getMapSizesByExecutorId(10, 0) === - Seq((BlockManagerId("a", "hostA", 1000), ArrayBuffer((ShuffleBlockId(10, 0, 0), size1000))))) ->>>>>>> master + Seq((BlockManagerId("a", "hostA", 1000), ArrayBuffer((ShuffleBlockId(10, 0, 0, 0), size1000))))) rpcEnv.shutdown() slaveRpcEnv.shutdown() @@ -292,13 +282,8 @@ class AkkaUtilsSuite extends SparkFunSuite with LocalSparkContext with ResetSyst masterTracker.incrementEpoch() slaveTracker.updateEpoch(masterTracker.getEpoch) -<<<<<<< HEAD - assert(slaveTracker.getServerStatuses(10, 0).toSeq === - Seq(ServerAttemptSize(BlockManagerId("a", "hostA", 1000), 0, size1000))) -======= assert(slaveTracker.getMapSizesByExecutorId(10, 0) === - Seq((BlockManagerId("a", "hostA", 1000), ArrayBuffer((ShuffleBlockId(10, 0, 0), size1000))))) ->>>>>>> master + Seq((BlockManagerId("a", "hostA", 1000), ArrayBuffer((ShuffleBlockId(10, 0, 0, 0), size1000))))) rpcEnv.shutdown() slaveRpcEnv.shutdown() From 529aa959aedc6651e396a160fef8e29789264b91 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Wed, 22 Jul 2015 17:47:37 -0500 Subject: [PATCH 61/73] more fix merge --- .../org/apache/spark/shuffle/ShuffleSuite.scala | 15 +++++++++------ 1 file changed, 9 insertions(+), 6 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/shuffle/ShuffleSuite.scala b/core/src/test/scala/org/apache/spark/shuffle/ShuffleSuite.scala index e5420004821fe..4de656f085618 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/ShuffleSuite.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/ShuffleSuite.scala @@ -329,22 +329,23 @@ abstract class ShuffleSuite extends SparkFunSuite with Matchers with LocalSparkC 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, 0, - new TaskContextImpl(0, 0, 0L, 0, taskMemoryManager, false, stageAttemptId = 0, - taskMetrics = new TaskMetrics)) + new TaskContextImpl(0, 0, 0L, 0, taskMemoryManager, metricsSystem, false, + stageAttemptId = 0, taskMetrics = new TaskMetrics)) 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, 1, - new TaskContextImpl(0, 0, 1L, 0, taskMemoryManager, false, stageAttemptId = 1, - taskMetrics = new TaskMetrics)) + new TaskContextImpl(0, 0, 1L, 0, taskMemoryManager, metricsSystem, false, + stageAttemptId = 1, taskMetrics = new TaskMetrics)) val data2 = (11 to 20).map { x => x -> x} // interleave writes of both attempts -- we want to test that both attempts can occur @@ -358,7 +359,8 @@ abstract class ShuffleSuite extends SparkFunSuite with Matchers with LocalSparkC // register the output from attempt 1, and try to read it mapOutput1.foreach { mapStatus => mapTrackerMaster.registerMapOutputs(0, Array(mapStatus))} val reader1 = manager.getReader[Int, Int](shuffleHandle, 0, 1, - new TaskContextImpl(1, 0, 2L, 0, taskMemoryManager, false, taskMetrics = new TaskMetrics)) + new TaskContextImpl(1, 0, 2L, 0, taskMemoryManager, metricsSystem, false, + taskMetrics = new TaskMetrics)) reader1.read().toIndexedSeq should be (data1.toIndexedSeq) // now for attempt 2 (registeringMapOutputs always blows away all previous outputs, so we @@ -366,7 +368,8 @@ abstract class ShuffleSuite extends SparkFunSuite with Matchers with LocalSparkC mapOutput2.foreach { mapStatus => mapTrackerMaster.registerMapOutputs(0, Array(mapStatus))} val reader2 = manager.getReader[Int, Int](shuffleHandle, 0, 1, - new TaskContextImpl(1, 0, 2L, 0, taskMemoryManager, false, taskMetrics = new TaskMetrics)) + new TaskContextImpl(1, 0, 2L, 0, taskMemoryManager, metricsSystem, false, + taskMetrics = new TaskMetrics)) reader2.read().toIndexedSeq should be(data2.toIndexedSeq) From 23af9156854b56eafcaa8b10be522df8750da4a0 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Wed, 22 Jul 2015 17:53:07 -0500 Subject: [PATCH 62/73] no longer need ServerAttemptSize after other changes in master --- .../src/main/scala/org/apache/spark/MapOutputTracker.scala | 7 ------- 1 file changed, 7 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala index cd1c6e6832812..4d260819fa3d6 100644 --- a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala +++ b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala @@ -36,13 +36,6 @@ private[spark] case class GetMapOutputStatuses(shuffleId: Int) extends MapOutputTrackerMessage private[spark] case object StopMapOutputTracker extends MapOutputTrackerMessage -/** - * Metadata used on the shuffle-read side to know where to find the data from the map side. - * This corresponds to one map partition and one reduce partition, though those partitions - * are not explicitly represented -- users must know from context. - */ -private[spark] case class ServerAttemptSize(bmId: BlockManagerId, stageAttempt: Int, size: Long) - /** RpcEndpoint class for MapOutputTrackerMaster */ private[spark] class MapOutputTrackerMasterEndpoint( override val rpcEnv: RpcEnv, tracker: MapOutputTrackerMaster, conf: SparkConf) From c288ff915c13ff01c66f4dc4013fcce7a10a3e5e Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Wed, 22 Jul 2015 17:59:28 -0500 Subject: [PATCH 63/73] style --- .../scala/org/apache/spark/MapOutputTrackerSuite.scala | 10 ++++++---- .../scala/org/apache/spark/util/AkkaUtilsSuite.scala | 6 ++++-- 2 files changed, 10 insertions(+), 6 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala b/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala index 5bc1935da5c51..ba6dda0d0befa 100644 --- a/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala +++ b/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala @@ -59,9 +59,10 @@ class MapOutputTrackerSuite extends SparkFunSuite { Array(10000L, 1000L))) val statuses = tracker.getMapSizesByExecutorId(10, 0) assert(statuses.toSet === - Seq((BlockManagerId("a", "hostA", 1000), ArrayBuffer((ShuffleBlockId(10, 0, 0, 0), size1000))), - (BlockManagerId("b", "hostB", 1000), ArrayBuffer((ShuffleBlockId(10, 1, 0, 0), size10000)))) - .toSet) + Seq( + (BlockManagerId("a", "hostA", 1000), ArrayBuffer((ShuffleBlockId(10, 0, 0, 0), size1000))), + (BlockManagerId("b", "hostB", 1000), ArrayBuffer((ShuffleBlockId(10, 1, 0, 0), size10000))) + ).toSet) tracker.stop() rpcEnv.shutdown() } @@ -138,7 +139,8 @@ class MapOutputTrackerSuite extends SparkFunSuite { masterTracker.incrementEpoch() slaveTracker.updateEpoch(masterTracker.getEpoch) assert(slaveTracker.getMapSizesByExecutorId(10, 0) === - Seq((BlockManagerId("a", "hostA", 1000), ArrayBuffer((ShuffleBlockId(10, 0, 0, 0), size1000))))) + Seq((BlockManagerId("a", "hostA", 1000), + ArrayBuffer((ShuffleBlockId(10, 0, 0, 0), size1000))))) masterTracker.unregisterMapOutput(10, 0, BlockManagerId("a", "hostA", 1000)) masterTracker.incrementEpoch() diff --git a/core/src/test/scala/org/apache/spark/util/AkkaUtilsSuite.scala b/core/src/test/scala/org/apache/spark/util/AkkaUtilsSuite.scala index d40fb6f69b0dd..630fe672e3bbd 100644 --- a/core/src/test/scala/org/apache/spark/util/AkkaUtilsSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/AkkaUtilsSuite.scala @@ -237,7 +237,8 @@ class AkkaUtilsSuite extends SparkFunSuite with LocalSparkContext with ResetSyst // this should succeed since security off assert(slaveTracker.getMapSizesByExecutorId(10, 0) === - Seq((BlockManagerId("a", "hostA", 1000), ArrayBuffer((ShuffleBlockId(10, 0, 0, 0), size1000))))) + Seq((BlockManagerId("a", "hostA", 1000), + ArrayBuffer((ShuffleBlockId(10, 0, 0, 0), size1000))))) rpcEnv.shutdown() slaveRpcEnv.shutdown() @@ -283,7 +284,8 @@ class AkkaUtilsSuite extends SparkFunSuite with LocalSparkContext with ResetSyst slaveTracker.updateEpoch(masterTracker.getEpoch) assert(slaveTracker.getMapSizesByExecutorId(10, 0) === - Seq((BlockManagerId("a", "hostA", 1000), ArrayBuffer((ShuffleBlockId(10, 0, 0, 0), size1000))))) + Seq((BlockManagerId("a", "hostA", 1000), + ArrayBuffer((ShuffleBlockId(10, 0, 0, 0), size1000))))) rpcEnv.shutdown() slaveRpcEnv.shutdown() From f392acc46e110c5a37ec13f0936056dfc1d05b30 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Wed, 22 Jul 2015 21:39:21 -0500 Subject: [PATCH 64/73] use interpolation for logging --- .../org/apache/spark/scheduler/TaskSetManager.scala | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) 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 26c8c2323594c..123e8c75cb454 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala @@ -631,10 +631,11 @@ private[spark] class TaskSetManager( task, Success, result.value(), result.accumUpdates, info, result.metrics) if (!successful(index)) { tasksSuccessful += 1 - // include the partition here b/c on a retry, the partition is *not* the same as info.id - logInfo(("Finished task %s in stage %s (TID %d, partition %d) in %d ms on executor %s (%s) " + - "(%d/%d)").format(info.id, taskSet.id, task.partitionId, info.taskId, info.duration, - info.executorId, info.host, tasksSuccessful, numTasks)) + // include the partition here b/c on a partial 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) { From 9cd9c75df281eda6a39540760ff3d07a6d3fe426 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Wed, 22 Jul 2015 21:59:44 -0500 Subject: [PATCH 65/73] style, comments --- .../spark/shuffle/FileShuffleBlockResolver.scala | 8 ++++++-- .../spark/shuffle/IndexShuffleBlockResolver.scala | 8 ++++---- .../scala/org/apache/spark/shuffle/ShuffleSuite.scala | 10 ++++++++-- 3 files changed, 18 insertions(+), 8 deletions(-) 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 60c7ce43eb31a..787bee505ccfe 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/FileShuffleBlockResolver.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/FileShuffleBlockResolver.scala @@ -106,8 +106,12 @@ private[spark] class FileShuffleBlockResolver(conf: SparkConf) * Get a ShuffleWriterGroup for the given map task, which will register it as complete * when the writers are closed successfully */ - def forMapTask(shuffleAndAttempt: ShuffleIdAndAttempt, mapId: Int, numBuckets: Int, - serializer: Serializer, writeMetrics: ShuffleWriteMetrics): ShuffleWriterGroup = { + def forMapTask( + shuffleAndAttempt: ShuffleIdAndAttempt, + mapId: Int, + numBuckets: Int, + serializer: Serializer, + writeMetrics: ShuffleWriteMetrics): ShuffleWriterGroup = { new ShuffleWriterGroup { shuffleStates.putIfAbsent(shuffleAndAttempt, new ShuffleState(numBuckets)) private val shuffleState = shuffleStates(shuffleAndAttempt) 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 14ac72958f2b6..e8f19516f86c3 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala @@ -47,14 +47,14 @@ private[spark] class IndexShuffleBlockResolver(conf: SparkConf) extends ShuffleB private val transportConf = SparkTransportConf.fromSparkConf(conf) def getDataFile(shuffleId: Int, mapId: Int, stageAttemptId: Int): File = { - blockManager.diskBlockManager.getFile(ShuffleDataBlockId(shuffleId, mapId, NOOP_REDUCE_ID, - stageAttemptId)) + blockManager.diskBlockManager.getFile( + ShuffleDataBlockId(shuffleId, mapId, NOOP_REDUCE_ID, stageAttemptId)) } @VisibleForTesting private[shuffle] def getIndexFile(shuffleId: Int, mapId: Int, stageAttemptId: Int): File = { - blockManager.diskBlockManager.getFile(ShuffleIndexBlockId(shuffleId, mapId, NOOP_REDUCE_ID, - stageAttemptId)) + blockManager.diskBlockManager.getFile( + ShuffleIndexBlockId(shuffleId, mapId, NOOP_REDUCE_ID, stageAttemptId)) } /** diff --git a/core/src/test/scala/org/apache/spark/shuffle/ShuffleSuite.scala b/core/src/test/scala/org/apache/spark/shuffle/ShuffleSuite.scala index d5b9d42e54851..a08c5f8263dbf 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/ShuffleSuite.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/ShuffleSuite.scala @@ -350,7 +350,7 @@ abstract class ShuffleSuite extends SparkFunSuite with Matchers with LocalSparkC // interleave writes of both attempts -- we want to test that both attempts can occur // simultaneously, and everything is still OK - val interleaver = new InterleavingIterator( + val interleaver = new InterleaveIterators( data1, {iter: Iterator[(Int, Int)] => writer1.write(iter); writer1.stop(true)}, data2, {iter: Iterator[(Int, Int)] => writer2.write(iter); writer2.stop(true)}) val (mapOutput1, mapOutput2) = interleaver.run() @@ -394,7 +394,13 @@ abstract class ShuffleSuite extends SparkFunSuite with Matchers with LocalSparkC } -class InterleavingIterator[T, R]( +/** + * 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], From 5547611896f644b84cdd8651c96b2efcd8a56e7c Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Thu, 8 Oct 2015 09:31:39 -0500 Subject: [PATCH 66/73] fix merge --- .../apache/spark/shuffle/FileShuffleBlockResolver.scala | 4 +++- .../scala/org/apache/spark/shuffle/ShuffleSuite.scala | 8 ++++---- .../spark/sql/execution/UnsafeRowSerializerSuite.scala | 2 +- 3 files changed, 8 insertions(+), 6 deletions(-) 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 ebc3c2cfbb87d..05c38446bb6a3 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._ @@ -88,7 +89,8 @@ private[spark] class FileShuffleBlockResolver(conf: SparkConf) val serializerInstance = serializer.newInstance() val writers: Array[DiskBlockObjectWriter] = { Array.tabulate[DiskBlockObjectWriter](numReducers) { bucketId => - val blockId = ShuffleBlockId(shuffleAndAttempt.shuffleId, mapId, bucketId) + val blockId = ShuffleBlockId(shuffleAndAttempt.shuffleId, mapId, bucketId, + shuffleAndAttempt.stageAttemptId) val blockFile = blockManager.diskBlockManager.getFile(blockId) // Because of previous failures, the shuffle file may already exist on this machine. // If so, remove it. diff --git a/core/src/test/scala/org/apache/spark/shuffle/ShuffleSuite.scala b/core/src/test/scala/org/apache/spark/shuffle/ShuffleSuite.scala index 0658e4ff06341..77842fd55b1d7 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/ShuffleSuite.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/ShuffleSuite.scala @@ -337,7 +337,7 @@ abstract class ShuffleSuite extends SparkFunSuite with Matchers with LocalSparkC // first attempt -- its successful val writer1 = manager.getWriter[Int, Int](shuffleHandle, 0, 0, new TaskContextImpl(0, 0, 0L, 0, taskMemoryManager, metricsSystem, - InternalAccumulator.create, false, stageAttemptId = 0, taskMetrics = new TaskMetrics)) + InternalAccumulator.create(sc), false, stageAttemptId = 0, taskMetrics = new TaskMetrics)) val data1 = (1 to 10).map { x => x -> x} // second attempt -- also successful. We'll write out different data, @@ -345,7 +345,7 @@ abstract class ShuffleSuite extends SparkFunSuite with Matchers with LocalSparkC // depending on what gets spilled, what gets combined, etc. val writer2 = manager.getWriter[Int, Int](shuffleHandle, 0, 1, new TaskContextImpl(0, 0, 1L, 0, taskMemoryManager, metricsSystem, - InternalAccumulator.create, false, stageAttemptId = 1, taskMetrics = new TaskMetrics)) + InternalAccumulator.create(sc), false, stageAttemptId = 1, taskMetrics = new TaskMetrics)) val data2 = (11 to 20).map { x => x -> x} // interleave writes of both attempts -- we want to test that both attempts can occur @@ -360,7 +360,7 @@ abstract class ShuffleSuite extends SparkFunSuite with Matchers with LocalSparkC mapOutput1.foreach { mapStatus => mapTrackerMaster.registerMapOutputs(0, Array(mapStatus))} val reader1 = manager.getReader[Int, Int](shuffleHandle, 0, 1, new TaskContextImpl(1, 0, 2L, 0, taskMemoryManager, metricsSystem, - InternalAccumulator.create, false, taskMetrics = new TaskMetrics)) + InternalAccumulator.create(sc), false, taskMetrics = new TaskMetrics)) reader1.read().toIndexedSeq should be (data1.toIndexedSeq) // now for attempt 2 (registeringMapOutputs always blows away all previous outputs, so we @@ -369,7 +369,7 @@ abstract class ShuffleSuite extends SparkFunSuite with Matchers with LocalSparkC val reader2 = manager.getReader[Int, Int](shuffleHandle, 0, 1, new TaskContextImpl(1, 0, 2L, 0, taskMemoryManager, metricsSystem, - InternalAccumulator.create, false, taskMetrics = new TaskMetrics)) + InternalAccumulator.create(sc), false, taskMetrics = new TaskMetrics)) reader2.read().toIndexedSeq should be(data2.toIndexedSeq) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/UnsafeRowSerializerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/UnsafeRowSerializerSuite.scala index f7d48bc53ebbc..b1f8338d7ee23 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/UnsafeRowSerializerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/UnsafeRowSerializerSuite.scala @@ -125,7 +125,7 @@ class UnsafeRowSerializerSuite extends SparkFunSuite with LocalSparkContext { val taskContext = new TaskContextImpl(0, 0, 0, 0, null, null, InternalAccumulator.create(sc)) taskContext.taskMetrics.shuffleWriteMetrics = Some(new ShuffleWriteMetrics) - sorter.writePartitionedFile(ShuffleBlockId(0, 0, 0), taskContext, outputFile) + sorter.writePartitionedFile(ShuffleBlockId(0, 0, 0, 0), taskContext, outputFile) } { // Clean up if (sc != null) { From c7b3017487ee0d2f0a34fb29ba7dc4875a51ffcb Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Thu, 8 Oct 2015 09:57:32 -0500 Subject: [PATCH 67/73] style --- core/src/main/scala/org/apache/spark/MapOutputTracker.scala | 3 ++- .../org/apache/spark/shuffle/FileShuffleBlockResolver.scala | 3 ++- 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala index c626d6964a674..cddcf9ddccc22 100644 --- a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala +++ b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala @@ -543,7 +543,8 @@ private[spark] object MapOutputTracker extends Logging { } else { for (part <- startPartition until endPartition) { splitsByAddress.getOrElseUpdate(status.location, ArrayBuffer()) += - ((ShuffleBlockId(shuffleId, mapId, part, status.stageAttemptId), status.getSizeForBlock(part))) + ((ShuffleBlockId(shuffleId, mapId, part, status.stageAttemptId), + status.getSizeForBlock(part))) } } } 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 05c38446bb6a3..3f1ba872fea71 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/FileShuffleBlockResolver.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/FileShuffleBlockResolver.scala @@ -139,7 +139,8 @@ private[spark] class FileShuffleBlockResolver(conf: SparkConf) shuffleStates.get(shuffleAndAttempt) match { case Some(state) => for (mapId <- state.completedMapTasks.asScala; reduceId <- 0 until state.numReducers) { - val blockId = new ShuffleBlockId(shuffleAndAttempt.shuffleId, mapId, reduceId, shuffleAndAttempt.stageAttemptId) + val blockId = new ShuffleBlockId(shuffleAndAttempt.shuffleId, mapId, reduceId, + shuffleAndAttempt.stageAttemptId) val file = blockManager.diskBlockManager.getFile(blockId) if (!file.delete()) { logWarning(s"Error deleting ${file.getPath()}") From 812aa0ed9c37bb83cbf75f3e1302e12daba33ac1 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Thu, 8 Oct 2015 10:41:30 -0500 Subject: [PATCH 68/73] FetchFailed should include the stageAttemptId, so it has a full block id --- .../shuffle/unsafe/UnsafeShuffleWriter.java | 4 +++ .../org/apache/spark/TaskEndReason.scala | 3 +- .../apache/spark/scheduler/DAGScheduler.scala | 2 +- .../apache/spark/scheduler/JobLogger.scala | 2 +- .../spark/scheduler/TaskSetManager.scala | 2 +- .../spark/shuffle/FetchFailedException.scala | 2 +- .../org/apache/spark/util/JsonProtocol.scala | 5 ++- .../spark/scheduler/DAGSchedulerSuite.scala | 31 ++++++++++--------- .../ui/jobs/JobProgressListenerSuite.scala | 2 +- .../apache/spark/util/JsonProtocolSuite.scala | 13 ++++++-- 10 files changed, 42 insertions(+), 24 deletions(-) diff --git a/core/src/main/java/org/apache/spark/shuffle/unsafe/UnsafeShuffleWriter.java b/core/src/main/java/org/apache/spark/shuffle/unsafe/UnsafeShuffleWriter.java index e3a3f42414ae4..a2b536923076e 100644 --- a/core/src/main/java/org/apache/spark/shuffle/unsafe/UnsafeShuffleWriter.java +++ b/core/src/main/java/org/apache/spark/shuffle/unsafe/UnsafeShuffleWriter.java @@ -92,6 +92,10 @@ private static final class MyByteArrayOutputStream extends ByteArrayOutputStream private MyByteArrayOutputStream serBuffer; private SerializationStream serOutputStream; + /** + * This is just to allow tests to explore more code paths, without requiring too much complexity + * in the test cases. In normal usage, it will be true. + */ private final boolean allowSpillMove; /** diff --git a/core/src/main/scala/org/apache/spark/TaskEndReason.scala b/core/src/main/scala/org/apache/spark/TaskEndReason.scala index 9335c5f4160bf..fda29553562d3 100644 --- a/core/src/main/scala/org/apache/spark/TaskEndReason.scala +++ b/core/src/main/scala/org/apache/spark/TaskEndReason.scala @@ -78,12 +78,13 @@ case class FetchFailed( shuffleId: Int, mapId: Int, reduceId: Int, + stageAttemptId: Int, message: String) extends TaskFailedReason { override def toErrorString: String = { val bmAddressString = if (bmAddress == null) "null" else bmAddress.toString s"FetchFailed($bmAddressString, shuffleId=$shuffleId, mapId=$mapId, reduceId=$reduceId, " + - s"message=\n$message\n)" + s"stageAttemptId=$stageAttemptId, message=\n$message\n)" } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index ade372be092ae..aa8e3861d5e61 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -1228,7 +1228,7 @@ class DAGScheduler( logInfo("Resubmitted " + task + ", so marking it as still running") stage.pendingPartitions += task.partitionId - case FetchFailed(bmAddress, shuffleId, mapId, reduceId, failureMessage) => + case FetchFailed(bmAddress, shuffleId, mapId, reduceId, stageAttemptId, failureMessage) => val failedStage = stageIdToStage(task.stageId) val mapStage = shuffleToMapStage(shuffleId) diff --git a/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala b/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala index f96eb8ca0ae00..2f0af776df759 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala @@ -223,7 +223,7 @@ class JobLogger(val user: String, val logDirName: String) extends SparkListener taskStatus += " STATUS=RESUBMITTED TID=" + taskInfo.taskId + " STAGE_ID=" + taskEnd.stageId stageLogInfo(taskEnd.stageId, taskStatus) - case FetchFailed(bmAddress, shuffleId, mapId, reduceId, message) => + case FetchFailed(bmAddress, shuffleId, mapId, reduceId, stageAttemptId, message) => taskStatus += " STATUS=FETCHFAILED TID=" + taskInfo.taskId + " STAGE_ID=" + taskEnd.stageId + " SHUFFLE_ID=" + shuffleId + " MAP_ID=" + mapId + " REDUCE_ID=" + reduceId 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 05a9036a45688..99d709f626df1 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala @@ -631,7 +631,7 @@ private[spark] class TaskSetManager( task, Success, result.value(), result.accumUpdates, info, result.metrics) if (!successful(index)) { tasksSuccessful += 1 - // include the partition here b/c on a partial retry, the partition is *not* necessarily + // 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} " + diff --git a/core/src/main/scala/org/apache/spark/shuffle/FetchFailedException.scala b/core/src/main/scala/org/apache/spark/shuffle/FetchFailedException.scala index fd848be90cc2d..46b2395c181d4 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/FetchFailedException.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/FetchFailedException.scala @@ -48,7 +48,7 @@ private[spark] class FetchFailedException( } def toTaskEndReason: TaskEndReason = FetchFailed(bmAddress, shuffleId, mapId, reduceId, - Utils.exceptionString(this)) + stageAttemptId, Utils.exceptionString(this)) } /** diff --git a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala index 40729fa5a4ffe..3e43615f0947f 100644 --- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -353,6 +353,7 @@ private[spark] object JsonProtocol { ("Shuffle ID" -> fetchFailed.shuffleId) ~ ("Map ID" -> fetchFailed.mapId) ~ ("Reduce ID" -> fetchFailed.reduceId) ~ + ("Stage Attempt ID" -> fetchFailed.stageAttemptId) ~ ("Message" -> fetchFailed.message) case exceptionFailure: ExceptionFailure => val stackTrace = stackTraceToJson(exceptionFailure.stackTrace) @@ -786,8 +787,10 @@ private[spark] object JsonProtocol { val shuffleId = (json \ "Shuffle ID").extract[Int] val mapId = (json \ "Map ID").extract[Int] val reduceId = (json \ "Reduce ID").extract[Int] + val stageAttemptId = Utils.jsonOption(json \ "Stage Attempt ID").map(_.extract[Int]) + .getOrElse(0) val message = Utils.jsonOption(json \ "Message").map(_.extract[String]) - new FetchFailed(blockManagerAddress, shuffleId, mapId, reduceId, + new FetchFailed(blockManagerAddress, shuffleId, mapId, reduceId, stageAttemptId, message.getOrElse("Unknown reason")) case `exceptionFailure` => val className = (json \ "Class Name").extract[String] diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala index 2c8ce01dbe37f..e17a133e1cb84 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -505,7 +505,7 @@ class DAGSchedulerSuite // the 2nd ResultTask failed complete(taskSets(1), Seq( (Success, 42), - (FetchFailed(makeBlockManagerId("hostA"), shuffleId, 0, 0, "ignored"), null))) + (FetchFailed(makeBlockManagerId("hostA"), shuffleId, 0, 0, 0, "ignored"), null))) // this will get called // blockManagerMaster.removeExecutor("exec-hostA") // ask the scheduler to try it again @@ -583,7 +583,8 @@ class DAGSchedulerSuite val stageAttempt = taskSets.last checkStageId(stageId, attemptIdx, stageAttempt) complete(stageAttempt, stageAttempt.tasks.zipWithIndex.map { case (task, idx) => - (FetchFailed(makeBlockManagerId("hostA"), shuffleDep.shuffleId, 0, idx, "ignored"), null) + (FetchFailed(makeBlockManagerId("hostA"), shuffleDep.shuffleId, 0, idx, attemptIdx, + "ignored"), null) }.toSeq) } @@ -810,7 +811,7 @@ class DAGSchedulerSuite // The first result task fails, with a fetch failure for the output from the first mapper. runEvent(CompletionEvent( taskSets(1).tasks(0), - FetchFailed(makeBlockManagerId("hostA"), shuffleId, 0, 0, "ignored"), + FetchFailed(makeBlockManagerId("hostA"), shuffleId, 0, 0, 0, "ignored"), null, Map[Long, Any](), createFakeTaskInfo(), @@ -821,7 +822,7 @@ class DAGSchedulerSuite // The second ResultTask fails, with a fetch failure for the output from the second mapper. runEvent(CompletionEvent( taskSets(1).tasks(0), - FetchFailed(makeBlockManagerId("hostA"), shuffleId, 1, 1, "ignored"), + FetchFailed(makeBlockManagerId("hostA"), shuffleId, 1, 1, 0, "ignored"), null, Map[Long, Any](), createFakeTaskInfo(), @@ -863,7 +864,7 @@ class DAGSchedulerSuite // The first result task fails, with a fetch failure for the output from the first mapper. runEvent(CompletionEvent( taskSets(1).tasks(0), - FetchFailed(makeBlockManagerId("hostA"), shuffleId, 0, 0, "ignored"), + FetchFailed(makeBlockManagerId("hostA"), shuffleId, 0, 0, 0, "ignored"), null, Map[Long, Any](), createFakeTaskInfo(), @@ -881,7 +882,7 @@ class DAGSchedulerSuite // The second ResultTask fails, with a fetch failure for the output from the second mapper. runEvent(CompletionEvent( taskSets(1).tasks(1), - FetchFailed(makeBlockManagerId("hostB"), shuffleId, 1, 1, "ignored"), + FetchFailed(makeBlockManagerId("hostB"), shuffleId, 1, 1, 0, "ignored"), null, Map[Long, Any](), createFakeTaskInfo(), @@ -933,7 +934,7 @@ class DAGSchedulerSuite // The first result task fails, with a fetch failure for the output from the first mapper. runEvent(CompletionEvent( taskSets(1).tasks(0), - FetchFailed(makeBlockManagerId("hostA"), shuffleId, 0, 0, "ignored"), + FetchFailed(makeBlockManagerId("hostA"), shuffleId, 0, 0, 0, "ignored"), null, Map[Long, Any](), createFakeTaskInfo(), @@ -952,7 +953,7 @@ class DAGSchedulerSuite // A late FetchFailed arrives from the second task in the original reduce stage. runEvent(CompletionEvent( taskSets(1).tasks(1), - FetchFailed(makeBlockManagerId("hostB"), shuffleId, 1, 1, "ignored"), + FetchFailed(makeBlockManagerId("hostB"), shuffleId, 1, 1, 0, "ignored"), null, Map[Long, Any](), createFakeTaskInfo(), @@ -1080,7 +1081,7 @@ class DAGSchedulerSuite runEvent(ExecutorLost("exec-hostA")) runEvent(CompletionEvent( taskSets(1).tasks(0), - FetchFailed(null, firstShuffleId, 2, 0, "Fetch failed"), + FetchFailed(null, firstShuffleId, 2, 0, 0, "Fetch failed"), null, null, createFakeTaskInfo(), @@ -1315,7 +1316,8 @@ class DAGSchedulerSuite (Success, makeMapStatus("hostC", 1)))) // fail the third stage because hostA went down complete(taskSets(2), Seq( - (FetchFailed(makeBlockManagerId("hostA"), shuffleDepTwo.shuffleId, 0, 0, "ignored"), null))) + (FetchFailed(makeBlockManagerId("hostA"), shuffleDepTwo.shuffleId, 0, 0, 0, "ignored"), + null))) // TODO assert this: // blockManagerMaster.removeExecutor("exec-hostA") // have DAGScheduler try again @@ -1346,7 +1348,8 @@ class DAGSchedulerSuite (Success, makeMapStatus("hostB", 1)))) // pretend stage 2 failed because hostA went down complete(taskSets(2), Seq( - (FetchFailed(makeBlockManagerId("hostA"), shuffleDepTwo.shuffleId, 0, 0, "ignored"), null))) + (FetchFailed(makeBlockManagerId("hostA"), shuffleDepTwo.shuffleId, 0, 0, 0, "ignored"), + null))) // TODO assert this: // blockManagerMaster.removeExecutor("exec-hostA") // DAGScheduler should notice the cached copy of the second shuffle and try to get it rerun. @@ -1591,7 +1594,7 @@ class DAGSchedulerSuite submit(reduceRdd, Array(0, 1)) complete(taskSets(1), Seq( (Success, 42), - (FetchFailed(makeBlockManagerId("hostA"), shuffleId, 0, 0, "ignored"), null))) + (FetchFailed(makeBlockManagerId("hostA"), shuffleId, 0, 0, 0, "ignored"), null))) // Ask the scheduler to try it again; TaskSet 2 will rerun the map task that we couldn't fetch // from, then TaskSet 3 will run the reduce stage scheduler.resubmitFailedStages() @@ -1650,7 +1653,7 @@ class DAGSchedulerSuite assert(taskSets(1).stageId === 1) complete(taskSets(1), Seq( (Success, makeMapStatus("hostA", rdd2.partitions.length)), - (FetchFailed(makeBlockManagerId("hostA"), dep1.shuffleId, 0, 0, "ignored"), null))) + (FetchFailed(makeBlockManagerId("hostA"), dep1.shuffleId, 0, 0, 0, "ignored"), null))) scheduler.resubmitFailedStages() assert(listener2.results.size === 0) // Second stage listener should not have a result yet @@ -1676,7 +1679,7 @@ class DAGSchedulerSuite assert(taskSets(4).stageId === 2) complete(taskSets(4), Seq( (Success, 52), - (FetchFailed(makeBlockManagerId("hostD"), dep2.shuffleId, 0, 0, "ignored"), null))) + (FetchFailed(makeBlockManagerId("hostD"), dep2.shuffleId, 0, 0, 0, "ignored"), null))) scheduler.resubmitFailedStages() // TaskSet 5 will rerun stage 1's lost task, then TaskSet 6 will rerun stage 2 diff --git a/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala b/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala index b140387d309f3..6bb626cf22fa2 100644 --- a/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala @@ -239,7 +239,7 @@ class JobProgressListenerSuite extends SparkFunSuite with LocalSparkContext with // Go through all the failure cases to make sure we are counting them as failures. val taskFailedReasons = Seq( Resubmitted, - new FetchFailed(null, 0, 0, 0, "ignored"), + new FetchFailed(null, 0, 0, 0, 0, "ignored"), ExceptionFailure("Exception", "description", null, null, None, None), TaskResultLost, TaskKilled, diff --git a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala index 132046aa9a0de..622a278d7c436 100644 --- a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala @@ -140,7 +140,7 @@ class JsonProtocolSuite extends SparkFunSuite { // TaskEndReason val fetchFailed = FetchFailed(BlockManagerId("With or", "without you", 15), 17, 18, 19, - "Some exception") + 20, "Some exception") val fetchMetadataFailed = new MetadataFetchFailedException(17, 19, "metadata Fetch failed exception").toTaskEndReason val exceptionFailure = new ExceptionFailure(exception, None) @@ -261,13 +261,20 @@ class JsonProtocolSuite extends SparkFunSuite { test("FetchFailed backwards compatibility") { // FetchFailed in Spark 1.1.0 does not have an "Message" property. - val fetchFailed = FetchFailed(BlockManagerId("With or", "without you", 15), 17, 18, 19, + val fetchFailed = FetchFailed(BlockManagerId("With or", "without you", 15), 17, 18, 19, 20, "ignored") val oldEvent = JsonProtocol.taskEndReasonToJson(fetchFailed) .removeField({ _._1 == "Message" }) val expectedFetchFailed = FetchFailed(BlockManagerId("With or", "without you", 15), 17, 18, 19, - "Unknown reason") + 20, "Unknown reason") assert(expectedFetchFailed === JsonProtocol.taskEndReasonFromJson(oldEvent)) + + // FetchFailed pre Spark 1.6.0 does not have "Stage Attempt ID" property + val pre16Event = JsonProtocol.taskEndReasonToJson(fetchFailed) + .removeField({ _._1 == "Stage Attempt ID" }) + val expectedPre16FetchFailed = FetchFailed(BlockManagerId("With or", "without you", 15), 17, 18, + 19, 0, "ignored") + assert(expectedPre16FetchFailed === JsonProtocol.taskEndReasonFromJson(pre16Event)) } test("ShuffleReadMetrics: Local bytes read and time taken backwards compatibility") { From 26baad9a3c8b8bf3e841857c77cf9f0236b90fcb Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Thu, 8 Oct 2015 11:16:35 -0500 Subject: [PATCH 69/73] style --- .../org/apache/spark/shuffle/sort/SortShuffleManager.scala | 2 +- .../org/apache/spark/shuffle/unsafe/UnsafeShuffleManager.scala | 2 +- .../src/test/scala/org/apache/spark/shuffle/ShuffleSuite.scala | 3 --- 3 files changed, 2 insertions(+), 5 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleManager.scala b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleManager.scala index 073974e2708dd..664f7add8dcee 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleManager.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleManager.scala @@ -78,7 +78,7 @@ private[spark] class SortShuffleManager(conf: SparkConf) extends ShuffleManager if (shuffleMapNumber.containsKey(shuffleId)) { val numMaps = shuffleMapNumber.remove(shuffleId) val attempts = clearStageAttemptsForShuffle(shuffleId) - (0 until numMaps).map{ mapId => + (0 until numMaps).foreach { mapId => attempts.foreach { stageAttemptId => shuffleBlockResolver.removeDataByMap(shuffleId, mapId, stageAttemptId) } diff --git a/core/src/main/scala/org/apache/spark/shuffle/unsafe/UnsafeShuffleManager.scala b/core/src/main/scala/org/apache/spark/shuffle/unsafe/UnsafeShuffleManager.scala index 53aef458832ef..43bc815715dfe 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/unsafe/UnsafeShuffleManager.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/unsafe/UnsafeShuffleManager.scala @@ -189,7 +189,7 @@ private[spark] class UnsafeShuffleManager(conf: SparkConf) extends ShuffleManage Option(numMapsForShufflesThatUsedNewPath.remove(shuffleId)).foreach { numMaps => val attempts = clearStageAttemptsForShuffle(shuffleId) (0 until numMaps).foreach { mapId => - attempts.foreach{ stageAttemptId => + attempts.foreach { stageAttemptId => shuffleBlockResolver.removeDataByMap(shuffleId, mapId, stageAttemptId) } } diff --git a/core/src/test/scala/org/apache/spark/shuffle/ShuffleSuite.scala b/core/src/test/scala/org/apache/spark/shuffle/ShuffleSuite.scala index 77842fd55b1d7..a533dd62fd782 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/ShuffleSuite.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/ShuffleSuite.scala @@ -355,7 +355,6 @@ abstract class ShuffleSuite extends SparkFunSuite with Matchers with LocalSparkC data2, {iter: Iterator[(Int, Int)] => writer2.write(iter); writer2.stop(true)}) val (mapOutput1, mapOutput2) = interleaver.run() - // register the output from attempt 1, and try to read it mapOutput1.foreach { mapStatus => mapTrackerMaster.registerMapOutputs(0, Array(mapStatus))} val reader1 = manager.getReader[Int, Int](shuffleHandle, 0, 1, @@ -372,7 +371,6 @@ abstract class ShuffleSuite extends SparkFunSuite with Matchers with LocalSparkC InternalAccumulator.create(sc), false, taskMetrics = new TaskMetrics)) reader2.read().toIndexedSeq should be(data2.toIndexedSeq) - // make sure that when the shuffle gets unregistered, we cleanup from all attempts val shuffleFiles1 = manager.getShuffleFiles(shuffleHandle, 0, 0, 0) val shuffleFiles2 = manager.getShuffleFiles(shuffleHandle, 0, 0, 1) @@ -437,7 +435,6 @@ class InterleaveIterators[T, R]( } } - object ShuffleSuite { def mergeCombineException(x: Int, y: Int): Int = { From f37be9120499d699a2da34cb69620cd8dba9cb8a Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Thu, 8 Oct 2015 11:19:41 -0500 Subject: [PATCH 70/73] mima --- project/MimaExcludes.scala | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index 7495820ff3546..92c1eca72af06 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -80,6 +80,10 @@ object MimaExcludes { "org.apache.spark.ml.regression.LeastSquaresAggregator.add"), ProblemFilters.exclude[MissingMethodProblem]( "org.apache.spark.ml.regression.LeastSquaresCostFun.this") + ) ++ Seq( + // SPARK-8029. False positive, this is a @Private java class + ProblemFilters.exclude[MissingMethodProblem]( + "org.apache.spark.shuffle.unsafe.UnsafeShuffleWriter.this") ) case v if v.startsWith("1.5") => Seq( From fac0f1c6ed655a7b361881da7b1a140dbf029c7e Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Thu, 8 Oct 2015 16:26:37 -0500 Subject: [PATCH 71/73] comment formatting --- .../scala/org/apache/spark/shuffle/ShuffleManager.scala | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/shuffle/ShuffleManager.scala b/core/src/main/scala/org/apache/spark/shuffle/ShuffleManager.scala index 72e5b624646cf..9eac6f255f273 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/ShuffleManager.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/ShuffleManager.scala @@ -43,10 +43,11 @@ private[spark] trait ShuffleManager { numMaps: Int, dependency: ShuffleDependency[K, V, C]): ShuffleHandle - /** Get a writer for a given partition. Called on executors by map tasks. - * Implementations should call [[addShuffleAttempt]] to update internal state, so we can track - * all attempts for each shuffle. - * */ + /** + * Get a writer for a given partition. Called on executors by map tasks. + * Implementations should call [[addShuffleAttempt]] to update internal state, so we can track + * all attempts for each shuffle. + */ def getWriter[K, V]( handle: ShuffleHandle, mapId: Int, From a38d7603b1b2efb656eaa9fb4df7848d8f45048f Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Fri, 9 Oct 2015 09:25:34 -0500 Subject: [PATCH 72/73] get rid of DAGSchedulerFailureRecoverySuite --- .../DAGSchedulerFailureRecoverySuite.scala | 110 ------------------ 1 file changed, 110 deletions(-) delete mode 100644 core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerFailureRecoverySuite.scala diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerFailureRecoverySuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerFailureRecoverySuite.scala deleted file mode 100644 index 69a3f5652ec00..0000000000000 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerFailureRecoverySuite.scala +++ /dev/null @@ -1,110 +0,0 @@ -/* -* Licensed to the Apache Software Foundation (ASF) under one or more -* contributor license agreements. See the NOTICE file distributed with -* this work for additional information regarding copyright ownership. -* The ASF licenses this file to You under the Apache License, Version 2.0 -* (the "License"); you may not use this file except in compliance with -* the License. You may obtain a copy of the License at -* -* http://www.apache.org/licenses/LICENSE-2.0 -* -* Unless required by applicable law or agreed to in writing, software -* distributed under the License is distributed on an "AS IS" BASIS, -* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -* See the License for the specific language governing permissions and -* limitations under the License. -*/ -package org.apache.spark.scheduler - -import java.util.Date - -import scala.collection.mutable.{ArrayBuffer, HashMap} - -import org.apache.spark.shuffle.FetchFailedException -import org.apache.spark.storage.BlockManagerId -import org.apache.spark._ - -class DAGSchedulerFailureRecoverySuite extends SparkFunSuite with Logging { - - // TODO we should run this with a matrix of configurations: different shufflers, - // external shuffle service, etc. But that is really pushing the question of how to run - // such a long test ... - - ignore("no concurrent retries for stage attempts (SPARK-7308)") { - // see SPARK-7308 for a detailed description of the conditions this is trying to recreate. - // note that this is somewhat convoluted for a test case, but isn't actually very unusual - // under a real workload. We only fail the first attempt of stage 2, but that - // could be enough to cause havoc. - - (0 until 100).foreach { idx => - logInfo(new Date() + "\ttrial " + idx) - - val conf = new SparkConf().set("spark.executor.memory", "100m") - val clusterSc = new SparkContext("local-cluster[5,4,100]", "test-cluster", conf) - val bms = ArrayBuffer[BlockManagerId]() - val stageFailureCount = HashMap[Int, Int]() - clusterSc.addSparkListener(new SparkListener { - override def onBlockManagerAdded(bmAdded: SparkListenerBlockManagerAdded): Unit = { - bms += bmAdded.blockManagerId - } - - override def onStageCompleted(stageCompleted: SparkListenerStageCompleted): Unit = { - if (stageCompleted.stageInfo.failureReason.isDefined) { - val stage = stageCompleted.stageInfo.stageId - stageFailureCount(stage) = stageFailureCount.getOrElse(stage, 0) + 1 - val reason = stageCompleted.stageInfo.failureReason.get - logInfo("stage " + stage + " failed: " + stageFailureCount(stage)) - } - } - }) - try { - val rawData = clusterSc.parallelize(1 to 1e6.toInt, 20).map { x => (x % 100) -> x }.cache() - rawData.count() - - // choose any executor block manager for the fetch failures. Just can't be driver - // to avoid broadcast failures - val someBlockManager = bms.filter{!_.isDriver}(0) - - val shuffled = rawData.groupByKey(100).mapPartitionsWithIndex { case (idx, itr) => - // we want one failure quickly, and more failures after stage 0 has finished its - // second attempt - val stageAttemptId = TaskContext.get().asInstanceOf[TaskContextImpl].stageAttemptId - if (stageAttemptId == 0) { - if (idx == 0) { - throw new FetchFailedException(someBlockManager, 0, 0, idx, stageAttemptId, - cause = new RuntimeException("simulated fetch failure")) - } else if (idx > 0 && math.random < 0.2) { - Thread.sleep(5000) - throw new FetchFailedException(someBlockManager, 0, 0, idx, stageAttemptId, - cause = new RuntimeException("simulated fetch failure")) - } else { - // want to make sure plenty of these finish after task 0 fails, and some even finish - // after the previous stage is retried and this stage retry is started - Thread.sleep((500 + math.random * 5000).toLong) - } - } - itr.map { x => ((x._1 + 5) % 100) -> x._2 } - } - val shuffledAgain = shuffled.flatMap { case (k, vs) => vs.map(k -> _) }.groupByKey(100) - val data = shuffledAgain.mapPartitions { itr => itr.flatMap(_._2) }.cache().collect() - val count = data.size - assert(count === 1e6.toInt) - assert(data.toSet === (1 to 1e6.toInt).toSet) - - // we should only get one failure from stage 2, everything else should be fine - // However, one submissions can result in multiple failures (SPARK-8103), so for - // now this has to have very weak checks. Once there is a fix for SPARK-8103, these - // checks should be made tighter - - assert(stageFailureCount.getOrElse(1, 0) === 0) - assert(stageFailureCount.getOrElse(2, 0) <= 10) // should be 1 failure - assert(stageFailureCount.getOrElse(3, 0) <= 10) // should be 0 failures - } finally { - clusterSc.stop() - } - } - } - - - -} From 37ac799f53bc4d09319c51a45e1079c9f9cee166 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Fri, 9 Oct 2015 09:26:08 -0500 Subject: [PATCH 73/73] ExternalShuffleBlockResolver can handle blockIds w/out stageAttemptId --- .../shuffle/ExternalShuffleBlockResolver.java | 17 ++++-- .../ExternalShuffleBlockResolverSuite.java | 59 ++++++++++++++++++- .../shuffle/TestShuffleDataContext.java | 32 +++++++++- 3 files changed, 99 insertions(+), 9 deletions(-) diff --git a/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java b/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java index fa501c4a03e72..1dd2e8c38ffcd 100644 --- a/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java +++ b/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java @@ -168,7 +168,7 @@ public void registerExecutor( */ public ManagedBuffer getBlockData(String appId, String execId, String blockId) { String[] blockIdParts = blockId.split("_"); - if (blockIdParts.length != 5) { + if (blockIdParts.length < 4 || blockIdParts.length > 5) { throw new IllegalArgumentException("Unexpected block id format: " + blockId); } else if (!blockIdParts[0].equals("shuffle")) { throw new IllegalArgumentException("Expected shuffle block id, got: " + blockId); @@ -176,7 +176,6 @@ public ManagedBuffer getBlockData(String appId, String execId, String blockId) { int shuffleId = Integer.parseInt(blockIdParts[1]); int mapId = Integer.parseInt(blockIdParts[2]); int reduceId = Integer.parseInt(blockIdParts[3]); - int stageAttemptId = Integer.parseInt(blockIdParts[4]); ExecutorShuffleInfo executor = executors.get(new AppExecId(appId, execId)); if (executor == null) { @@ -188,7 +187,14 @@ public ManagedBuffer getBlockData(String appId, String execId, String blockId) { return getHashBasedShuffleBlockData(executor, blockId); } else if ("org.apache.spark.shuffle.sort.SortShuffleManager".equals(executor.shuffleManager) || "org.apache.spark.shuffle.unsafe.UnsafeShuffleManager".equals(executor.shuffleManager)) { - return getSortBasedShuffleBlockData(executor, shuffleId, mapId, reduceId, stageAttemptId); + // for backwards compatibility, we also handle legacy shuffle block ids which don't have + // a stageAttemptId + String baseFileName = "shuffle_" + shuffleId + "_" + mapId + "_0"; + if (blockIdParts.length == 5) { + int stageAttemptId = Integer.parseInt(blockIdParts[4]); + baseFileName = baseFileName + "_" + stageAttemptId; + } + return getSortBasedShuffleBlockData(executor, baseFileName, reduceId); } else { throw new UnsupportedOperationException( "Unsupported shuffle manager: " + executor.shuffleManager); @@ -267,8 +273,9 @@ private ManagedBuffer getHashBasedShuffleBlockData(ExecutorShuffleInfo executor, * and the block id format is from ShuffleDataBlockId and ShuffleIndexBlockId. */ private ManagedBuffer getSortBasedShuffleBlockData( - ExecutorShuffleInfo executor, int shuffleId, int mapId, int reduceId, int stageAttemptId) { - String baseFileName = "shuffle_" + shuffleId + "_" + mapId + "_0_" + stageAttemptId; + ExecutorShuffleInfo executor, + String baseFileName, + int reduceId) { File indexFile = getFile(executor.localDirs, executor.subDirsPerLocalDir, baseFileName + ".index"); diff --git a/network/shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolverSuite.java b/network/shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolverSuite.java index 48738f43ab692..c137355a81f5b 100644 --- a/network/shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolverSuite.java +++ b/network/shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolverSuite.java @@ -102,9 +102,16 @@ public void testBadRequests() throws IOException { // pass } - // no stageAttemptId + // wrong number of parts (note that we allow a missing stageAttemptId) try { - resolver.getBlockData("app0", "exec1", "shuffle_1_1_0"); + resolver.getBlockData("app0", "exec1", "shuffle_1_1_0_0_0"); + fail("Should have failed"); + } catch (RuntimeException e) { + assertTrue("Bad error message: " + e, e.getMessage().contains("Unexpected block id format")); + } + + try { + resolver.getBlockData("app0", "exec1", "shuffle_1_1"); fail("Should have failed"); } catch (RuntimeException e) { assertTrue("Bad error message: " + e, e.getMessage().contains("Unexpected block id format")); @@ -145,6 +152,54 @@ private void testReadBlockData(ExternalShuffleBlockResolver resolver, String blo assertEquals(expected, block0); } + @Test + public void supportLegacySortShuffleBlockIds() throws IOException { + // In Spark 1.6, the stage attempt ID was added to shuffle block ids (SPARK-8029). However, + // during a rolling upgrade, the shuffle service may be restarted with new code but still + // need to serve old apps. So we make sure we can still handle old blocks + + ExternalShuffleBlockResolver resolver = new ExternalShuffleBlockResolver(conf, null); + resolver.registerExecutor("app0", "exec0", + dataContext.createExecutorInfo("org.apache.spark.shuffle.sort.SortShuffleManager")); + + dataContext.insertLegacySortShuffleData(2, 1, + new byte[][]{"legacy".getBytes(), "block".getBytes()}); + + testReadBlockData(resolver, "shuffle_2_1_0", "legacy"); + testReadBlockData(resolver, "shuffle_2_1_1", "block"); + + // verify everything still works when we also register some blocks which do have a + // stageAttemptId + testSortShuffleBlocks(); + + testReadBlockData(resolver, "shuffle_2_1_0", "legacy"); + testReadBlockData(resolver, "shuffle_2_1_1", "block"); + } + + @Test + public void supportLegacyHashShuffleBlockIds() throws IOException { + // In Spark 1.6, the stage attempt ID was added to shuffle block ids (SPARK-8029). However, + // during a rolling upgrade, the shuffle service may be restarted with new code but still + // need to serve old apps. So we make sure we can still handle old blocks + + ExternalShuffleBlockResolver resolver = new ExternalShuffleBlockResolver(conf, null); + resolver.registerExecutor("app0", "exec0", + dataContext.createExecutorInfo("org.apache.spark.shuffle.hash.HashShuffleManager")); + + dataContext.insertLegacyHashShuffleData(2, 0, + new byte[][] { "more legacy".getBytes(), "hash".getBytes() } ); + + testReadBlockData(resolver, "shuffle_2_0_0", "more legacy"); + testReadBlockData(resolver, "shuffle_2_0_1", "hash"); + + // verify everything still works when we also register some blocks which do have a + // stageAttemptId + testHashShuffleBlocks(); + + testReadBlockData(resolver, "shuffle_2_0_0", "more legacy"); + testReadBlockData(resolver, "shuffle_2_0_1", "hash"); + } + @Test public void jsonSerializationOfExecutorRegistration() throws IOException { ObjectMapper mapper = new ObjectMapper(); diff --git a/network/shuffle/src/test/java/org/apache/spark/network/shuffle/TestShuffleDataContext.java b/network/shuffle/src/test/java/org/apache/spark/network/shuffle/TestShuffleDataContext.java index e98765c1d4876..c4d2a85aba58f 100644 --- a/network/shuffle/src/test/java/org/apache/spark/network/shuffle/TestShuffleDataContext.java +++ b/network/shuffle/src/test/java/org/apache/spark/network/shuffle/TestShuffleDataContext.java @@ -57,10 +57,24 @@ public void cleanup() { } /** Creates reducer blocks in a sort-based data format within our local dirs. */ - public void insertSortShuffleData(int shuffleId, int mapId, int stageAttemptId, + public void insertSortShuffleData( + int shuffleId, + int mapId, + int stageAttemptId, byte[][] blocks) throws IOException { String blockId = "shuffle_" + shuffleId + "_" + mapId + "_0_" + stageAttemptId; + insertSortShuffleData(blockId, blocks); + } + + public void insertLegacySortShuffleData( + int shuffleId, + int mapId, + byte[][] blocks) throws IOException { + String blockId = "shuffle_" + shuffleId + "_" + mapId + "_0"; + insertSortShuffleData(blockId, blocks); + } + private void insertSortShuffleData(String blockId, byte[][] blocks) throws IOException { OutputStream dataStream = new FileOutputStream( ExternalShuffleBlockResolver.getFile(localDirs, subDirsPerLocalDir, blockId + ".data")); DataOutputStream indexStream = new DataOutputStream(new FileOutputStream( @@ -78,8 +92,22 @@ public void insertSortShuffleData(int shuffleId, int mapId, int stageAttemptId, indexStream.close(); } + public void insertLegacyHashShuffleData( + int shuffleId, + int mapId, + byte[][] blocks) throws IOException { + for (int i = 0; i < blocks.length; i ++) { + String blockId = "shuffle_" + shuffleId + "_" + mapId + "_" + i; + Files.write(blocks[i], + ExternalShuffleBlockResolver.getFile(localDirs, subDirsPerLocalDir, blockId)); + } + } + /** Creates reducer blocks in a hash-based data format within our local dirs. */ - public void insertHashShuffleData(int shuffleId, int mapId, int stageAttemptId, + public void insertHashShuffleData( + int shuffleId, + int mapId, + int stageAttemptId, byte[][] blocks) throws IOException { for (int i = 0; i < blocks.length; i ++) { String blockId = "shuffle_" + shuffleId + "_" + mapId + "_" + i + "_" + stageAttemptId;