From 36d489842b92e0d75873ed4b116552669c2ea7b4 Mon Sep 17 00:00:00 2001 From: Yifei Huang Date: Thu, 9 May 2019 11:14:59 -0700 Subject: [PATCH 01/27] semantic fix that hasn't merged yet (to revert) --- .../spark/shuffle/ShuffleDriverComponentsSuite.scala | 12 ++++++++++-- 1 file changed, 10 insertions(+), 2 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/shuffle/ShuffleDriverComponentsSuite.scala b/core/src/test/scala/org/apache/spark/shuffle/ShuffleDriverComponentsSuite.scala index dbb954945a8b6..e8372c0458600 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/ShuffleDriverComponentsSuite.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/ShuffleDriverComponentsSuite.scala @@ -22,8 +22,9 @@ import java.util import com.google.common.collect.ImmutableMap import org.apache.spark.{LocalSparkContext, SparkConf, SparkContext, SparkEnv, SparkFunSuite} -import org.apache.spark.api.shuffle.{ShuffleDataIO, ShuffleDriverComponents, ShuffleExecutorComponents, ShuffleWriteSupport} +import org.apache.spark.api.shuffle.{ShuffleDataIO, ShuffleDriverComponents, ShuffleExecutorComponents, ShuffleReadSupport, ShuffleWriteSupport} import org.apache.spark.internal.config.SHUFFLE_IO_PLUGIN_CLASS +import org.apache.spark.shuffle.io.DefaultShuffleReadSupport import org.apache.spark.shuffle.sort.io.DefaultShuffleWriteSupport class ShuffleDriverComponentsSuite extends SparkFunSuite with LocalSparkContext { @@ -66,6 +67,13 @@ class TestShuffleExecutorComponents(sparkConf: SparkConf) extends ShuffleExecuto override def writes(): ShuffleWriteSupport = { val blockManager = SparkEnv.get.blockManager val blockResolver = new IndexShuffleBlockResolver(sparkConf, blockManager) - new DefaultShuffleWriteSupport(sparkConf, blockResolver) + new DefaultShuffleWriteSupport(sparkConf, blockResolver, blockManager.shuffleServerId) + } + + override def reads(): ShuffleReadSupport = { + val blockManager = SparkEnv.get.blockManager + val mapOutputTracker = SparkEnv.get.mapOutputTracker + val serializerManager = SparkEnv.get.serializerManager + new DefaultShuffleReadSupport(blockManager, mapOutputTracker, serializerManager, sparkConf) } } From 48f52c85e5722c85e5260c75c07a79daf4eade85 Mon Sep 17 00:00:00 2001 From: Yifei Huang Date: Thu, 9 May 2019 11:19:05 -0700 Subject: [PATCH 02/27] specify ShuffleLocation interface --- .../spark/api/shuffle/MapShuffleLocations.java | 2 +- .../spark/api/shuffle/ShuffleLocation.java | 5 ++++- .../shuffle/sort/DefaultMapShuffleLocations.java | 16 ++++++++++++++-- .../org/apache/spark/MapOutputTracker.scala | 11 +++++++---- 4 files changed, 26 insertions(+), 8 deletions(-) diff --git a/core/src/main/java/org/apache/spark/api/shuffle/MapShuffleLocations.java b/core/src/main/java/org/apache/spark/api/shuffle/MapShuffleLocations.java index b0aed4d08d387..d4ba69c0089b2 100644 --- a/core/src/main/java/org/apache/spark/api/shuffle/MapShuffleLocations.java +++ b/core/src/main/java/org/apache/spark/api/shuffle/MapShuffleLocations.java @@ -35,5 +35,5 @@ public interface MapShuffleLocations extends Serializable { /** * Get the location for a given shuffle block written by this map task. */ - ShuffleLocation getLocationForBlock(int reduceId); + ShuffleLocation[] getLocationsForBlock(int reduceId); } diff --git a/core/src/main/java/org/apache/spark/api/shuffle/ShuffleLocation.java b/core/src/main/java/org/apache/spark/api/shuffle/ShuffleLocation.java index d06c11b3c01ee..7a983709a1874 100644 --- a/core/src/main/java/org/apache/spark/api/shuffle/ShuffleLocation.java +++ b/core/src/main/java/org/apache/spark/api/shuffle/ShuffleLocation.java @@ -21,4 +21,7 @@ * Marker interface representing a location of a shuffle block. Implementations of shuffle readers * and writers are expected to cast this down to an implementation-specific representation. */ -public interface ShuffleLocation {} +public interface ShuffleLocation { + String host(); + int port(); +} diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/DefaultMapShuffleLocations.java b/core/src/main/java/org/apache/spark/shuffle/sort/DefaultMapShuffleLocations.java index ffd97c0f26605..c5ee66f14eb03 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/DefaultMapShuffleLocations.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/DefaultMapShuffleLocations.java @@ -45,9 +45,11 @@ public DefaultMapShuffleLocations load(BlockManagerId blockManagerId) { }); private final BlockManagerId location; + private final ShuffleLocation[] locationsArray; public DefaultMapShuffleLocations(BlockManagerId blockManagerId) { this.location = blockManagerId; + this.locationsArray = new ShuffleLocation[] {this}; } public static DefaultMapShuffleLocations get(BlockManagerId blockManagerId) { @@ -55,8 +57,8 @@ public static DefaultMapShuffleLocations get(BlockManagerId blockManagerId) { } @Override - public ShuffleLocation getLocationForBlock(int reduceId) { - return this; + public ShuffleLocation[] getLocationsForBlock(int reduceId) { + return locationsArray; } public BlockManagerId getBlockManagerId() { @@ -73,4 +75,14 @@ public boolean equals(Object other) { public int hashCode() { return Objects.hashCode(location); } + + @Override + public String host() { + return location.host(); + } + + @Override + public int port() { + return location.port(); + } } diff --git a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala index ebddf5ff6f6e0..a8c329af586da 100644 --- a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala +++ b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala @@ -885,13 +885,16 @@ private[spark] object MapOutputTracker extends Logging { for (part <- startPartition until endPartition) { val size = status.getSizeForBlock(part) if (size != 0) { - if (status.mapShuffleLocations == null) { + if (status.mapShuffleLocations == null + || status.mapShuffleLocations.getLocationsForBlock(part).isEmpty) { splitsByAddress.getOrElseUpdate(Option.empty, ListBuffer()) += ((ShuffleBlockId(shuffleId, mapId, part), size)) } else { - val shuffleLoc = status.mapShuffleLocations.getLocationForBlock(part) - splitsByAddress.getOrElseUpdate(Option.apply(shuffleLoc), ListBuffer()) += - ((ShuffleBlockId(shuffleId, mapId, part), size)) + val shuffleLocations = status.mapShuffleLocations.getLocationsForBlock(part) + shuffleLocations.foreach { location => + splitsByAddress.getOrElseUpdate(Option.apply(location), ListBuffer()) += + ((ShuffleBlockId(shuffleId, mapId, part), size)) + } } } } From d161feb0b02d80205dafda140a0e4af2100c1e92 Mon Sep 17 00:00:00 2001 From: Yifei Huang Date: Thu, 9 May 2019 12:23:55 -0700 Subject: [PATCH 03/27] more api changes --- .../api/shuffle/MapShuffleLocations.java | 6 +++ .../spark/api/shuffle/ShuffleBlockInfo.java | 6 +-- .../spark/api/shuffle/ShuffleLocation.java | 11 ++++-- .../sort/DefaultMapShuffleLocations.java | 10 ++++- .../org/apache/spark/MapOutputTracker.scala | 20 +++++----- .../shuffle/BlockStoreShuffleReader.scala | 2 +- .../io/DefaultShuffleReadSupport.scala | 4 +- .../storage/ShuffleBlockFetcherIterator.scala | 1 + .../apache/spark/MapOutputTrackerSuite.scala | 2 +- .../spark/scheduler/DAGSchedulerSuite.scala | 37 +++++++++---------- .../BlockStoreShuffleReaderSuite.scala | 7 ++-- 11 files changed, 62 insertions(+), 44 deletions(-) diff --git a/core/src/main/java/org/apache/spark/api/shuffle/MapShuffleLocations.java b/core/src/main/java/org/apache/spark/api/shuffle/MapShuffleLocations.java index d4ba69c0089b2..33caa74292058 100644 --- a/core/src/main/java/org/apache/spark/api/shuffle/MapShuffleLocations.java +++ b/core/src/main/java/org/apache/spark/api/shuffle/MapShuffleLocations.java @@ -36,4 +36,10 @@ public interface MapShuffleLocations extends Serializable { * Get the location for a given shuffle block written by this map task. */ ShuffleLocation[] getLocationsForBlock(int reduceId); + + /** + * Deletes a ShuffleLocation from this MapShuffleLocations. Returns true if + * the removal of this ShuffleLocation results in missing partitions. + */ + boolean removeShuffleLocation(ShuffleLocation location); } diff --git a/core/src/main/java/org/apache/spark/api/shuffle/ShuffleBlockInfo.java b/core/src/main/java/org/apache/spark/api/shuffle/ShuffleBlockInfo.java index a312831cb6282..46cb3343374df 100644 --- a/core/src/main/java/org/apache/spark/api/shuffle/ShuffleBlockInfo.java +++ b/core/src/main/java/org/apache/spark/api/shuffle/ShuffleBlockInfo.java @@ -31,10 +31,10 @@ public class ShuffleBlockInfo { private final int mapId; private final int reduceId; private final long length; - private final Optional shuffleLocation; + private final ShuffleLocation[] shuffleLocation; public ShuffleBlockInfo(int shuffleId, int mapId, int reduceId, long length, - Optional shuffleLocation) { + ShuffleLocation[] shuffleLocation) { this.shuffleId = shuffleId; this.mapId = mapId; this.reduceId = reduceId; @@ -58,7 +58,7 @@ public long getLength() { return length; } - public Optional getShuffleLocation() { + public ShuffleLocation[] getShuffleLocation() { return shuffleLocation; } diff --git a/core/src/main/java/org/apache/spark/api/shuffle/ShuffleLocation.java b/core/src/main/java/org/apache/spark/api/shuffle/ShuffleLocation.java index 7a983709a1874..daa1d34c6ac34 100644 --- a/core/src/main/java/org/apache/spark/api/shuffle/ShuffleLocation.java +++ b/core/src/main/java/org/apache/spark/api/shuffle/ShuffleLocation.java @@ -21,7 +21,12 @@ * Marker interface representing a location of a shuffle block. Implementations of shuffle readers * and writers are expected to cast this down to an implementation-specific representation. */ -public interface ShuffleLocation { - String host(); - int port(); +public abstract class ShuffleLocation { + public abstract String host(); + public abstract int port(); + + @Override + public final String toString() { + return String.format("ShuffleLocation %s:%d", host(), port()); + } } diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/DefaultMapShuffleLocations.java b/core/src/main/java/org/apache/spark/shuffle/sort/DefaultMapShuffleLocations.java index c5ee66f14eb03..0455b74969ea3 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/DefaultMapShuffleLocations.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/DefaultMapShuffleLocations.java @@ -27,7 +27,7 @@ import java.util.Objects; -public class DefaultMapShuffleLocations implements MapShuffleLocations, ShuffleLocation { +public class DefaultMapShuffleLocations extends ShuffleLocation implements MapShuffleLocations { /** * We borrow the cache size from the BlockManagerId's cache - around 1MB, which should be @@ -61,6 +61,14 @@ public ShuffleLocation[] getLocationsForBlock(int reduceId) { return locationsArray; } + @Override + public boolean removeShuffleLocation(ShuffleLocation location) { + if (location.host().equals(this.host()) && location.port() == this.port()) { + return true; + } + return false; + } + public BlockManagerId getBlockManagerId() { return location; } diff --git a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala index a8c329af586da..8111996325507 100644 --- a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala +++ b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala @@ -283,7 +283,7 @@ private[spark] abstract class MapOutputTracker(conf: SparkConf) extends Logging // For testing def getMapSizesByShuffleLocation(shuffleId: Int, reduceId: Int) - : Iterator[(Option[ShuffleLocation], Seq[(BlockId, Long)])] = { + : Iterator[(Array[ShuffleLocation], Seq[(BlockId, Long)])] = { getMapSizesByShuffleLocation(shuffleId, reduceId, reduceId + 1) } @@ -297,7 +297,7 @@ private[spark] abstract class MapOutputTracker(conf: SparkConf) extends Logging * describing the shuffle blocks that are stored at that block manager. */ def getMapSizesByShuffleLocation(shuffleId: Int, startPartition: Int, endPartition: Int) - : Iterator[(Option[ShuffleLocation], Seq[(BlockId, Long)])] + : Iterator[(Array[ShuffleLocation], Seq[(BlockId, Long)])] /** * Deletes map output status information for the specified shuffle stage. @@ -647,7 +647,7 @@ private[spark] class MapOutputTrackerMaster( // Get blocks sizes by executor Id. Note that zero-sized blocks are excluded in the result. // This method is only called in local-mode. def getMapSizesByShuffleLocation(shuffleId: Int, startPartition: Int, endPartition: Int) - : Iterator[(Option[ShuffleLocation], Seq[(BlockId, Long)])] = { + : Iterator[(Array[ShuffleLocation], Seq[(BlockId, Long)])] = { logDebug(s"Fetching outputs for shuffle $shuffleId, partitions $startPartition-$endPartition") shuffleStatuses.get(shuffleId) match { case Some (shuffleStatus) => @@ -684,7 +684,7 @@ private[spark] class MapOutputTrackerWorker(conf: SparkConf) extends MapOutputTr // Get blocks sizes by executor Id. Note that zero-sized blocks are excluded in the result. override def getMapSizesByShuffleLocation(shuffleId: Int, startPartition: Int, endPartition: Int) - : Iterator[(Option[ShuffleLocation], Seq[(BlockId, Long)])] = { + : Iterator[(Array[ShuffleLocation], Seq[(BlockId, Long)])] = { logDebug(s"Fetching outputs for shuffle $shuffleId, partitions $startPartition-$endPartition") val statuses = getStatuses(shuffleId) try { @@ -873,9 +873,9 @@ private[spark] object MapOutputTracker extends Logging { shuffleId: Int, startPartition: Int, endPartition: Int, - statuses: Array[MapStatus]): Iterator[(Option[ShuffleLocation], Seq[(BlockId, Long)])] = { + statuses: Array[MapStatus]): Iterator[(Array[ShuffleLocation], Seq[(BlockId, Long)])] = { assert (statuses != null) - val splitsByAddress = new HashMap[Option[ShuffleLocation], ListBuffer[(BlockId, Long)]] + val splitsByAddress = new HashMap[Array[ShuffleLocation], ListBuffer[(BlockId, Long)]] for ((status, mapId) <- statuses.iterator.zipWithIndex) { if (status == null) { val errorMessage = s"Missing an output location for shuffle $shuffleId" @@ -887,14 +887,12 @@ private[spark] object MapOutputTracker extends Logging { if (size != 0) { if (status.mapShuffleLocations == null || status.mapShuffleLocations.getLocationsForBlock(part).isEmpty) { - splitsByAddress.getOrElseUpdate(Option.empty, ListBuffer()) += + splitsByAddress.getOrElseUpdate(Array.empty, ListBuffer()) += ((ShuffleBlockId(shuffleId, mapId, part), size)) } else { val shuffleLocations = status.mapShuffleLocations.getLocationsForBlock(part) - shuffleLocations.foreach { location => - splitsByAddress.getOrElseUpdate(Option.apply(location), ListBuffer()) += - ((ShuffleBlockId(shuffleId, mapId, part), size)) - } + splitsByAddress.getOrElseUpdate(shuffleLocations, ListBuffer()) += + ((ShuffleBlockId(shuffleId, mapId, part), size)) } } } diff --git a/core/src/main/scala/org/apache/spark/shuffle/BlockStoreShuffleReader.scala b/core/src/main/scala/org/apache/spark/shuffle/BlockStoreShuffleReader.scala index 530c3694ad1ec..977535c140d14 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/BlockStoreShuffleReader.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/BlockStoreShuffleReader.scala @@ -69,7 +69,7 @@ private[spark] class BlockStoreShuffleReader[K, C]( block.mapId, block.reduceId, blockInfo._2, - Optional.ofNullable(shuffleLocationInfo._1.orNull)) + shuffleLocationInfo._1) } } } diff --git a/core/src/main/scala/org/apache/spark/shuffle/io/DefaultShuffleReadSupport.scala b/core/src/main/scala/org/apache/spark/shuffle/io/DefaultShuffleReadSupport.scala index 9b9b8508e88aa..be0263287fe30 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/io/DefaultShuffleReadSupport.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/io/DefaultShuffleReadSupport.scala @@ -95,8 +95,8 @@ private class ShuffleBlockFetcherIterable( blockManager, mapOutputTracker.getMapSizesByShuffleLocation(shuffleId, minReduceId, maxReduceId + 1) .map { shuffleLocationInfo => - val defaultShuffleLocation = shuffleLocationInfo._1 - .get.asInstanceOf[DefaultMapShuffleLocations] + val defaultShuffleLocation = shuffleLocationInfo._1(0) + .asInstanceOf[DefaultMapShuffleLocations] (defaultShuffleLocation.getBlockManagerId, shuffleLocationInfo._2) }, serializerManager.wrapStream, diff --git a/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala index 287ffdd6e10e6..bfed8dda6bf5b 100644 --- a/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala +++ b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala @@ -31,6 +31,7 @@ import org.apache.spark.network.buffer.{FileSegmentManagedBuffer, ManagedBuffer} import org.apache.spark.network.shuffle._ import org.apache.spark.network.util.TransportConf import org.apache.spark.shuffle.{FetchFailedException, ShuffleReadMetricsReporter} +import org.apache.spark.shuffle.sort.DefaultMapShuffleLocations import org.apache.spark.util.{CompletionIterator, TaskCompletionListener, Utils} import org.apache.spark.util.io.ChunkedByteBufferOutputStream diff --git a/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala b/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala index 8fcbc845d1a7b..0bd10475316b4 100644 --- a/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala +++ b/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala @@ -325,7 +325,7 @@ class MapOutputTrackerSuite extends SparkFunSuite { Array(size10000, size0, size1000, size0))) assert(tracker.containsShuffle(10)) assert(tracker.getMapSizesByShuffleLocation(10, 0, 4) - .map(x => (x._1.get, x._2)).toSeq === + .map(x => (x._1(0), x._2)).toSeq === Seq( (DefaultMapShuffleLocations.get(BlockManagerId("b", "hostB", 1000)), Seq((ShuffleBlockId(10, 1, 0), size10000), (ShuffleBlockId(10, 1, 2), size1000))), 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 21b4e56c9e801..b04551c0f731b 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -24,12 +24,11 @@ import scala.annotation.meta.param import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet, Map} import scala.language.reflectiveCalls import scala.util.control.NonFatal - import org.scalatest.concurrent.{Signaler, ThreadSignaler, TimeLimits} import org.scalatest.time.SpanSugar._ import org.apache.spark._ -import org.apache.spark.api.shuffle.MapShuffleLocations +import org.apache.spark.api.shuffle.{MapShuffleLocations, ShuffleLocation} import org.apache.spark.broadcast.BroadcastManager import org.apache.spark.executor.ExecutorMetrics import org.apache.spark.internal.config @@ -703,7 +702,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi (Success, makeMapStatus("hostA", 1)), (Success, makeMapStatus("hostB", 1)))) assert(mapOutputTracker.getMapSizesByShuffleLocation(shuffleId, 0).map(_._1).toSet === - HashSet(makeMaybeShuffleLocation("hostA"), makeMaybeShuffleLocation("hostB"))) + HashSet(makeShuffleLocationArray("hostA"), makeShuffleLocationArray("hostB"))) complete(taskSets(1), Seq((Success, 42))) assert(results === Map(0 -> 42)) assertDataStructuresEmpty() @@ -731,7 +730,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi // we can see both result blocks now assert(mapOutputTracker .getMapSizesByShuffleLocation(shuffleId, 0) - .map(_._1.get.asInstanceOf[DefaultMapShuffleLocations].getBlockManagerId.host) + .map(_._1(0).asInstanceOf[DefaultMapShuffleLocations].getBlockManagerId.host) .toSet === HashSet("hostA", "hostB")) complete(taskSets(3), Seq((Success, 43))) assert(results === Map(0 -> 42, 1 -> 43)) @@ -774,7 +773,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi } } else { assert(mapOutputTracker.getMapSizesByShuffleLocation(shuffleId, 0).map(_._1).toSet === - HashSet(makeMaybeShuffleLocation("hostA"), makeMaybeShuffleLocation("hostB"))) + HashSet(makeShuffleLocationArray("hostA"), makeShuffleLocationArray("hostB"))) } } } @@ -1069,7 +1068,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi // The MapOutputTracker should know about both map output locations. assert(mapOutputTracker .getMapSizesByShuffleLocation(shuffleId, 0) - .map(_._1.get.asInstanceOf[DefaultMapShuffleLocations].getBlockManagerId.host) + .map(_._1(0).asInstanceOf[DefaultMapShuffleLocations].getBlockManagerId.host) .toSet === HashSet("hostA", "hostB")) // The first result task fails, with a fetch failure for the output from the first mapper. @@ -1201,11 +1200,11 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi // The MapOutputTracker should know about both map output locations. assert(mapOutputTracker .getMapSizesByShuffleLocation(shuffleId, 0) - .map(_._1.get.asInstanceOf[DefaultMapShuffleLocations].getBlockManagerId.host) + .map(_._1(0).asInstanceOf[DefaultMapShuffleLocations].getBlockManagerId.host) .toSet === HashSet("hostA", "hostB")) assert(mapOutputTracker .getMapSizesByShuffleLocation(shuffleId, 1) - .map(_._1.get.asInstanceOf[DefaultMapShuffleLocations].getBlockManagerId.host) + .map(_._1(0).asInstanceOf[DefaultMapShuffleLocations].getBlockManagerId.host) .toSet === HashSet("hostA", "hostB")) // The first result task fails, with a fetch failure for the output from the first mapper. @@ -1397,7 +1396,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi makeMapStatus("hostA", reduceRdd.partitions.size))) assert(shuffleStage.numAvailableOutputs === 2) assert(mapOutputTracker.getMapSizesByShuffleLocation(shuffleId, 0).map(_._1).toSet === - HashSet(makeMaybeShuffleLocation("hostB"), makeMaybeShuffleLocation("hostA"))) + HashSet(makeShuffleLocationArray("hostB"), makeShuffleLocationArray("hostA"))) // finish the next stage normally, which completes the job complete(taskSets(1), Seq((Success, 42), (Success, 43))) @@ -1803,7 +1802,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi // have hostC complete the resubmitted task complete(taskSets(1), Seq((Success, makeMapStatus("hostC", 1)))) assert(mapOutputTracker.getMapSizesByShuffleLocation(shuffleId, 0).map(_._1).toSet === - HashSet(makeMaybeShuffleLocation("hostC"), makeMaybeShuffleLocation("hostB"))) + HashSet(makeShuffleLocationArray("hostC"), makeShuffleLocationArray("hostB"))) // Make sure that the reduce stage was now submitted. assert(taskSets.size === 3) @@ -2066,7 +2065,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi complete(taskSets(0), Seq( (Success, makeMapStatus("hostA", 1)))) assert(mapOutputTracker.getMapSizesByShuffleLocation(shuffleId, 0).map(_._1).toSet === - HashSet(makeMaybeShuffleLocation("hostA"))) + HashSet(makeShuffleLocationArray("hostA"))) // Reducer should run on the same host that map task ran val reduceTaskSet = taskSets(1) @@ -2112,7 +2111,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi complete(taskSets(0), Seq( (Success, makeMapStatus("hostA", 1)))) assert(mapOutputTracker.getMapSizesByShuffleLocation(shuffleId, 0).map(_._1).toSet === - HashSet(makeMaybeShuffleLocation("hostA"))) + HashSet(makeShuffleLocationArray("hostA"))) // Reducer should run where RDD 2 has preferences, even though it also has a shuffle dep val reduceTaskSet = taskSets(1) @@ -2276,7 +2275,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi (Success, makeMapStatus("hostA", rdd1.partitions.length)), (Success, makeMapStatus("hostB", rdd1.partitions.length)))) assert(mapOutputTracker.getMapSizesByShuffleLocation(dep1.shuffleId, 0).map(_._1).toSet === - HashSet(makeMaybeShuffleLocation("hostA"), makeMaybeShuffleLocation("hostB"))) + HashSet(makeShuffleLocationArray("hostA"), makeShuffleLocationArray("hostB"))) assert(listener1.results.size === 1) // When attempting the second stage, show a fetch failure @@ -2292,7 +2291,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi complete(taskSets(2), Seq( (Success, makeMapStatus("hostC", rdd2.partitions.length)))) assert(mapOutputTracker.getMapSizesByShuffleLocation(dep1.shuffleId, 0).map(_._1).toSet === - HashSet(makeMaybeShuffleLocation("hostC"), makeMaybeShuffleLocation("hostB"))) + HashSet(makeShuffleLocationArray("hostC"), makeShuffleLocationArray("hostB"))) assert(listener2.results.size === 0) // Second stage listener should still not have a result @@ -2302,7 +2301,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi (Success, makeMapStatus("hostB", rdd2.partitions.length)), (Success, makeMapStatus("hostD", rdd2.partitions.length)))) assert(mapOutputTracker.getMapSizesByShuffleLocation(dep2.shuffleId, 0).map(_._1).toSet === - HashSet(makeMaybeShuffleLocation("hostB"), makeMaybeShuffleLocation("hostD"))) + HashSet(makeShuffleLocationArray("hostB"), makeShuffleLocationArray("hostD"))) assert(listener2.results.size === 1) // Finally, the reduce job should be running as task set 4; make it see a fetch failure, @@ -2341,7 +2340,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi (Success, makeMapStatus("hostA", rdd1.partitions.length)), (Success, makeMapStatus("hostB", rdd1.partitions.length)))) assert(mapOutputTracker.getMapSizesByShuffleLocation(dep1.shuffleId, 0).map(_._1).toSet === - HashSet(makeMaybeShuffleLocation("hostA"), makeMaybeShuffleLocation("hostB"))) + HashSet(makeShuffleLocationArray("hostA"), makeShuffleLocationArray("hostB"))) assert(listener1.results.size === 1) // When attempting stage1, trigger a fetch failure. @@ -2367,7 +2366,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi complete(taskSets(2), Seq( (Success, makeMapStatus("hostC", rdd2.partitions.length)))) assert(mapOutputTracker.getMapSizesByShuffleLocation(dep1.shuffleId, 0).map(_._1).toSet === - Set(makeMaybeShuffleLocation("hostC"), makeMaybeShuffleLocation("hostB"))) + Set(makeShuffleLocationArray("hostC"), makeShuffleLocationArray("hostB"))) // After stage0 is finished, stage1 will be submitted and found there is no missing // partitions in it. Then listener got triggered. @@ -2924,8 +2923,8 @@ object DAGSchedulerSuite { DefaultMapShuffleLocations.get(makeBlockManagerId(host)) } - def makeMaybeShuffleLocation(host: String): Option[MapShuffleLocations] = { - Some(DefaultMapShuffleLocations.get(makeBlockManagerId(host))) + def makeShuffleLocationArray(host: String): Array[ShuffleLocation] = { + DefaultMapShuffleLocations.get(makeBlockManagerId(host)).getLocationsForBlock(0) } } diff --git a/core/src/test/scala/org/apache/spark/shuffle/BlockStoreShuffleReaderSuite.scala b/core/src/test/scala/org/apache/spark/shuffle/BlockStoreShuffleReaderSuite.scala index 6468914bf3185..98e879a8db8aa 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/BlockStoreShuffleReaderSuite.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/BlockStoreShuffleReaderSuite.scala @@ -116,12 +116,13 @@ class BlockStoreShuffleReaderSuite extends SparkFunSuite with LocalSparkContext (shuffleBlockId, byteOutputStream.size().toLong) } val blocksToRetrieve = Seq( - (Option.apply(DefaultMapShuffleLocations.get(localBlockManagerId)), shuffleBlockIdsAndSizes)) + (Array(DefaultMapShuffleLocations.get(localBlockManagerId).asInstanceOf[ShuffleLocation]), + shuffleBlockIdsAndSizes)) val mapOutputTracker = mock(classOf[MapOutputTracker]) when(mapOutputTracker.getMapSizesByShuffleLocation(shuffleId, reduceId, reduceId + 1)) - .thenAnswer(new Answer[Iterator[(Option[ShuffleLocation], Seq[(BlockId, Long)])]] { + .thenAnswer(new Answer[Iterator[(Array[ShuffleLocation], Seq[(BlockId, Long)])]] { def answer(invocationOnMock: InvocationOnMock): - Iterator[(Option[ShuffleLocation], Seq[(BlockId, Long)])] = { + Iterator[(Array[ShuffleLocation], Seq[(BlockId, Long)])] = { blocksToRetrieve.iterator } }) From 070151bfe777c66ee79fa935ab7ecfa89f96c7c6 Mon Sep 17 00:00:00 2001 From: Yifei Huang Date: Fri, 10 May 2019 09:34:37 -0700 Subject: [PATCH 04/27] wip --- .../api/shuffle/MapShuffleLocations.java | 3 +- .../spark/api/shuffle/ShuffleLocation.java | 3 + .../sort/DefaultMapShuffleLocations.java | 14 +++-- .../org/apache/spark/MapOutputTracker.scala | 32 ++++++++--- .../org/apache/spark/TaskEndReason.scala | 5 +- .../apache/spark/scheduler/DAGScheduler.scala | 56 ++++++++++++------- .../spark/scheduler/TaskSetManager.scala | 13 ++++- .../spark/shuffle/FetchFailedException.scala | 14 +++-- .../storage/ShuffleBlockFetcherIterator.scala | 4 +- .../org/apache/spark/util/JsonProtocol.scala | 21 +++++-- .../spark/InternalAccumulatorSuite.scala | 12 ++-- .../apache/spark/MapOutputTrackerSuite.scala | 9 ++- .../org/apache/spark/SparkContextSuite.scala | 8 ++- .../apache/spark/executor/ExecutorSuite.scala | 4 +- .../spark/scheduler/DAGSchedulerSuite.scala | 52 +++++++++-------- .../OutputCommitCoordinatorSuite.scala | 8 ++- .../scheduler/SchedulerIntegrationSuite.scala | 2 +- .../spark/scheduler/TaskSetManagerSuite.scala | 7 ++- .../org/apache/spark/ui/UISeleniumSuite.scala | 8 ++- .../apache/spark/util/JsonProtocolSuite.scala | 17 +++--- 20 files changed, 189 insertions(+), 103 deletions(-) diff --git a/core/src/main/java/org/apache/spark/api/shuffle/MapShuffleLocations.java b/core/src/main/java/org/apache/spark/api/shuffle/MapShuffleLocations.java index 33caa74292058..1d2ceceb5bb0c 100644 --- a/core/src/main/java/org/apache/spark/api/shuffle/MapShuffleLocations.java +++ b/core/src/main/java/org/apache/spark/api/shuffle/MapShuffleLocations.java @@ -17,6 +17,7 @@ package org.apache.spark.api.shuffle; import org.apache.spark.annotation.Experimental; +import org.apache.spark.api.java.Optional; import java.io.Serializable; @@ -41,5 +42,5 @@ public interface MapShuffleLocations extends Serializable { * Deletes a ShuffleLocation from this MapShuffleLocations. Returns true if * the removal of this ShuffleLocation results in missing partitions. */ - boolean removeShuffleLocation(ShuffleLocation location); + boolean removeShuffleLocation(String host, Optional port); } diff --git a/core/src/main/java/org/apache/spark/api/shuffle/ShuffleLocation.java b/core/src/main/java/org/apache/spark/api/shuffle/ShuffleLocation.java index daa1d34c6ac34..8e0a5b726ef6f 100644 --- a/core/src/main/java/org/apache/spark/api/shuffle/ShuffleLocation.java +++ b/core/src/main/java/org/apache/spark/api/shuffle/ShuffleLocation.java @@ -17,6 +17,8 @@ package org.apache.spark.api.shuffle; +import org.apache.spark.api.java.Optional; + /** * Marker interface representing a location of a shuffle block. Implementations of shuffle readers * and writers are expected to cast this down to an implementation-specific representation. @@ -24,6 +26,7 @@ public abstract class ShuffleLocation { public abstract String host(); public abstract int port(); + public abstract Optional execId(); @Override public final String toString() { diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/DefaultMapShuffleLocations.java b/core/src/main/java/org/apache/spark/shuffle/sort/DefaultMapShuffleLocations.java index 0455b74969ea3..69c250e012b71 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/DefaultMapShuffleLocations.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/DefaultMapShuffleLocations.java @@ -21,6 +21,7 @@ import com.google.common.cache.CacheLoader; import com.google.common.cache.LoadingCache; +import org.apache.spark.api.java.Optional; import org.apache.spark.api.shuffle.MapShuffleLocations; import org.apache.spark.api.shuffle.ShuffleLocation; import org.apache.spark.storage.BlockManagerId; @@ -62,11 +63,11 @@ public ShuffleLocation[] getLocationsForBlock(int reduceId) { } @Override - public boolean removeShuffleLocation(ShuffleLocation location) { - if (location.host().equals(this.host()) && location.port() == this.port()) { - return true; + public boolean removeShuffleLocation(String host, Optional port) { + if (port.isPresent()) { + return this.host().equals(host) && this.port() == port.get(); } - return false; + return this.host().equals(host); } public BlockManagerId getBlockManagerId() { @@ -93,4 +94,9 @@ public String host() { public int port() { return location.port(); } + + @Override + public Optional execId() { + return Optional.of(location.executorId()); + } } diff --git a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala index 8111996325507..d1e97e1baad15 100644 --- a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala +++ b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala @@ -28,6 +28,7 @@ import scala.concurrent.duration.Duration import scala.reflect.ClassTag import scala.util.control.NonFatal +import org.apache.spark.api.java.Optional import org.apache.spark.api.shuffle.{MapShuffleLocations, ShuffleLocation} import org.apache.spark.broadcast.{Broadcast, BroadcastManager} import org.apache.spark.internal.Logging @@ -102,11 +103,19 @@ private class ShuffleStatus(numPartitions: Int) { * This is a no-op if there is no registered map output or if the registered output is from a * different block manager. */ - def removeMapOutput(mapId: Int, bmAddress: BlockManagerId): Unit = synchronized { - if (mapStatuses(mapId) != null && mapStatuses(mapId).location == bmAddress) { - _numAvailableOutputs -= 1 - mapStatuses(mapId) = null - invalidateSerializedMapOutputStatusCache() + def removeMapOutput(mapId: Int, shuffleLocations: Array[ShuffleLocation]): Unit = synchronized { + if (mapStatuses(mapId) != null) { + var shouldDelete = false + shuffleLocations.foreach { location => + shouldDelete = mapStatuses(mapId) + .mapShuffleLocations + .removeShuffleLocation(location.host(), Optional.of(location.port())) + } + if (shouldDelete) { + _numAvailableOutputs -= 1 + mapStatuses(mapId) = null + invalidateSerializedMapOutputStatusCache() + } } } @@ -115,7 +124,14 @@ private class ShuffleStatus(numPartitions: Int) { * outputs which are served by an external shuffle server (if one exists). */ def removeOutputsOnHost(host: String): Unit = { - removeOutputsByFilter(x => x.host == host) + for (mapId <- 0 until mapStatuses.length) { + if (mapStatuses(mapId) != null && + mapStatuses(mapId).mapShuffleLocations.removeShuffleLocation(host, Optional.empty())) { + _numAvailableOutputs -= 1 + mapStatuses(mapId) = null + invalidateSerializedMapOutputStatusCache() + } + } } /** @@ -424,10 +440,10 @@ private[spark] class MapOutputTrackerMaster( } /** Unregister map output information of the given shuffle, mapper and block manager */ - def unregisterMapOutput(shuffleId: Int, mapId: Int, bmAddress: BlockManagerId) { + def unregisterMapOutput(shuffleId: Int, mapId: Int, shuffleLocations: Array[ShuffleLocation]) { shuffleStatuses.get(shuffleId) match { case Some(shuffleStatus) => - shuffleStatus.removeMapOutput(mapId, bmAddress) + shuffleStatus.removeMapOutput(mapId, shuffleLocations) incrementEpoch() case None => throw new SparkException("unregisterMapOutput called for nonexistent shuffle ID") diff --git a/core/src/main/scala/org/apache/spark/TaskEndReason.scala b/core/src/main/scala/org/apache/spark/TaskEndReason.scala index 33901bc8380e9..6b8c3d72869bd 100644 --- a/core/src/main/scala/org/apache/spark/TaskEndReason.scala +++ b/core/src/main/scala/org/apache/spark/TaskEndReason.scala @@ -20,6 +20,7 @@ package org.apache.spark import java.io.{ObjectInputStream, ObjectOutputStream} import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.api.shuffle.ShuffleLocation import org.apache.spark.internal.Logging import org.apache.spark.scheduler.AccumulableInfo import org.apache.spark.storage.BlockManagerId @@ -81,14 +82,14 @@ case object Resubmitted extends TaskFailedReason { */ @DeveloperApi case class FetchFailed( - bmAddress: BlockManagerId, // Note that bmAddress can be null + shuffleLocation: Array[ShuffleLocation], // Note that bmAddress can be null shuffleId: Int, mapId: Int, reduceId: Int, message: String) extends TaskFailedReason { override def toErrorString: String = { - val bmAddressString = if (bmAddress == null) "null" else bmAddress.toString + val bmAddressString = if (shuffleLocation == null) "null" else shuffleLocation.toString s"FetchFailed($bmAddressString, shuffleId=$shuffleId, mapId=$mapId, reduceId=$reduceId, " + s"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 dd1b2595461fc..8b71d1b046dbb 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -30,7 +30,6 @@ import scala.concurrent.duration._ import scala.language.existentials import scala.language.postfixOps import scala.util.control.NonFatal - import org.apache.commons.lang3.SerializationUtils import org.apache.spark._ @@ -43,6 +42,7 @@ import org.apache.spark.network.util.JavaUtils import org.apache.spark.partial.{ApproximateActionListener, ApproximateEvaluator, PartialResult} import org.apache.spark.rdd.{DeterministicLevel, RDD, RDDCheckpointData} import org.apache.spark.rpc.RpcTimeout +import org.apache.spark.shuffle.sort.io.DefaultShuffleDataIO import org.apache.spark.storage._ import org.apache.spark.storage.BlockManagerMessages.BlockManagerHeartbeat import org.apache.spark.util._ @@ -1478,7 +1478,7 @@ private[spark] class DAGScheduler( } } - case FetchFailed(bmAddress, shuffleId, mapId, _, failureMessage) => + case FetchFailed(shuffleLocations, shuffleId, mapId, _, failureMessage) => val failedStage = stageIdToStage(task.stageId) val mapStage = shuffleIdToMapStage(shuffleId) @@ -1511,7 +1511,7 @@ private[spark] class DAGScheduler( mapOutputTracker.unregisterAllMapOutput(shuffleId) } else if (mapId != -1) { // Mark the map whose fetch failed as broken in the map stage - mapOutputTracker.unregisterMapOutput(shuffleId, mapId, bmAddress) + mapOutputTracker.unregisterMapOutput(shuffleId, mapId, shuffleLocations) } if (failedStage.rdd.isBarrier()) { @@ -1626,22 +1626,33 @@ private[spark] class DAGScheduler( } // TODO: mark the executor as failed only if there were lots of fetch failures on it - if (bmAddress != null) { - val hostToUnregisterOutputs = if (env.blockManager.externalShuffleServiceEnabled && - unRegisterOutputOnHostOnFetchFailure) { - // We had a fetch failure with the external shuffle service, so we - // assume all shuffle data on the node is bad. - Some(bmAddress.host) - } else { - // Unregister shuffle data just for one executor (we don't have any - // reason to believe shuffle data has been lost for the entire host). - None - } - removeExecutorAndUnregisterOutputs( - execId = bmAddress.executorId, - fileLost = true, - hostToUnregisterOutputs = hostToUnregisterOutputs, - maybeEpoch = Some(task.epoch)) + if (shuffleLocations != null) { + val toRemoveHost = + if (env.conf.get(config.SHUFFLE_IO_PLUGIN_CLASS) == + classOf[DefaultShuffleDataIO].getName) { + env.blockManager.externalShuffleServiceEnabled && + unRegisterOutputOnHostOnFetchFailure + } else { + unRegisterOutputOnHostOnFetchFailure + } + + shuffleLocations.foreach(location => { + val maybeExecId = location.execId() + if (!maybeExecId.isPresent) { + if (unRegisterOutputOnHostOnFetchFailure) { + // If execId is not present, then it's an external location, so we remove it + removeShuffleLocationHosts(shuffleLocations.map(_.host())) + } + } else if (toRemoveHost) { + // If execId is present, then it's an executor host + removeExecutorAndUnregisterOutputs( + execId = maybeExecId.get(), + fileLost = true, + hostToUnregisterOutputs = if (toRemoveHost) Some(location.host()) else None, + maybeEpoch = Some(task.epoch) + ) + } + }) } } @@ -1793,6 +1804,11 @@ private[spark] class DAGScheduler( maybeEpoch = None) } + private def removeShuffleLocationHosts(hosts: Array[String]): Unit = { + logInfo("Removing all outputs at hosts %s".format(hosts.toString)) + hosts.foreach(host => mapOutputTracker.removeOutputsOnHost(host)) + } + private def removeExecutorAndUnregisterOutputs( execId: String, fileLost: Boolean, @@ -1806,7 +1822,7 @@ private[spark] class DAGScheduler( if (fileLost) { hostToUnregisterOutputs match { case Some(host) => - logInfo("Shuffle files lost for host: %s (epoch %d)".format(host, currentEpoch)) + logInfo("Shuffle files lost for hosts: %s (epoch %d)".format(host, currentEpoch)) mapOutputTracker.removeOutputsOnHost(host) case None => logInfo("Shuffle files lost for executor: %s (epoch %d)".format(execId, currentEpoch)) 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 ea31fe80ef567..839e7674084bc 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala @@ -30,6 +30,7 @@ import org.apache.spark.TaskState.TaskState import org.apache.spark.internal.{config, Logging} import org.apache.spark.internal.config._ import org.apache.spark.scheduler.SchedulingMode._ +import org.apache.spark.shuffle.sort.DefaultMapShuffleLocations import org.apache.spark.util.{AccumulatorV2, Clock, LongAccumulator, SystemClock, Utils} import org.apache.spark.util.collection.MedianHeap @@ -848,9 +849,17 @@ private[spark] class TaskSetManager( } isZombie = true - if (fetchFailed.bmAddress != null) { + // Fetches from remote locations shouldn't affect executor scheduling since these remote + // locations shouldn't be running executors, so only fetches using the default Spark + // implementation (DefaultMapShuffleLocations) of fetching from executor disk should result + // in blacklistable executors. + if (fetchFailed.shuffleLocation != null && + fetchFailed.shuffleLocation.isInstanceOf[Array[DefaultMapShuffleLocations]]) { + val bmAddress = fetchFailed.shuffleLocation + .asInstanceOf[DefaultMapShuffleLocations] + .getBlockManagerId blacklistTracker.foreach(_.updateBlacklistForFetchFailure( - fetchFailed.bmAddress.host, fetchFailed.bmAddress.executorId)) + bmAddress.host, bmAddress.executorId)) } None 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 265a8acfa8d61..f84efd10663ba 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/FetchFailedException.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/FetchFailedException.scala @@ -18,7 +18,7 @@ package org.apache.spark.shuffle import org.apache.spark.{FetchFailed, TaskContext, TaskFailedReason} -import org.apache.spark.storage.BlockManagerId +import org.apache.spark.api.shuffle.ShuffleLocation import org.apache.spark.util.Utils /** @@ -33,7 +33,7 @@ import org.apache.spark.util.Utils * (or risk triggering any other exceptions). See SPARK-19276. */ private[spark] class FetchFailedException( - bmAddress: BlockManagerId, + shuffleLocations: Array[ShuffleLocation], shuffleId: Int, mapId: Int, reduceId: Int, @@ -42,12 +42,12 @@ private[spark] class FetchFailedException( extends Exception(message, cause) { def this( - bmAddress: BlockManagerId, + shuffleLocations: Array[ShuffleLocation], shuffleId: Int, mapId: Int, reduceId: Int, cause: Throwable) { - this(bmAddress, shuffleId, mapId, reduceId, cause.getMessage, cause) + this(shuffleLocations, shuffleId, mapId, reduceId, cause.getMessage, cause) } // SPARK-19276. We set the fetch failure in the task context, so that even if there is user-code @@ -56,7 +56,11 @@ private[spark] class FetchFailedException( // because the TaskContext is not defined in some test cases. Option(TaskContext.get()).map(_.setFetchFailed(this)) - def toTaskFailedReason: TaskFailedReason = FetchFailed(bmAddress, shuffleId, mapId, reduceId, + def toTaskFailedReason: TaskFailedReason = FetchFailed( + shuffleLocations, + shuffleId, + mapId, + reduceId, Utils.exceptionString(this)) } diff --git a/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala index bfed8dda6bf5b..f21c9bed1fe02 100644 --- a/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala +++ b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala @@ -580,7 +580,9 @@ final class ShuffleBlockFetcherIterator( private def throwFetchFailedException(blockId: BlockId, address: BlockManagerId, e: Throwable) = { blockId match { case ShuffleBlockId(shufId, mapId, reduceId) => - throw new FetchFailedException(address, shufId.toInt, mapId.toInt, reduceId, e) + throw new FetchFailedException( + DefaultMapShuffleLocations.get(address).getLocationsForBlock(0), + shufId.toInt, mapId.toInt, reduceId, 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/util/JsonProtocol.scala b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala index d84dd5800ebb7..d21da638edc09 100644 --- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -21,7 +21,6 @@ import java.util.{Properties, UUID} import scala.collection.JavaConverters._ import scala.collection.Map - import com.fasterxml.jackson.databind.ObjectMapper import com.fasterxml.jackson.module.scala.DefaultScalaModule import org.json4s.DefaultFormats @@ -30,6 +29,7 @@ import org.json4s.JsonDSL._ import org.json4s.jackson.JsonMethods._ import org.apache.spark._ +import org.apache.spark.api.shuffle.ShuffleLocation import org.apache.spark.executor._ import org.apache.spark.metrics.ExecutorMetricType import org.apache.spark.rdd.RDDOperationScope @@ -407,9 +407,9 @@ private[spark] object JsonProtocol { val reason = Utils.getFormattedClassName(taskEndReason) val json: JObject = taskEndReason match { case fetchFailed: FetchFailed => - val blockManagerAddress = Option(fetchFailed.bmAddress). - map(blockManagerIdToJson).getOrElse(JNothing) - ("Block Manager Address" -> blockManagerAddress) ~ + val blockManagerAddress = Option(fetchFailed.shuffleLocation). + map(mapper.writeValueAsString).getOrElse("None") + ("Shuffle Locations" -> blockManagerAddress) ~ ("Shuffle ID" -> fetchFailed.shuffleId) ~ ("Map ID" -> fetchFailed.mapId) ~ ("Reduce ID" -> fetchFailed.reduceId) ~ @@ -948,12 +948,13 @@ private[spark] object JsonProtocol { case `success` => Success case `resubmitted` => Resubmitted case `fetchFailed` => - val blockManagerAddress = blockManagerIdFromJson(json \ "Block Manager Address") + val locations = shuffleLocationsFromString( + (json \ "Shuffle Locations").extract[String]) val shuffleId = (json \ "Shuffle ID").extract[Int] val mapId = (json \ "Map ID").extract[Int] val reduceId = (json \ "Reduce ID").extract[Int] val message = jsonOption(json \ "Message").map(_.extract[String]) - new FetchFailed(blockManagerAddress, shuffleId, mapId, reduceId, + new FetchFailed(locations.get, shuffleId, mapId, reduceId, message.getOrElse("Unknown reason")) case `exceptionFailure` => val className = (json \ "Class Name").extract[String] @@ -996,6 +997,14 @@ private[spark] object JsonProtocol { } } + def shuffleLocationsFromString(string: String): Option[Array[ShuffleLocation]] = { + if (string == "None") { + return None + } + Some(mapper.readValue(string, classOf[Array[ShuffleLocation]])) + } + + def blockManagerIdFromJson(json: JValue): BlockManagerId = { // On metadata fetch fail, block manager ID can be null (SPARK-4471) if (json == JNothing) { diff --git a/core/src/test/scala/org/apache/spark/InternalAccumulatorSuite.scala b/core/src/test/scala/org/apache/spark/InternalAccumulatorSuite.scala index 28cbeeda7a88d..879bbdf893d2d 100644 --- a/core/src/test/scala/org/apache/spark/InternalAccumulatorSuite.scala +++ b/core/src/test/scala/org/apache/spark/InternalAccumulatorSuite.scala @@ -22,6 +22,7 @@ import scala.collection.mutable.ArrayBuffer import org.apache.spark.executor.TaskMetrics import org.apache.spark.scheduler.AccumulableInfo import org.apache.spark.shuffle.FetchFailedException +import org.apache.spark.shuffle.sort.DefaultMapShuffleLocations import org.apache.spark.util.{AccumulatorContext, AccumulatorV2} @@ -138,11 +139,12 @@ class InternalAccumulatorSuite extends SparkFunSuite with LocalSparkContext { val isFirstStageAttempt = taskContext.taskAttemptId() < numPartitions * 2L if (isFirstStageAttempt) { throw new FetchFailedException( - SparkEnv.get.blockManager.blockManagerId, - sid, - taskContext.partitionId(), - taskContext.partitionId(), - "simulated fetch failure") + shuffleLocations = + Array(DefaultMapShuffleLocations.get(SparkEnv.get.blockManager.blockManagerId)), + shuffleId = sid, + mapId = taskContext.partitionId(), + reduceId = taskContext.partitionId(), + message = "simulated fetch failure") } else { iter } diff --git a/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala b/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala index 0bd10475316b4..01bc00db149dd 100644 --- a/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala +++ b/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala @@ -119,8 +119,10 @@ class MapOutputTrackerSuite extends SparkFunSuite { assert(0 == tracker.getNumCachedSerializedBroadcast) // As if we had two simultaneous fetch failures - tracker.unregisterMapOutput(10, 0, BlockManagerId("a", "hostA", 1000)) - tracker.unregisterMapOutput(10, 0, BlockManagerId("a", "hostA", 1000)) + tracker.unregisterMapOutput(10, 0, + Array(DefaultMapShuffleLocations.get(BlockManagerId("a", "hostA", 1000)))) + tracker.unregisterMapOutput(10, 0, + Array(DefaultMapShuffleLocations.get(BlockManagerId("a", "hostA", 1000)))) // The remaining reduce task might try to grab the output despite the shuffle failure; // this should cause it to fail, and the scheduler will ignore the failure due to the @@ -160,7 +162,8 @@ class MapOutputTrackerSuite extends SparkFunSuite { assert(0 == masterTracker.getNumCachedSerializedBroadcast) val masterTrackerEpochBeforeLossOfMapOutput = masterTracker.getEpoch - masterTracker.unregisterMapOutput(10, 0, BlockManagerId("a", "hostA", 1000)) + masterTracker.unregisterMapOutput(10, 0, + Array(DefaultMapShuffleLocations.get(BlockManagerId("a", "hostA", 1000)))) assert(masterTracker.getEpoch > masterTrackerEpochBeforeLossOfMapOutput) slaveTracker.updateEpoch(masterTracker.getEpoch) intercept[FetchFailedException] { slaveTracker.getMapSizesByShuffleLocation(10, 0) } diff --git a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala index 7a16f7b715e63..758ef72032adb 100644 --- a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala @@ -23,7 +23,6 @@ import java.nio.charset.StandardCharsets import java.util.concurrent.{CountDownLatch, Semaphore, TimeUnit} import scala.concurrent.duration._ - import com.google.common.io.Files import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileSystem, Path} @@ -37,6 +36,7 @@ import org.apache.spark.internal.config._ import org.apache.spark.internal.config.UI._ import org.apache.spark.scheduler.{SparkListener, SparkListenerExecutorMetricsUpdate, SparkListenerJobStart, SparkListenerTaskEnd, SparkListenerTaskStart} import org.apache.spark.shuffle.FetchFailedException +import org.apache.spark.shuffle.sort.DefaultMapShuffleLocations import org.apache.spark.util.{ThreadUtils, Utils} @@ -689,8 +689,10 @@ class SparkContextSuite extends SparkFunSuite with LocalSparkContext with Eventu if (context.stageAttemptNumber == 0) { if (context.partitionId == 0) { // Make the first task in the first stage attempt fail. - throw new FetchFailedException(SparkEnv.get.blockManager.blockManagerId, 0, 0, 0, - new java.io.IOException("fake")) + throw new FetchFailedException( + shuffleLocations = + Array(DefaultMapShuffleLocations.get(SparkEnv.get.blockManager.blockManagerId)), + shuffleId =0, mapId =0, reduceId =0, cause = new java.io.IOException("fake")) } else { // Make the second task in the first stage attempt sleep to generate a zombie task Thread.sleep(60000) diff --git a/core/src/test/scala/org/apache/spark/executor/ExecutorSuite.scala b/core/src/test/scala/org/apache/spark/executor/ExecutorSuite.scala index 558cd3626ab91..c4ead61f47efc 100644 --- a/core/src/test/scala/org/apache/spark/executor/ExecutorSuite.scala +++ b/core/src/test/scala/org/apache/spark/executor/ExecutorSuite.scala @@ -28,7 +28,6 @@ import scala.collection.mutable.ArrayBuffer import scala.collection.mutable.Map import scala.concurrent.duration._ import scala.language.postfixOps - import org.mockito.ArgumentCaptor import org.mockito.ArgumentMatchers.{any, eq => meq} import org.mockito.Mockito.{inOrder, verify, when} @@ -49,6 +48,7 @@ import org.apache.spark.rpc.{RpcEndpointRef, RpcEnv, RpcTimeout} import org.apache.spark.scheduler.{FakeTask, ResultTask, Task, TaskDescription} import org.apache.spark.serializer.{JavaSerializer, SerializerManager} import org.apache.spark.shuffle.FetchFailedException +import org.apache.spark.shuffle.sort.DefaultMapShuffleLocations import org.apache.spark.storage.{BlockManager, BlockManagerId} import org.apache.spark.util.{LongAccumulator, UninterruptibleThread} @@ -438,7 +438,7 @@ class FetchFailureThrowingRDD(sc: SparkContext) extends RDD[Int](sc, Nil) { override def hasNext: Boolean = true override def next(): Int = { throw new FetchFailedException( - bmAddress = BlockManagerId("1", "hostA", 1234), + Array(DefaultMapShuffleLocations.get(BlockManagerId("1", "hostA", 1234))), shuffleId = 0, mapId = 0, reduceId = 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 b04551c0f731b..2248b81306a78 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -474,7 +474,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi // reduce stage fails with a fetch failure from one host complete(taskSets(2), Seq( - (FetchFailed(BlockManagerId("exec-hostA2", "hostA", 12345), firstShuffleId, 0, 0, "ignored"), + (FetchFailed(makeShuffleLocationArray("exec-hostA2", "hostA", 12345), firstShuffleId, 0, 0, "ignored"), null) )) @@ -720,7 +720,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi // the 2nd ResultTask failed complete(taskSets(1), Seq( (Success, 42), - (FetchFailed(makeBlockManagerId("hostA"), shuffleId, 0, 0, "ignored"), null))) + (FetchFailed(makeShuffleLocationArray("hostA"), shuffleId, 0, 0, "ignored"), null))) // this will get called // blockManagerMaster.removeExecutor("exec-hostA") // ask the scheduler to try it again @@ -839,7 +839,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi 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(makeShuffleLocationArray("hostA"), shuffleDep.shuffleId, 0, idx, "ignored"), null) }.toSeq) } @@ -1074,7 +1074,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi // The first result task fails, with a fetch failure for the output from the first mapper. runEvent(makeCompletionEvent( taskSets(1).tasks(0), - FetchFailed(makeBlockManagerId("hostA"), shuffleId, 0, 0, "ignored"), + FetchFailed(makeShuffleLocationArray("hostA"), shuffleId, 0, 0, "ignored"), null)) sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS) assert(sparkListener.failedStages.contains(1)) @@ -1082,7 +1082,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi // The second ResultTask fails, with a fetch failure for the output from the second mapper. runEvent(makeCompletionEvent( taskSets(1).tasks(0), - FetchFailed(makeBlockManagerId("hostA"), shuffleId, 1, 1, "ignored"), + FetchFailed(makeShuffleLocationArray("hostA"), shuffleId, 1, 1, "ignored"), null)) // The SparkListener should not receive redundant failure events. sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS) @@ -1103,7 +1103,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi // The first result task fails, with a fetch failure for the output from the first mapper. runEvent(makeCompletionEvent( taskSets(1).tasks(0), - FetchFailed(makeBlockManagerId("hostA"), shuffleId, 0, 0, "ignored"), + FetchFailed(makeShuffleLocationArray("hostA"), shuffleId, 0, 0, "ignored"), null)) assert(mapOutputTracker.findMissingPartitions(shuffleId) === Some(Seq(0, 1))) @@ -1210,7 +1210,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi // The first result task fails, with a fetch failure for the output from the first mapper. runEvent(makeCompletionEvent( taskSets(1).tasks(0), - FetchFailed(makeBlockManagerId("hostA"), shuffleId, 0, 0, "ignored"), + FetchFailed(makeShuffleLocationArray("hostA"), shuffleId, 0, 0, "ignored"), null)) sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS) assert(sparkListener.failedStages.contains(1)) @@ -1225,7 +1225,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi // The second ResultTask fails, with a fetch failure for the output from the second mapper. runEvent(makeCompletionEvent( taskSets(1).tasks(1), - FetchFailed(makeBlockManagerId("hostB"), shuffleId, 1, 1, "ignored"), + FetchFailed(makeShuffleLocationArray("hostB"), shuffleId, 1, 1, "ignored"), null)) // Another ResubmitFailedStages event should not result in another attempt for the map @@ -1274,7 +1274,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi // The first result task fails, with a fetch failure for the output from the first mapper. runEvent(makeCompletionEvent( taskSets(1).tasks(0), - FetchFailed(makeBlockManagerId("hostA"), shuffleId, 0, 0, "ignored"), + FetchFailed(makeShuffleLocationArray("hostA"), shuffleId, 0, 0, "ignored"), null)) // Trigger resubmission of the failed map stage and finish the re-started map task. @@ -1290,7 +1290,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi // A late FetchFailed arrives from the second task in the original reduce stage. runEvent(makeCompletionEvent( taskSets(1).tasks(1), - FetchFailed(makeBlockManagerId("hostB"), shuffleId, 1, 1, "ignored"), + FetchFailed(makeShuffleLocationArray("hostB"), shuffleId, 1, 1, "ignored"), null)) // Running ResubmitFailedStages shouldn't result in any more attempts for the map stage, because @@ -1750,7 +1750,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi // lets say there is a fetch failure in this task set, which makes us go back and // run stage 0, attempt 1 complete(taskSets(1), Seq( - (FetchFailed(makeBlockManagerId("hostA"), shuffleDep1.shuffleId, 0, 0, "ignored"), null))) + (FetchFailed(makeShuffleLocationArray("hostA"), shuffleDep1.shuffleId, 0, 0, "ignored"), null))) scheduler.resubmitFailedStages() // stage 0, attempt 1 should have the properties of job2 @@ -1831,7 +1831,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi (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(makeShuffleLocationArray("hostA"), shuffleDepTwo.shuffleId, 0, 0, "ignored"), null))) // TODO assert this: // blockManagerMaster.removeExecutor("exec-hostA") // have DAGScheduler try again @@ -1862,7 +1862,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi (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(makeShuffleLocationArray("hostA"), shuffleDepTwo.shuffleId, 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. @@ -2223,7 +2223,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi submit(reduceRdd, Array(0, 1)) complete(taskSets(1), Seq( (Success, 42), - (FetchFailed(makeBlockManagerId("hostA"), shuffleId, 0, 0, "ignored"), null))) + (FetchFailed(makeShuffleLocationArray("hostA"), shuffleId, 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() @@ -2282,7 +2282,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi 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(makeShuffleLocationArray("hostA"), dep1.shuffleId, 0, 0, "ignored"), null))) scheduler.resubmitFailedStages() assert(listener2.results.size === 0) // Second stage listener should not have a result yet @@ -2309,7 +2309,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi assert(taskSets(4).stageId === 2) complete(taskSets(4), Seq( (Success, 52), - (FetchFailed(makeBlockManagerId("hostD"), dep2.shuffleId, 0, 0, "ignored"), null))) + (FetchFailed(makeShuffleLocationArray("hostD"), dep2.shuffleId, 0, 0, "ignored"), null))) scheduler.resubmitFailedStages() // TaskSet 5 will rerun stage 1's lost task, then TaskSet 6 will rerun stage 2 @@ -2347,7 +2347,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi assert(taskSets(1).stageId === 1) complete(taskSets(1), Seq( (Success, makeMapStatus("hostC", rdd2.partitions.length)), - (FetchFailed(makeBlockManagerId("hostA"), dep1.shuffleId, 0, 0, "ignored"), null))) + (FetchFailed(makeShuffleLocationArray("hostA"), dep1.shuffleId, 0, 0, "ignored"), null))) scheduler.resubmitFailedStages() // Stage1 listener should not have a result yet assert(listener2.results.size === 0) @@ -2482,7 +2482,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi rdd1.map { case (x, _) if (x == 1) => throw new FetchFailedException( - BlockManagerId("1", "1", 1), shuffleHandle.shuffleId, 0, 0, "test") + makeShuffleLocationArray("1", "1", 1), shuffleHandle.shuffleId, 0, 0, "test") case (x, _) => x }.count() } @@ -2495,7 +2495,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi rdd1.map { case (x, _) if (x == 1) && FailThisAttempt._fail.getAndSet(false) => throw new FetchFailedException( - BlockManagerId("1", "1", 1), shuffleHandle.shuffleId, 0, 0, "test") + makeShuffleLocationArray("1", "1", 1), shuffleHandle.shuffleId, 0, 0, "test") } } @@ -2549,7 +2549,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi assert(taskSets(1).stageId === 1 && taskSets(1).stageAttemptId === 0) runEvent(makeCompletionEvent( taskSets(1).tasks(0), - FetchFailed(makeBlockManagerId("hostA"), shuffleIdA, 0, 0, + FetchFailed(makeShuffleLocationArray("hostA"), shuffleIdA, 0, 0, "Fetch failure of task: stageId=1, stageAttempt=0, partitionId=0"), result = null)) @@ -2737,7 +2737,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi // The first task of the final stage failed with fetch failure runEvent(makeCompletionEvent( taskSets(2).tasks(0), - FetchFailed(makeBlockManagerId("hostC"), shuffleId2, 0, 0, "ignored"), + FetchFailed(makeShuffleLocationArray("hostC"), shuffleId2, 0, 0, "ignored"), null)) val failedStages = scheduler.failedStages.toSeq @@ -2756,7 +2756,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi // The first task of the `shuffleMapRdd2` failed with fetch failure runEvent(makeCompletionEvent( taskSets(3).tasks(0), - FetchFailed(makeBlockManagerId("hostA"), shuffleId1, 0, 0, "ignored"), + FetchFailed(makeShuffleLocationArray("hostA"), shuffleId1, 0, 0, "ignored"), null)) // The job should fail because Spark can't rollback the shuffle map stage. @@ -2781,7 +2781,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi // Fail the second task with FetchFailed. runEvent(makeCompletionEvent( taskSets.last.tasks(1), - FetchFailed(makeBlockManagerId("hostA"), shuffleId, 0, 0, "ignored"), + FetchFailed(makeShuffleLocationArray("hostA"), shuffleId, 0, 0, "ignored"), null)) // The job should fail because Spark can't rollback the result stage. @@ -2824,7 +2824,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi // Fail the second task with FetchFailed. runEvent(makeCompletionEvent( taskSets.last.tasks(1), - FetchFailed(makeBlockManagerId("hostA"), shuffleId, 0, 0, "ignored"), + FetchFailed(makeShuffleLocationArray("hostA"), shuffleId, 0, 0, "ignored"), null)) assert(failure == null, "job should not fail") @@ -2926,6 +2926,10 @@ object DAGSchedulerSuite { def makeShuffleLocationArray(host: String): Array[ShuffleLocation] = { DefaultMapShuffleLocations.get(makeBlockManagerId(host)).getLocationsForBlock(0) } + + def makeShuffleLocationArray(execId: String, host: String, port: Int): Array[ShuffleLocation] = { + DefaultMapShuffleLocations.get(makeBlockManagerId(host)).getLocationsForBlock(0) + } } object FailThisAttempt { diff --git a/core/src/test/scala/org/apache/spark/scheduler/OutputCommitCoordinatorSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/OutputCommitCoordinatorSuite.scala index a560013dba963..0d23e8f46bf74 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/OutputCommitCoordinatorSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/OutputCommitCoordinatorSuite.scala @@ -23,7 +23,6 @@ import java.util.concurrent.TimeoutException import scala.concurrent.duration._ import scala.language.postfixOps - import org.apache.hadoop.mapred._ import org.apache.hadoop.mapreduce.TaskType import org.mockito.ArgumentMatchers.{any, eq => meq} @@ -36,6 +35,7 @@ import org.apache.spark._ import org.apache.spark.internal.io.{FileCommitProtocol, HadoopMapRedCommitProtocol, SparkHadoopWriterUtils} import org.apache.spark.rdd.{FakeOutputCommitter, RDD} import org.apache.spark.shuffle.FetchFailedException +import org.apache.spark.shuffle.sort.DefaultMapShuffleLocations import org.apache.spark.util.{ThreadUtils, Utils} /** @@ -257,8 +257,10 @@ class OutputCommitCoordinatorSuite extends SparkFunSuite with BeforeAndAfter { .reduceByKey { case (_, _) => val ctx = TaskContext.get() if (ctx.stageAttemptNumber() == 0) { - throw new FetchFailedException(SparkEnv.get.blockManager.blockManagerId, 1, 1, 1, - new Exception("Failure for test.")) + throw new FetchFailedException( + shuffleLocations = + Array(DefaultMapShuffleLocations.get(SparkEnv.get.blockManager.blockManagerId)), + shuffleId = 1, mapId = 1, reduceId = 1, cause = new Exception("Failure for test.")) } else { ctx.stageId() } diff --git a/core/src/test/scala/org/apache/spark/scheduler/SchedulerIntegrationSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/SchedulerIntegrationSuite.scala index 83305a96e6794..a94b677d59023 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/SchedulerIntegrationSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/SchedulerIntegrationSuite.scala @@ -627,7 +627,7 @@ class BasicSchedulerIntegrationSuite extends SchedulerIntegrationSuite[SingleCor backend.taskSuccess(taskDescription, DAGSchedulerSuite.makeMapStatus("hostA", 10)) case (1, 0, 0) => val fetchFailed = FetchFailed( - DAGSchedulerSuite.makeBlockManagerId("hostA"), shuffleId, 0, 0, "ignored") + DAGSchedulerSuite.makeShuffleLocationArray("hostA"), shuffleId, 0, 0, "ignored") backend.taskFailed(taskDescription, fetchFailed) case (1, _, partition) => backend.taskSuccess(taskDescription, 42 + partition) 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 60acd3ed4cd49..de07ed45b8d6f 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala @@ -21,7 +21,6 @@ import java.util.{Properties, Random} import scala.collection.mutable import scala.collection.mutable.ArrayBuffer - import org.mockito.ArgumentMatchers.{any, anyInt, anyString} import org.mockito.Mockito.{mock, never, spy, times, verify, when} import org.mockito.invocation.InvocationOnMock @@ -31,6 +30,7 @@ import org.apache.spark._ import org.apache.spark.internal.Logging import org.apache.spark.internal.config import org.apache.spark.serializer.SerializerInstance +import org.apache.spark.shuffle.sort.DefaultMapShuffleLocations import org.apache.spark.storage.BlockManagerId import org.apache.spark.util.{AccumulatorV2, ManualClock, Utils} @@ -1246,7 +1246,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg // now fail those tasks tsmSpy.handleFailedTask(taskDescs(0).taskId, TaskState.FAILED, - FetchFailed(BlockManagerId(taskDescs(0).executorId, "host1", 12345), 0, 0, 0, "ignored")) + FetchFailed(Array(DefaultMapShuffleLocations.get(BlockManagerId(taskDescs(0).executorId, "host1", 12345))), 0, 0, 0, "ignored")) tsmSpy.handleFailedTask(taskDescs(1).taskId, TaskState.FAILED, ExecutorLostFailure(taskDescs(1).executorId, exitCausedByApp = false, reason = None)) tsmSpy.handleFailedTask(taskDescs(2).taskId, TaskState.FAILED, @@ -1286,7 +1286,8 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg // Fail the task with fetch failure tsm.handleFailedTask(taskDescs(0).taskId, TaskState.FAILED, - FetchFailed(BlockManagerId(taskDescs(0).executorId, "host1", 12345), 0, 0, 0, "ignored")) + FetchFailed( + Array(DefaultMapShuffleLocations.get(BlockManagerId(taskDescs(0).executorId, "host1", 12345))), 0, 0, 0, "ignored")) assert(blacklistTracker.isNodeBlacklisted("host1")) } 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 b184b74bf3cb0..9ba260d924025 100644 --- a/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala @@ -19,11 +19,10 @@ package org.apache.spark.ui import java.net.{HttpURLConnection, URL} import java.util.Locale -import javax.servlet.http.{HttpServletRequest, HttpServletResponse} +import javax.servlet.http.{HttpServletRequest, HttpServletResponse} import scala.io.Source import scala.xml.Node - import com.gargoylesoftware.htmlunit.DefaultCssErrorHandler import org.json4s._ import org.json4s.jackson.JsonMethods @@ -43,6 +42,7 @@ import org.apache.spark.internal.config._ import org.apache.spark.internal.config.Status._ import org.apache.spark.internal.config.UI._ import org.apache.spark.shuffle.FetchFailedException +import org.apache.spark.shuffle.sort.DefaultMapShuffleLocations import org.apache.spark.status.api.v1.{JacksonMessageWriter, RDDDataDistribution, StageStatus} private[spark] class SparkUICssErrorHandler extends DefaultCssErrorHandler { @@ -315,7 +315,9 @@ class UISeleniumSuite extends SparkFunSuite with WebBrowser with Matchers with B val mapId = 0 val reduceId = taskContext.partitionId() val message = "Simulated fetch failure" - throw new FetchFailedException(bmAddress, shuffleId, mapId, reduceId, message) + throw new FetchFailedException( + shuffleLocations = Array(DefaultMapShuffleLocations.get(bmAddress)), + shuffleId = shuffleId, mapId = mapId, reduceId = reduceId, message = message) } else { x } 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 c3ff379c84fff..7b53155aca99a 100644 --- a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala @@ -21,7 +21,6 @@ import java.util.Properties import scala.collection.JavaConverters._ import scala.collection.Map - import org.json4s.JsonAST.{JArray, JInt, JString, JValue} import org.json4s.JsonDSL._ import org.json4s.jackson.JsonMethods._ @@ -35,6 +34,7 @@ import org.apache.spark.rdd.RDDOperationScope import org.apache.spark.scheduler._ import org.apache.spark.scheduler.cluster.ExecutorInfo import org.apache.spark.shuffle.MetadataFetchFailedException +import org.apache.spark.shuffle.sort.DefaultMapShuffleLocations import org.apache.spark.storage._ class JsonProtocolSuite extends SparkFunSuite { @@ -169,8 +169,9 @@ class JsonProtocolSuite extends SparkFunSuite { testJobResult(jobFailed) // TaskEndReason - val fetchFailed = FetchFailed(BlockManagerId("With or", "without you", 15), 17, 18, 19, - "Some exception") + val fetchFailed = FetchFailed( + Array(DefaultMapShuffleLocations.get(BlockManagerId("With or", "without you", 15))), + 17, 18, 19, "Some exception") val fetchMetadataFailed = new MetadataFetchFailedException(17, 19, "metadata Fetch failed exception").toTaskFailedReason val exceptionFailure = new ExceptionFailure(exception, Seq.empty[AccumulableInfo]) @@ -286,11 +287,13 @@ class JsonProtocolSuite extends SparkFunSuite { test("FetchFailed backwards compatibility") { // FetchFailed in Spark 1.1.0 does not have a "Message" property. - val fetchFailed = FetchFailed(BlockManagerId("With or", "without you", 15), 17, 18, 19, - "ignored") + val fetchFailed = FetchFailed( + Array(DefaultMapShuffleLocations.get(BlockManagerId("With or", "without you", 15))), + 17, 18, 19, "ignored") val oldEvent = JsonProtocol.taskEndReasonToJson(fetchFailed) .removeField({ _._1 == "Message" }) - val expectedFetchFailed = FetchFailed(BlockManagerId("With or", "without you", 15), 17, 18, 19, + val expectedFetchFailed = FetchFailed( + Array(DefaultMapShuffleLocations.get(BlockManagerId("With or", "without you", 15))), 17, 18, 19, "Unknown reason") assert(expectedFetchFailed === JsonProtocol.taskEndReasonFromJson(oldEvent)) } @@ -713,7 +716,7 @@ private[spark] object JsonProtocolSuite extends Assertions { assert(r1.shuffleId === r2.shuffleId) assert(r1.mapId === r2.mapId) assert(r1.reduceId === r2.reduceId) - assert(r1.bmAddress === r2.bmAddress) + assert(r1.shuffleLocation === r2.shuffleLocation) assert(r1.message === r2.message) case (r1: ExceptionFailure, r2: ExceptionFailure) => assert(r1.className === r2.className) From 190e2deadd3aba53e0657206bd7c428a974da114 Mon Sep 17 00:00:00 2001 From: Yifei Huang Date: Fri, 10 May 2019 14:55:31 -0700 Subject: [PATCH 05/27] fix --- .../apache/spark/scheduler/DAGScheduler.scala | 29 +++++++++++-------- 1 file changed, 17 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 8b71d1b046dbb..9a70273a2d527 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -1637,22 +1637,27 @@ private[spark] class DAGScheduler( } shuffleLocations.foreach(location => { + var epochAllowsRemoval = false + // if there's an executor id, remove it val maybeExecId = location.execId() - if (!maybeExecId.isPresent) { - if (unRegisterOutputOnHostOnFetchFailure) { - // If execId is not present, then it's an external location, so we remove it - removeShuffleLocationHosts(shuffleLocations.map(_.host())) + val currentEpoch = Some(task.epoch).getOrElse(mapOutputTracker.getEpoch) + if (maybeExecId.isPresent) { + val execId = maybeExecId.get() + if (!failedEpoch.contains(execId) || failedEpoch(execId) < currentEpoch) { + failedEpoch(execId) = currentEpoch + epochAllowsRemoval = true + blockManagerMaster.removeExecutor(execId) + mapOutputTracker.removeOutputsOnExecutor(execId) } - } else if (toRemoveHost) { - // If execId is present, then it's an executor host - removeExecutorAndUnregisterOutputs( - execId = maybeExecId.get(), - fileLost = true, - hostToUnregisterOutputs = if (toRemoveHost) Some(location.host()) else None, - maybeEpoch = Some(task.epoch) - ) + } else { + epochAllowsRemoval = true + } + + if (toRemoveHost && epochAllowsRemoval) { + mapOutputTracker.removeOutputsOnHost(location.host()) } }) + clearCacheLocs() } } From a34a10339fb2929e5d61494a8cec485e3a7b8ecb Mon Sep 17 00:00:00 2001 From: Yifei Huang Date: Fri, 10 May 2019 18:30:28 -0700 Subject: [PATCH 06/27] style --- .../apache/spark/scheduler/DAGScheduler.scala | 1 + .../org/apache/spark/util/JsonProtocol.scala | 1 + .../org/apache/spark/SparkContextSuite.scala | 3 ++- .../apache/spark/executor/ExecutorSuite.scala | 1 + .../spark/scheduler/DAGSchedulerSuite.scala | 17 +++++++++++------ .../OutputCommitCoordinatorSuite.scala | 1 + .../spark/scheduler/TaskSetManagerSuite.scala | 7 +++++-- .../org/apache/spark/ui/UISeleniumSuite.scala | 3 ++- .../apache/spark/util/JsonProtocolSuite.scala | 5 +++-- 9 files changed, 27 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 9a70273a2d527..12162ab024c4f 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -30,6 +30,7 @@ import scala.concurrent.duration._ import scala.language.existentials import scala.language.postfixOps import scala.util.control.NonFatal + import org.apache.commons.lang3.SerializationUtils import org.apache.spark._ 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 d21da638edc09..0c4c18518f854 100644 --- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -21,6 +21,7 @@ import java.util.{Properties, UUID} import scala.collection.JavaConverters._ import scala.collection.Map + import com.fasterxml.jackson.databind.ObjectMapper import com.fasterxml.jackson.module.scala.DefaultScalaModule import org.json4s.DefaultFormats diff --git a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala index 758ef72032adb..149e61e56c4dd 100644 --- a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala @@ -23,6 +23,7 @@ import java.nio.charset.StandardCharsets import java.util.concurrent.{CountDownLatch, Semaphore, TimeUnit} import scala.concurrent.duration._ + import com.google.common.io.Files import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileSystem, Path} @@ -692,7 +693,7 @@ class SparkContextSuite extends SparkFunSuite with LocalSparkContext with Eventu throw new FetchFailedException( shuffleLocations = Array(DefaultMapShuffleLocations.get(SparkEnv.get.blockManager.blockManagerId)), - shuffleId =0, mapId =0, reduceId =0, cause = new java.io.IOException("fake")) + shuffleId = 0, mapId = 0, reduceId = 0, cause = new java.io.IOException("fake")) } else { // Make the second task in the first stage attempt sleep to generate a zombie task Thread.sleep(60000) diff --git a/core/src/test/scala/org/apache/spark/executor/ExecutorSuite.scala b/core/src/test/scala/org/apache/spark/executor/ExecutorSuite.scala index c4ead61f47efc..65621a418134c 100644 --- a/core/src/test/scala/org/apache/spark/executor/ExecutorSuite.scala +++ b/core/src/test/scala/org/apache/spark/executor/ExecutorSuite.scala @@ -28,6 +28,7 @@ import scala.collection.mutable.ArrayBuffer import scala.collection.mutable.Map import scala.concurrent.duration._ import scala.language.postfixOps + import org.mockito.ArgumentCaptor import org.mockito.ArgumentMatchers.{any, eq => meq} import org.mockito.Mockito.{inOrder, verify, when} 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 2248b81306a78..f15c0cc152660 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -24,6 +24,7 @@ import scala.annotation.meta.param import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet, Map} import scala.language.reflectiveCalls import scala.util.control.NonFatal + import org.scalatest.concurrent.{Signaler, ThreadSignaler, TimeLimits} import org.scalatest.time.SpanSugar._ @@ -474,8 +475,8 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi // reduce stage fails with a fetch failure from one host complete(taskSets(2), Seq( - (FetchFailed(makeShuffleLocationArray("exec-hostA2", "hostA", 12345), firstShuffleId, 0, 0, "ignored"), - null) + (FetchFailed(makeShuffleLocationArray("exec-hostA2", "hostA", 12345), + firstShuffleId, 0, 0, "ignored"), null) )) // Here is the main assertion -- make sure that we de-register @@ -839,7 +840,8 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi val stageAttempt = taskSets.last checkStageId(stageId, attemptIdx, stageAttempt) complete(stageAttempt, stageAttempt.tasks.zipWithIndex.map { case (task, idx) => - (FetchFailed(makeShuffleLocationArray("hostA"), shuffleDep.shuffleId, 0, idx, "ignored"), null) + (FetchFailed(makeShuffleLocationArray("hostA"), + shuffleDep.shuffleId, 0, idx, "ignored"), null) }.toSeq) } @@ -1750,7 +1752,8 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi // lets say there is a fetch failure in this task set, which makes us go back and // run stage 0, attempt 1 complete(taskSets(1), Seq( - (FetchFailed(makeShuffleLocationArray("hostA"), shuffleDep1.shuffleId, 0, 0, "ignored"), null))) + (FetchFailed(makeShuffleLocationArray("hostA"), + shuffleDep1.shuffleId, 0, 0, "ignored"), null))) scheduler.resubmitFailedStages() // stage 0, attempt 1 should have the properties of job2 @@ -1831,7 +1834,8 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi (Success, makeMapStatus("hostC", 1)))) // fail the third stage because hostA went down complete(taskSets(2), Seq( - (FetchFailed(makeShuffleLocationArray("hostA"), shuffleDepTwo.shuffleId, 0, 0, "ignored"), null))) + (FetchFailed(makeShuffleLocationArray("hostA"), + shuffleDepTwo.shuffleId, 0, 0, "ignored"), null))) // TODO assert this: // blockManagerMaster.removeExecutor("exec-hostA") // have DAGScheduler try again @@ -1862,7 +1866,8 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi (Success, makeMapStatus("hostB", 1)))) // pretend stage 2 failed because hostA went down complete(taskSets(2), Seq( - (FetchFailed(makeShuffleLocationArray("hostA"), shuffleDepTwo.shuffleId, 0, 0, "ignored"), null))) + (FetchFailed(makeShuffleLocationArray("hostA"), + shuffleDepTwo.shuffleId, 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. diff --git a/core/src/test/scala/org/apache/spark/scheduler/OutputCommitCoordinatorSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/OutputCommitCoordinatorSuite.scala index 0d23e8f46bf74..262487f8befe7 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/OutputCommitCoordinatorSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/OutputCommitCoordinatorSuite.scala @@ -23,6 +23,7 @@ import java.util.concurrent.TimeoutException import scala.concurrent.duration._ import scala.language.postfixOps + import org.apache.hadoop.mapred._ import org.apache.hadoop.mapreduce.TaskType import org.mockito.ArgumentMatchers.{any, eq => meq} 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 de07ed45b8d6f..cfcb2a4c546f1 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala @@ -21,6 +21,7 @@ import java.util.{Properties, Random} import scala.collection.mutable import scala.collection.mutable.ArrayBuffer + import org.mockito.ArgumentMatchers.{any, anyInt, anyString} import org.mockito.Mockito.{mock, never, spy, times, verify, when} import org.mockito.invocation.InvocationOnMock @@ -1246,7 +1247,8 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg // now fail those tasks tsmSpy.handleFailedTask(taskDescs(0).taskId, TaskState.FAILED, - FetchFailed(Array(DefaultMapShuffleLocations.get(BlockManagerId(taskDescs(0).executorId, "host1", 12345))), 0, 0, 0, "ignored")) + FetchFailed(Array(DefaultMapShuffleLocations.get( + BlockManagerId(taskDescs(0).executorId, "host1", 12345))), 0, 0, 0, "ignored")) tsmSpy.handleFailedTask(taskDescs(1).taskId, TaskState.FAILED, ExecutorLostFailure(taskDescs(1).executorId, exitCausedByApp = false, reason = None)) tsmSpy.handleFailedTask(taskDescs(2).taskId, TaskState.FAILED, @@ -1287,7 +1289,8 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg // Fail the task with fetch failure tsm.handleFailedTask(taskDescs(0).taskId, TaskState.FAILED, FetchFailed( - Array(DefaultMapShuffleLocations.get(BlockManagerId(taskDescs(0).executorId, "host1", 12345))), 0, 0, 0, "ignored")) + Array(DefaultMapShuffleLocations.get( + BlockManagerId(taskDescs(0).executorId, "host1", 12345))), 0, 0, 0, "ignored")) assert(blacklistTracker.isNodeBlacklisted("host1")) } 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 9ba260d924025..dc8ca509ee8e4 100644 --- a/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala @@ -19,10 +19,11 @@ package org.apache.spark.ui import java.net.{HttpURLConnection, URL} import java.util.Locale - import javax.servlet.http.{HttpServletRequest, HttpServletResponse} + import scala.io.Source import scala.xml.Node + import com.gargoylesoftware.htmlunit.DefaultCssErrorHandler import org.json4s._ import org.json4s.jackson.JsonMethods 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 7b53155aca99a..9d8c81ccbb2a0 100644 --- a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala @@ -21,6 +21,7 @@ import java.util.Properties import scala.collection.JavaConverters._ import scala.collection.Map + import org.json4s.JsonAST.{JArray, JInt, JString, JValue} import org.json4s.JsonDSL._ import org.json4s.jackson.JsonMethods._ @@ -293,8 +294,8 @@ class JsonProtocolSuite extends SparkFunSuite { val oldEvent = JsonProtocol.taskEndReasonToJson(fetchFailed) .removeField({ _._1 == "Message" }) val expectedFetchFailed = FetchFailed( - Array(DefaultMapShuffleLocations.get(BlockManagerId("With or", "without you", 15))), 17, 18, 19, - "Unknown reason") + Array(DefaultMapShuffleLocations.get(BlockManagerId("With or", "without you", 15))), + 17, 18, 19, "Unknown reason") assert(expectedFetchFailed === JsonProtocol.taskEndReasonFromJson(oldEvent)) } From 93c244990d066696c1b42cdf5a2eea8a5856c243 Mon Sep 17 00:00:00 2001 From: Yifei Huang Date: Mon, 13 May 2019 09:41:40 -0700 Subject: [PATCH 07/27] fix style, fix test --- .../spark/api/shuffle/ShuffleBlockInfo.java | 2 -- .../org/apache/spark/MapOutputTrackerSuite.scala | 16 ++++++++-------- 2 files changed, 8 insertions(+), 10 deletions(-) diff --git a/core/src/main/java/org/apache/spark/api/shuffle/ShuffleBlockInfo.java b/core/src/main/java/org/apache/spark/api/shuffle/ShuffleBlockInfo.java index 46cb3343374df..da5c74a689d1b 100644 --- a/core/src/main/java/org/apache/spark/api/shuffle/ShuffleBlockInfo.java +++ b/core/src/main/java/org/apache/spark/api/shuffle/ShuffleBlockInfo.java @@ -17,8 +17,6 @@ package org.apache.spark.api.shuffle; -import org.apache.spark.api.java.Optional; - import java.util.Objects; /** diff --git a/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala b/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala index 01bc00db149dd..7e7c41933d52e 100644 --- a/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala +++ b/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala @@ -69,13 +69,13 @@ class MapOutputTrackerSuite extends SparkFunSuite { tracker.registerMapOutput(10, 1, MapStatus(BlockManagerId("b", "hostB", 1000), Array(10000L, 1000L))) val statuses = tracker.getMapSizesByShuffleLocation(10, 0) - assert(statuses.toSet === - Seq( - (Some(DefaultMapShuffleLocations.get(BlockManagerId("a", "hostA", 1000))), - ArrayBuffer((ShuffleBlockId(10, 0, 0), size1000))), - (Some(DefaultMapShuffleLocations.get(BlockManagerId("b", "hostB", 1000))), - ArrayBuffer((ShuffleBlockId(10, 1, 0), size10000)))) - .toSet) + val output = Seq( + (DefaultMapShuffleLocations.get(BlockManagerId("a", "hostA", 1000)).getLocationsForBlock(0), + ArrayBuffer((ShuffleBlockId(10, 0, 0), size1000))), + (DefaultMapShuffleLocations.get(BlockManagerId("b", "hostB", 1000)).getLocationsForBlock(0), + ArrayBuffer((ShuffleBlockId(10, 1, 0), size10000)))) + .toSet + assert(statuses.toSet === output) assert(0 == tracker.getNumCachedSerializedBroadcast) tracker.stop() rpcEnv.shutdown() @@ -157,7 +157,7 @@ class MapOutputTrackerSuite extends SparkFunSuite { slaveTracker.updateEpoch(masterTracker.getEpoch) assert(slaveTracker.getMapSizesByShuffleLocation(10, 0).toSeq === Seq( - (Some(DefaultMapShuffleLocations.get(BlockManagerId("a", "hostA", 1000))), + (DefaultMapShuffleLocations.get(BlockManagerId("a", "hostA", 1000)).getLocationsForBlock(0), ArrayBuffer((ShuffleBlockId(10, 0, 0), size1000))))) assert(0 == masterTracker.getNumCachedSerializedBroadcast) From fcb6ac745816ec5389cc1fb844cbaaa750bf3298 Mon Sep 17 00:00:00 2001 From: Yifei Huang Date: Mon, 13 May 2019 10:25:27 -0700 Subject: [PATCH 08/27] cleanup --- .../api/shuffle/MapShuffleLocations.java | 4 ++-- .../spark/api/shuffle/ShuffleLocation.java | 20 ++++++++++++++++--- .../org/apache/spark/MapOutputTracker.scala | 12 +++++++---- .../org/apache/spark/TaskEndReason.scala | 2 +- .../apache/spark/scheduler/DAGScheduler.scala | 10 +++------- .../io/DefaultShuffleReadSupport.scala | 1 + .../storage/ShuffleBlockFetcherIterator.scala | 2 +- .../apache/spark/MapOutputTrackerSuite.scala | 5 ++--- 8 files changed, 35 insertions(+), 21 deletions(-) diff --git a/core/src/main/java/org/apache/spark/api/shuffle/MapShuffleLocations.java b/core/src/main/java/org/apache/spark/api/shuffle/MapShuffleLocations.java index 1d2ceceb5bb0c..8977b3f6b3fa7 100644 --- a/core/src/main/java/org/apache/spark/api/shuffle/MapShuffleLocations.java +++ b/core/src/main/java/org/apache/spark/api/shuffle/MapShuffleLocations.java @@ -39,8 +39,8 @@ public interface MapShuffleLocations extends Serializable { ShuffleLocation[] getLocationsForBlock(int reduceId); /** - * Deletes a ShuffleLocation from this MapShuffleLocations. Returns true if - * the removal of this ShuffleLocation results in missing partitions. + * Deletes a host or a host/port combination from this MapShuffleLocations. + * Returns true if the removal of this ShuffleLocation results in missing partitions. */ boolean removeShuffleLocation(String host, Optional port); } diff --git a/core/src/main/java/org/apache/spark/api/shuffle/ShuffleLocation.java b/core/src/main/java/org/apache/spark/api/shuffle/ShuffleLocation.java index 8e0a5b726ef6f..552f2888297d3 100644 --- a/core/src/main/java/org/apache/spark/api/shuffle/ShuffleLocation.java +++ b/core/src/main/java/org/apache/spark/api/shuffle/ShuffleLocation.java @@ -24,12 +24,26 @@ * and writers are expected to cast this down to an implementation-specific representation. */ public abstract class ShuffleLocation { + /** + * The host and port on which the shuffle block is located. + */ public abstract String host(); public abstract int port(); - public abstract Optional execId(); + + /** + * The executor on which the ShuffleLocation is located. Returns {@link Optional#empty()} if + * location is not associated with an executor. + */ + public Optional execId() { + return Optional.empty(); + } @Override - public final String toString() { - return String.format("ShuffleLocation %s:%d", host(), port()); + public String toString() { + String shuffleLocation = String.format("ShuffleLocation %s:%d", host(), port()); + if (execId().isPresent()) { + return String.format("%s (execId: %s)", shuffleLocation, execId().get()); + } + return shuffleLocation; } } diff --git a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala index d1e97e1baad15..d54f5b3556081 100644 --- a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala +++ b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala @@ -106,10 +106,14 @@ private class ShuffleStatus(numPartitions: Int) { def removeMapOutput(mapId: Int, shuffleLocations: Array[ShuffleLocation]): Unit = synchronized { if (mapStatuses(mapId) != null) { var shouldDelete = false - shuffleLocations.foreach { location => - shouldDelete = mapStatuses(mapId) - .mapShuffleLocations - .removeShuffleLocation(location.host(), Optional.of(location.port())) + if (shuffleLocations == null) { + shouldDelete = true + } else { + shuffleLocations.foreach { location => + shouldDelete = mapStatuses(mapId) + .mapShuffleLocations + .removeShuffleLocation(location.host(), Optional.of(location.port())) + } } if (shouldDelete) { _numAvailableOutputs -= 1 diff --git a/core/src/main/scala/org/apache/spark/TaskEndReason.scala b/core/src/main/scala/org/apache/spark/TaskEndReason.scala index 6b8c3d72869bd..fed51b23d3ee4 100644 --- a/core/src/main/scala/org/apache/spark/TaskEndReason.scala +++ b/core/src/main/scala/org/apache/spark/TaskEndReason.scala @@ -82,7 +82,7 @@ case object Resubmitted extends TaskFailedReason { */ @DeveloperApi case class FetchFailed( - shuffleLocation: Array[ShuffleLocation], // Note that bmAddress can be null + shuffleLocation: Array[ShuffleLocation], // Note that shuffleLocation can be null shuffleId: Int, mapId: Int, reduceId: Int, 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 12162ab024c4f..96205e76d854b 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -1639,7 +1639,7 @@ private[spark] class DAGScheduler( shuffleLocations.foreach(location => { var epochAllowsRemoval = false - // if there's an executor id, remove it + // If the location belonged to an executor, remove all outputs on the executor val maybeExecId = location.execId() val currentEpoch = Some(task.epoch).getOrElse(mapOutputTracker.getEpoch) if (maybeExecId.isPresent) { @@ -1651,6 +1651,7 @@ private[spark] class DAGScheduler( mapOutputTracker.removeOutputsOnExecutor(execId) } } else { + // If the location doesn't belong to an executor, the epoch doesn't matter epochAllowsRemoval = true } @@ -1810,11 +1811,6 @@ private[spark] class DAGScheduler( maybeEpoch = None) } - private def removeShuffleLocationHosts(hosts: Array[String]): Unit = { - logInfo("Removing all outputs at hosts %s".format(hosts.toString)) - hosts.foreach(host => mapOutputTracker.removeOutputsOnHost(host)) - } - private def removeExecutorAndUnregisterOutputs( execId: String, fileLost: Boolean, @@ -1828,7 +1824,7 @@ private[spark] class DAGScheduler( if (fileLost) { hostToUnregisterOutputs match { case Some(host) => - logInfo("Shuffle files lost for hosts: %s (epoch %d)".format(host, currentEpoch)) + logInfo("Shuffle files lost for host: %s (epoch %d)".format(host, currentEpoch)) mapOutputTracker.removeOutputsOnHost(host) case None => logInfo("Shuffle files lost for executor: %s (epoch %d)".format(execId, currentEpoch)) diff --git a/core/src/main/scala/org/apache/spark/shuffle/io/DefaultShuffleReadSupport.scala b/core/src/main/scala/org/apache/spark/shuffle/io/DefaultShuffleReadSupport.scala index be0263287fe30..7a79469497d8a 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/io/DefaultShuffleReadSupport.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/io/DefaultShuffleReadSupport.scala @@ -95,6 +95,7 @@ private class ShuffleBlockFetcherIterable( blockManager, mapOutputTracker.getMapSizesByShuffleLocation(shuffleId, minReduceId, maxReduceId + 1) .map { shuffleLocationInfo => + // there should be only one copy of the shuffle data in the default implementation val defaultShuffleLocation = shuffleLocationInfo._1(0) .asInstanceOf[DefaultMapShuffleLocations] (defaultShuffleLocation.getBlockManagerId, shuffleLocationInfo._2) diff --git a/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala index f21c9bed1fe02..60385a0a55830 100644 --- a/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala +++ b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala @@ -581,7 +581,7 @@ final class ShuffleBlockFetcherIterator( blockId match { case ShuffleBlockId(shufId, mapId, reduceId) => throw new FetchFailedException( - DefaultMapShuffleLocations.get(address).getLocationsForBlock(0), + DefaultMapShuffleLocations.get(address).getLocationsForBlock(reduceId), shufId.toInt, mapId.toInt, reduceId, e) case _ => throw new SparkException( diff --git a/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala b/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala index 7e7c41933d52e..d4b593ac7db7e 100644 --- a/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala +++ b/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala @@ -69,13 +69,12 @@ class MapOutputTrackerSuite extends SparkFunSuite { tracker.registerMapOutput(10, 1, MapStatus(BlockManagerId("b", "hostB", 1000), Array(10000L, 1000L))) val statuses = tracker.getMapSizesByShuffleLocation(10, 0) - val output = Seq( + assert(statuses.toSet === Seq( (DefaultMapShuffleLocations.get(BlockManagerId("a", "hostA", 1000)).getLocationsForBlock(0), ArrayBuffer((ShuffleBlockId(10, 0, 0), size1000))), (DefaultMapShuffleLocations.get(BlockManagerId("b", "hostB", 1000)).getLocationsForBlock(0), ArrayBuffer((ShuffleBlockId(10, 1, 0), size10000)))) - .toSet - assert(statuses.toSet === output) + .toSet) assert(0 == tracker.getNumCachedSerializedBroadcast) tracker.stop() rpcEnv.shutdown() From 715a7eda3f3a8d8bde42a67e95368eefe494d9a1 Mon Sep 17 00:00:00 2001 From: Yifei Huang Date: Mon, 13 May 2019 12:09:35 -0700 Subject: [PATCH 09/27] use seq in map output tracker --- .../org/apache/spark/MapOutputTracker.scala | 16 ++++++------ .../shuffle/BlockStoreShuffleReader.scala | 2 +- .../apache/spark/MapOutputTrackerSuite.scala | 6 ++--- .../spark/scheduler/DAGSchedulerSuite.scala | 26 +++++++++++-------- 4 files changed, 27 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 d54f5b3556081..2fcd3ebbefcbb 100644 --- a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala +++ b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala @@ -303,7 +303,7 @@ private[spark] abstract class MapOutputTracker(conf: SparkConf) extends Logging // For testing def getMapSizesByShuffleLocation(shuffleId: Int, reduceId: Int) - : Iterator[(Array[ShuffleLocation], Seq[(BlockId, Long)])] = { + : Iterator[(Seq[ShuffleLocation], Seq[(BlockId, Long)])] = { getMapSizesByShuffleLocation(shuffleId, reduceId, reduceId + 1) } @@ -317,7 +317,7 @@ private[spark] abstract class MapOutputTracker(conf: SparkConf) extends Logging * describing the shuffle blocks that are stored at that block manager. */ def getMapSizesByShuffleLocation(shuffleId: Int, startPartition: Int, endPartition: Int) - : Iterator[(Array[ShuffleLocation], Seq[(BlockId, Long)])] + : Iterator[(Seq[ShuffleLocation], Seq[(BlockId, Long)])] /** * Deletes map output status information for the specified shuffle stage. @@ -667,7 +667,7 @@ private[spark] class MapOutputTrackerMaster( // Get blocks sizes by executor Id. Note that zero-sized blocks are excluded in the result. // This method is only called in local-mode. def getMapSizesByShuffleLocation(shuffleId: Int, startPartition: Int, endPartition: Int) - : Iterator[(Array[ShuffleLocation], Seq[(BlockId, Long)])] = { + : Iterator[(Seq[ShuffleLocation], Seq[(BlockId, Long)])] = { logDebug(s"Fetching outputs for shuffle $shuffleId, partitions $startPartition-$endPartition") shuffleStatuses.get(shuffleId) match { case Some (shuffleStatus) => @@ -704,7 +704,7 @@ private[spark] class MapOutputTrackerWorker(conf: SparkConf) extends MapOutputTr // Get blocks sizes by executor Id. Note that zero-sized blocks are excluded in the result. override def getMapSizesByShuffleLocation(shuffleId: Int, startPartition: Int, endPartition: Int) - : Iterator[(Array[ShuffleLocation], Seq[(BlockId, Long)])] = { + : Iterator[(Seq[ShuffleLocation], Seq[(BlockId, Long)])] = { logDebug(s"Fetching outputs for shuffle $shuffleId, partitions $startPartition-$endPartition") val statuses = getStatuses(shuffleId) try { @@ -893,9 +893,9 @@ private[spark] object MapOutputTracker extends Logging { shuffleId: Int, startPartition: Int, endPartition: Int, - statuses: Array[MapStatus]): Iterator[(Array[ShuffleLocation], Seq[(BlockId, Long)])] = { + statuses: Array[MapStatus]): Iterator[(Seq[ShuffleLocation], Seq[(BlockId, Long)])] = { assert (statuses != null) - val splitsByAddress = new HashMap[Array[ShuffleLocation], ListBuffer[(BlockId, Long)]] + val splitsByAddress = new HashMap[Seq[ShuffleLocation], ListBuffer[(BlockId, Long)]] for ((status, mapId) <- statuses.iterator.zipWithIndex) { if (status == null) { val errorMessage = s"Missing an output location for shuffle $shuffleId" @@ -907,11 +907,11 @@ private[spark] object MapOutputTracker extends Logging { if (size != 0) { if (status.mapShuffleLocations == null || status.mapShuffleLocations.getLocationsForBlock(part).isEmpty) { - splitsByAddress.getOrElseUpdate(Array.empty, ListBuffer()) += + splitsByAddress.getOrElseUpdate(Seq.empty, ListBuffer()) += ((ShuffleBlockId(shuffleId, mapId, part), size)) } else { val shuffleLocations = status.mapShuffleLocations.getLocationsForBlock(part) - splitsByAddress.getOrElseUpdate(shuffleLocations, ListBuffer()) += + splitsByAddress.getOrElseUpdate(shuffleLocations.toSeq, ListBuffer()) += ((ShuffleBlockId(shuffleId, mapId, part), size)) } } diff --git a/core/src/main/scala/org/apache/spark/shuffle/BlockStoreShuffleReader.scala b/core/src/main/scala/org/apache/spark/shuffle/BlockStoreShuffleReader.scala index 977535c140d14..316e810c9bd66 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/BlockStoreShuffleReader.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/BlockStoreShuffleReader.scala @@ -69,7 +69,7 @@ private[spark] class BlockStoreShuffleReader[K, C]( block.mapId, block.reduceId, blockInfo._2, - shuffleLocationInfo._1) + shuffleLocationInfo._1.toArray) } } } diff --git a/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala b/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala index d4b593ac7db7e..5191dacf304ae 100644 --- a/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala +++ b/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala @@ -70,9 +70,9 @@ class MapOutputTrackerSuite extends SparkFunSuite { Array(10000L, 1000L))) val statuses = tracker.getMapSizesByShuffleLocation(10, 0) assert(statuses.toSet === Seq( - (DefaultMapShuffleLocations.get(BlockManagerId("a", "hostA", 1000)).getLocationsForBlock(0), + (Seq(DefaultMapShuffleLocations.get(BlockManagerId("a", "hostA", 1000))), ArrayBuffer((ShuffleBlockId(10, 0, 0), size1000))), - (DefaultMapShuffleLocations.get(BlockManagerId("b", "hostB", 1000)).getLocationsForBlock(0), + (Seq(DefaultMapShuffleLocations.get(BlockManagerId("b", "hostB", 1000))), ArrayBuffer((ShuffleBlockId(10, 1, 0), size10000)))) .toSet) assert(0 == tracker.getNumCachedSerializedBroadcast) @@ -156,7 +156,7 @@ class MapOutputTrackerSuite extends SparkFunSuite { slaveTracker.updateEpoch(masterTracker.getEpoch) assert(slaveTracker.getMapSizesByShuffleLocation(10, 0).toSeq === Seq( - (DefaultMapShuffleLocations.get(BlockManagerId("a", "hostA", 1000)).getLocationsForBlock(0), + (Seq(DefaultMapShuffleLocations.get(BlockManagerId("a", "hostA", 1000))), ArrayBuffer((ShuffleBlockId(10, 0, 0), size1000))))) assert(0 == masterTracker.getNumCachedSerializedBroadcast) 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 f15c0cc152660..3d06e34bd6fb8 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -703,7 +703,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi (Success, makeMapStatus("hostA", 1)), (Success, makeMapStatus("hostB", 1)))) assert(mapOutputTracker.getMapSizesByShuffleLocation(shuffleId, 0).map(_._1).toSet === - HashSet(makeShuffleLocationArray("hostA"), makeShuffleLocationArray("hostB"))) + HashSet(makeShuffleLocationSeq("hostA"), makeShuffleLocationSeq("hostB"))) complete(taskSets(1), Seq((Success, 42))) assert(results === Map(0 -> 42)) assertDataStructuresEmpty() @@ -774,7 +774,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi } } else { assert(mapOutputTracker.getMapSizesByShuffleLocation(shuffleId, 0).map(_._1).toSet === - HashSet(makeShuffleLocationArray("hostA"), makeShuffleLocationArray("hostB"))) + HashSet(makeShuffleLocationSeq("hostA"), makeShuffleLocationSeq("hostB"))) } } } @@ -1398,7 +1398,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi makeMapStatus("hostA", reduceRdd.partitions.size))) assert(shuffleStage.numAvailableOutputs === 2) assert(mapOutputTracker.getMapSizesByShuffleLocation(shuffleId, 0).map(_._1).toSet === - HashSet(makeShuffleLocationArray("hostB"), makeShuffleLocationArray("hostA"))) + HashSet(makeShuffleLocationSeq("hostB"), makeShuffleLocationSeq("hostA"))) // finish the next stage normally, which completes the job complete(taskSets(1), Seq((Success, 42), (Success, 43))) @@ -1805,7 +1805,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi // have hostC complete the resubmitted task complete(taskSets(1), Seq((Success, makeMapStatus("hostC", 1)))) assert(mapOutputTracker.getMapSizesByShuffleLocation(shuffleId, 0).map(_._1).toSet === - HashSet(makeShuffleLocationArray("hostC"), makeShuffleLocationArray("hostB"))) + HashSet(makeShuffleLocationSeq("hostC"), makeShuffleLocationSeq("hostB"))) // Make sure that the reduce stage was now submitted. assert(taskSets.size === 3) @@ -2070,7 +2070,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi complete(taskSets(0), Seq( (Success, makeMapStatus("hostA", 1)))) assert(mapOutputTracker.getMapSizesByShuffleLocation(shuffleId, 0).map(_._1).toSet === - HashSet(makeShuffleLocationArray("hostA"))) + HashSet(makeShuffleLocationSeq("hostA"))) // Reducer should run on the same host that map task ran val reduceTaskSet = taskSets(1) @@ -2116,7 +2116,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi complete(taskSets(0), Seq( (Success, makeMapStatus("hostA", 1)))) assert(mapOutputTracker.getMapSizesByShuffleLocation(shuffleId, 0).map(_._1).toSet === - HashSet(makeShuffleLocationArray("hostA"))) + HashSet(makeShuffleLocationSeq("hostA"))) // Reducer should run where RDD 2 has preferences, even though it also has a shuffle dep val reduceTaskSet = taskSets(1) @@ -2280,7 +2280,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi (Success, makeMapStatus("hostA", rdd1.partitions.length)), (Success, makeMapStatus("hostB", rdd1.partitions.length)))) assert(mapOutputTracker.getMapSizesByShuffleLocation(dep1.shuffleId, 0).map(_._1).toSet === - HashSet(makeShuffleLocationArray("hostA"), makeShuffleLocationArray("hostB"))) + HashSet(makeShuffleLocationSeq("hostA"), makeShuffleLocationSeq("hostB"))) assert(listener1.results.size === 1) // When attempting the second stage, show a fetch failure @@ -2296,7 +2296,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi complete(taskSets(2), Seq( (Success, makeMapStatus("hostC", rdd2.partitions.length)))) assert(mapOutputTracker.getMapSizesByShuffleLocation(dep1.shuffleId, 0).map(_._1).toSet === - HashSet(makeShuffleLocationArray("hostC"), makeShuffleLocationArray("hostB"))) + HashSet(makeShuffleLocationSeq("hostC"), makeShuffleLocationSeq("hostB"))) assert(listener2.results.size === 0) // Second stage listener should still not have a result @@ -2306,7 +2306,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi (Success, makeMapStatus("hostB", rdd2.partitions.length)), (Success, makeMapStatus("hostD", rdd2.partitions.length)))) assert(mapOutputTracker.getMapSizesByShuffleLocation(dep2.shuffleId, 0).map(_._1).toSet === - HashSet(makeShuffleLocationArray("hostB"), makeShuffleLocationArray("hostD"))) + HashSet(makeShuffleLocationSeq("hostB"), makeShuffleLocationSeq("hostD"))) assert(listener2.results.size === 1) // Finally, the reduce job should be running as task set 4; make it see a fetch failure, @@ -2345,7 +2345,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi (Success, makeMapStatus("hostA", rdd1.partitions.length)), (Success, makeMapStatus("hostB", rdd1.partitions.length)))) assert(mapOutputTracker.getMapSizesByShuffleLocation(dep1.shuffleId, 0).map(_._1).toSet === - HashSet(makeShuffleLocationArray("hostA"), makeShuffleLocationArray("hostB"))) + HashSet(makeShuffleLocationSeq("hostA"), makeShuffleLocationSeq("hostB"))) assert(listener1.results.size === 1) // When attempting stage1, trigger a fetch failure. @@ -2371,7 +2371,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi complete(taskSets(2), Seq( (Success, makeMapStatus("hostC", rdd2.partitions.length)))) assert(mapOutputTracker.getMapSizesByShuffleLocation(dep1.shuffleId, 0).map(_._1).toSet === - Set(makeShuffleLocationArray("hostC"), makeShuffleLocationArray("hostB"))) + Set(makeShuffleLocationSeq("hostC"), makeShuffleLocationSeq("hostB"))) // After stage0 is finished, stage1 will be submitted and found there is no missing // partitions in it. Then listener got triggered. @@ -2935,6 +2935,10 @@ object DAGSchedulerSuite { def makeShuffleLocationArray(execId: String, host: String, port: Int): Array[ShuffleLocation] = { DefaultMapShuffleLocations.get(makeBlockManagerId(host)).getLocationsForBlock(0) } + + def makeShuffleLocationSeq(host: String): Seq[ShuffleLocation] = { + Seq(DefaultMapShuffleLocations.get(makeBlockManagerId(host))) + } } object FailThisAttempt { From b68254c6168166a7f2541cc8fdfa6c86dbfbb17e Mon Sep 17 00:00:00 2001 From: Yifei Huang Date: Mon, 13 May 2019 13:58:24 -0700 Subject: [PATCH 10/27] always remove for remote --- .../main/scala/org/apache/spark/scheduler/DAGScheduler.scala | 2 +- 1 file changed, 1 insertion(+), 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 96205e76d854b..706fc5a9bdf27 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -1634,7 +1634,7 @@ private[spark] class DAGScheduler( env.blockManager.externalShuffleServiceEnabled && unRegisterOutputOnHostOnFetchFailure } else { - unRegisterOutputOnHostOnFetchFailure + true // always remove for remote shuffle storage } shuffleLocations.foreach(location => { From df8d4579403abccaca202804d4b028c782c8285a Mon Sep 17 00:00:00 2001 From: Yifei Huang Date: Mon, 13 May 2019 17:57:58 -0700 Subject: [PATCH 11/27] fix some tests --- .../scala/org/apache/spark/scheduler/TaskSetManager.scala | 6 +++--- .../apache/spark/shuffle/BlockStoreShuffleReaderSuite.scala | 6 +++--- .../shuffle/sort/BlockStoreShuffleReaderBenchmark.scala | 6 +++--- 3 files changed, 9 insertions(+), 9 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 839e7674084bc..a4e85c2a7379c 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala @@ -853,9 +853,9 @@ private[spark] class TaskSetManager( // locations shouldn't be running executors, so only fetches using the default Spark // implementation (DefaultMapShuffleLocations) of fetching from executor disk should result // in blacklistable executors. - if (fetchFailed.shuffleLocation != null && - fetchFailed.shuffleLocation.isInstanceOf[Array[DefaultMapShuffleLocations]]) { - val bmAddress = fetchFailed.shuffleLocation + if (fetchFailed.shuffleLocation != null && fetchFailed.shuffleLocation.nonEmpty + && fetchFailed.shuffleLocation(0).isInstanceOf[DefaultMapShuffleLocations]) { + val bmAddress = fetchFailed.shuffleLocation(0) .asInstanceOf[DefaultMapShuffleLocations] .getBlockManagerId blacklistTracker.foreach(_.updateBlacklistForFetchFailure( diff --git a/core/src/test/scala/org/apache/spark/shuffle/BlockStoreShuffleReaderSuite.scala b/core/src/test/scala/org/apache/spark/shuffle/BlockStoreShuffleReaderSuite.scala index 98e879a8db8aa..7f4ebaaddd87b 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/BlockStoreShuffleReaderSuite.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/BlockStoreShuffleReaderSuite.scala @@ -116,13 +116,13 @@ class BlockStoreShuffleReaderSuite extends SparkFunSuite with LocalSparkContext (shuffleBlockId, byteOutputStream.size().toLong) } val blocksToRetrieve = Seq( - (Array(DefaultMapShuffleLocations.get(localBlockManagerId).asInstanceOf[ShuffleLocation]), + (Seq(DefaultMapShuffleLocations.get(localBlockManagerId).asInstanceOf[ShuffleLocation]), shuffleBlockIdsAndSizes)) val mapOutputTracker = mock(classOf[MapOutputTracker]) when(mapOutputTracker.getMapSizesByShuffleLocation(shuffleId, reduceId, reduceId + 1)) - .thenAnswer(new Answer[Iterator[(Array[ShuffleLocation], Seq[(BlockId, Long)])]] { + .thenAnswer(new Answer[Iterator[(Seq[ShuffleLocation], Seq[(BlockId, Long)])]] { def answer(invocationOnMock: InvocationOnMock): - Iterator[(Array[ShuffleLocation], Seq[(BlockId, Long)])] = { + Iterator[(Seq[ShuffleLocation], Seq[(BlockId, Long)])] = { blocksToRetrieve.iterator } }) diff --git a/core/src/test/scala/org/apache/spark/shuffle/sort/BlockStoreShuffleReaderBenchmark.scala b/core/src/test/scala/org/apache/spark/shuffle/sort/BlockStoreShuffleReaderBenchmark.scala index 4f5bb264170de..32df6633886fd 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/sort/BlockStoreShuffleReaderBenchmark.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/sort/BlockStoreShuffleReaderBenchmark.scala @@ -197,14 +197,14 @@ object BlockStoreShuffleReaderBenchmark extends BenchmarkBase { } when(mapOutputTracker.getMapSizesByShuffleLocation(0, 0, 1)) - .thenAnswer(new Answer[Iterator[(Option[ShuffleLocation], Seq[(BlockId, Long)])]] { + .thenAnswer(new Answer[Iterator[(Seq[ShuffleLocation], Seq[(BlockId, Long)])]] { def answer(invocationOnMock: InvocationOnMock): - Iterator[(Option[ShuffleLocation], Seq[(BlockId, Long)])] = { + Iterator[(Seq[ShuffleLocation], Seq[(BlockId, Long)])] = { val shuffleBlockIdsAndSizes = (0 until NUM_MAPS).map { mapId => val shuffleBlockId = ShuffleBlockId(0, mapId, 0) (shuffleBlockId, dataFileLength) } - Seq((Option.apply(DefaultMapShuffleLocations.get(dataBlockId)), shuffleBlockIdsAndSizes)) + Seq((Seq(DefaultMapShuffleLocations.get(dataBlockId)), shuffleBlockIdsAndSizes)) .toIterator } }) From 97f6ff3426629ba76e234728203f63e03c0896c3 Mon Sep 17 00:00:00 2001 From: Yifei Huang Date: Tue, 14 May 2019 11:12:23 -0700 Subject: [PATCH 12/27] json serialize --- .../org/apache/spark/util/JsonProtocol.scala | 41 ++++++++++++++++--- .../apache/spark/util/JsonProtocolSuite.scala | 6 ++- 2 files changed, 39 insertions(+), 8 deletions(-) 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 0c4c18518f854..ac1d8dc269ae1 100644 --- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -21,7 +21,6 @@ import java.util.{Properties, UUID} import scala.collection.JavaConverters._ import scala.collection.Map - import com.fasterxml.jackson.databind.ObjectMapper import com.fasterxml.jackson.module.scala.DefaultScalaModule import org.json4s.DefaultFormats @@ -36,6 +35,7 @@ import org.apache.spark.metrics.ExecutorMetricType import org.apache.spark.rdd.RDDOperationScope import org.apache.spark.scheduler._ import org.apache.spark.scheduler.cluster.ExecutorInfo +import org.apache.spark.shuffle.sort.DefaultMapShuffleLocations import org.apache.spark.storage._ /** @@ -408,8 +408,7 @@ private[spark] object JsonProtocol { val reason = Utils.getFormattedClassName(taskEndReason) val json: JObject = taskEndReason match { case fetchFailed: FetchFailed => - val blockManagerAddress = Option(fetchFailed.shuffleLocation). - map(mapper.writeValueAsString).getOrElse("None") + val blockManagerAddress = shuffleLocationsToJson(fetchFailed.shuffleLocation) ("Shuffle Locations" -> blockManagerAddress) ~ ("Shuffle ID" -> fetchFailed.shuffleId) ~ ("Map ID" -> fetchFailed.mapId) ~ @@ -440,6 +439,24 @@ private[spark] object JsonProtocol { ("Reason" -> reason) ~ json } + def shuffleLocationsToJson(shuffleLocations: Array[ShuffleLocation]): JValue = { + if (shuffleLocations != null && shuffleLocations.nonEmpty) { + if (shuffleLocations(0).isInstanceOf[DefaultMapShuffleLocations]) { + val array = JArray(shuffleLocations.map(location => { + val blockManagerId = location.asInstanceOf[DefaultMapShuffleLocations].getBlockManagerId + blockManagerIdToJson(blockManagerId) + }).toList) + ("type" -> "Default") ~ + ("data" -> array) + } else { + ("type" -> "Custom") ~ + ("data" -> mapper.writeValueAsString(shuffleLocations)) + } + } else { + "type" -> "None" + } + } + def blockManagerIdToJson(blockManagerId: BlockManagerId): JValue = { ("Executor ID" -> blockManagerId.executorId) ~ ("Host" -> blockManagerId.host) ~ @@ -949,13 +966,13 @@ private[spark] object JsonProtocol { case `success` => Success case `resubmitted` => Resubmitted case `fetchFailed` => - val locations = shuffleLocationsFromString( - (json \ "Shuffle Locations").extract[String]) + val locations = shuffleLocationsFromJson( + (json \ "Shuffle Locations")) val shuffleId = (json \ "Shuffle ID").extract[Int] val mapId = (json \ "Map ID").extract[Int] val reduceId = (json \ "Reduce ID").extract[Int] val message = jsonOption(json \ "Message").map(_.extract[String]) - new FetchFailed(locations.get, shuffleId, mapId, reduceId, + new FetchFailed(locations, shuffleId, mapId, reduceId, message.getOrElse("Unknown reason")) case `exceptionFailure` => val className = (json \ "Class Name").extract[String] @@ -998,6 +1015,18 @@ private[spark] object JsonProtocol { } } + def shuffleLocationsFromJson(json: JValue): Array[ShuffleLocation] = { + val shuffleType = (json \ "type").extract[String] + if (shuffleType == "Default") { + (json \ "data").children.map(value => { + val block = blockManagerIdFromJson(value) + DefaultMapShuffleLocations.get(block) + }).toArray + } else { + null + } + } + def shuffleLocationsFromString(string: String): Option[Array[ShuffleLocation]] = { if (string == "None") { return None 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 9d8c81ccbb2a0..9ff5f8ed1b487 100644 --- a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala @@ -289,12 +289,14 @@ class JsonProtocolSuite extends SparkFunSuite { test("FetchFailed backwards compatibility") { // FetchFailed in Spark 1.1.0 does not have a "Message" property. val fetchFailed = FetchFailed( - Array(DefaultMapShuffleLocations.get(BlockManagerId("With or", "without you", 15))), + DefaultMapShuffleLocations.get(BlockManagerId("With or", "without you", 15)) + .getLocationsForBlock(0), 17, 18, 19, "ignored") val oldEvent = JsonProtocol.taskEndReasonToJson(fetchFailed) .removeField({ _._1 == "Message" }) val expectedFetchFailed = FetchFailed( - Array(DefaultMapShuffleLocations.get(BlockManagerId("With or", "without you", 15))), + DefaultMapShuffleLocations.get(BlockManagerId("With or", "without you", 15)) + .getLocationsForBlock(0), 17, 18, 19, "Unknown reason") assert(expectedFetchFailed === JsonProtocol.taskEndReasonFromJson(oldEvent)) } From 09eeb13aa29a658ee1d39e1a032b60cd152c02f3 Mon Sep 17 00:00:00 2001 From: Yifei Huang Date: Tue, 14 May 2019 11:41:37 -0700 Subject: [PATCH 13/27] array -> seq --- .../api/shuffle/MapShuffleLocations.java | 3 +- .../sort/DefaultMapShuffleLocations.java | 9 ++- .../org/apache/spark/MapOutputTracker.scala | 6 +- .../org/apache/spark/TaskEndReason.scala | 2 +- .../spark/scheduler/TaskSetManager.scala | 4 +- .../spark/shuffle/FetchFailedException.scala | 4 +- .../storage/ShuffleBlockFetcherIterator.scala | 3 +- .../org/apache/spark/util/JsonProtocol.scala | 9 +-- .../org/apache/spark/SparkContextSuite.scala | 4 +- .../apache/spark/executor/ExecutorSuite.scala | 2 +- .../spark/scheduler/DAGSchedulerSuite.scala | 56 +++++++++---------- .../OutputCommitCoordinatorSuite.scala | 2 +- .../scheduler/SchedulerIntegrationSuite.scala | 2 +- .../spark/scheduler/TaskSetManagerSuite.scala | 4 +- .../org/apache/spark/ui/UISeleniumSuite.scala | 3 +- .../apache/spark/util/JsonProtocolSuite.scala | 8 +-- 16 files changed, 60 insertions(+), 61 deletions(-) diff --git a/core/src/main/java/org/apache/spark/api/shuffle/MapShuffleLocations.java b/core/src/main/java/org/apache/spark/api/shuffle/MapShuffleLocations.java index 8977b3f6b3fa7..87932e57a4232 100644 --- a/core/src/main/java/org/apache/spark/api/shuffle/MapShuffleLocations.java +++ b/core/src/main/java/org/apache/spark/api/shuffle/MapShuffleLocations.java @@ -20,6 +20,7 @@ import org.apache.spark.api.java.Optional; import java.io.Serializable; +import java.util.List; /** * Represents metadata about where shuffle blocks were written in a single map task. @@ -36,7 +37,7 @@ public interface MapShuffleLocations extends Serializable { /** * Get the location for a given shuffle block written by this map task. */ - ShuffleLocation[] getLocationsForBlock(int reduceId); + List getLocationsForBlock(int reduceId); /** * Deletes a host or a host/port combination from this MapShuffleLocations. diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/DefaultMapShuffleLocations.java b/core/src/main/java/org/apache/spark/shuffle/sort/DefaultMapShuffleLocations.java index 69c250e012b71..374de7a96eb1b 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/DefaultMapShuffleLocations.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/DefaultMapShuffleLocations.java @@ -21,11 +21,14 @@ import com.google.common.cache.CacheLoader; import com.google.common.cache.LoadingCache; +import com.google.common.collect.ImmutableList; import org.apache.spark.api.java.Optional; import org.apache.spark.api.shuffle.MapShuffleLocations; import org.apache.spark.api.shuffle.ShuffleLocation; import org.apache.spark.storage.BlockManagerId; +import java.util.ArrayList; +import java.util.List; import java.util.Objects; public class DefaultMapShuffleLocations extends ShuffleLocation implements MapShuffleLocations { @@ -46,11 +49,11 @@ public DefaultMapShuffleLocations load(BlockManagerId blockManagerId) { }); private final BlockManagerId location; - private final ShuffleLocation[] locationsArray; + private final List locationsArray; public DefaultMapShuffleLocations(BlockManagerId blockManagerId) { this.location = blockManagerId; - this.locationsArray = new ShuffleLocation[] {this}; + this.locationsArray = ImmutableList.of(this); } public static DefaultMapShuffleLocations get(BlockManagerId blockManagerId) { @@ -58,7 +61,7 @@ public static DefaultMapShuffleLocations get(BlockManagerId blockManagerId) { } @Override - public ShuffleLocation[] getLocationsForBlock(int reduceId) { + public List getLocationsForBlock(int reduceId) { return locationsArray; } diff --git a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala index 2fcd3ebbefcbb..f764873394935 100644 --- a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala +++ b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala @@ -103,7 +103,7 @@ private class ShuffleStatus(numPartitions: Int) { * This is a no-op if there is no registered map output or if the registered output is from a * different block manager. */ - def removeMapOutput(mapId: Int, shuffleLocations: Array[ShuffleLocation]): Unit = synchronized { + def removeMapOutput(mapId: Int, shuffleLocations: Seq[ShuffleLocation]): Unit = synchronized { if (mapStatuses(mapId) != null) { var shouldDelete = false if (shuffleLocations == null) { @@ -444,7 +444,7 @@ private[spark] class MapOutputTrackerMaster( } /** Unregister map output information of the given shuffle, mapper and block manager */ - def unregisterMapOutput(shuffleId: Int, mapId: Int, shuffleLocations: Array[ShuffleLocation]) { + def unregisterMapOutput(shuffleId: Int, mapId: Int, shuffleLocations: Seq[ShuffleLocation]) { shuffleStatuses.get(shuffleId) match { case Some(shuffleStatus) => shuffleStatus.removeMapOutput(mapId, shuffleLocations) @@ -911,7 +911,7 @@ private[spark] object MapOutputTracker extends Logging { ((ShuffleBlockId(shuffleId, mapId, part), size)) } else { val shuffleLocations = status.mapShuffleLocations.getLocationsForBlock(part) - splitsByAddress.getOrElseUpdate(shuffleLocations.toSeq, ListBuffer()) += + splitsByAddress.getOrElseUpdate(shuffleLocations.asScala, ListBuffer()) += ((ShuffleBlockId(shuffleId, mapId, part), size)) } } diff --git a/core/src/main/scala/org/apache/spark/TaskEndReason.scala b/core/src/main/scala/org/apache/spark/TaskEndReason.scala index fed51b23d3ee4..d7ba6e3db36d9 100644 --- a/core/src/main/scala/org/apache/spark/TaskEndReason.scala +++ b/core/src/main/scala/org/apache/spark/TaskEndReason.scala @@ -82,7 +82,7 @@ case object Resubmitted extends TaskFailedReason { */ @DeveloperApi case class FetchFailed( - shuffleLocation: Array[ShuffleLocation], // Note that shuffleLocation can be null + shuffleLocation: Seq[ShuffleLocation], // Note that shuffleLocation can be null shuffleId: Int, mapId: Int, reduceId: Int, 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 a4e85c2a7379c..8b6d20dc08c89 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala @@ -854,8 +854,8 @@ private[spark] class TaskSetManager( // implementation (DefaultMapShuffleLocations) of fetching from executor disk should result // in blacklistable executors. if (fetchFailed.shuffleLocation != null && fetchFailed.shuffleLocation.nonEmpty - && fetchFailed.shuffleLocation(0).isInstanceOf[DefaultMapShuffleLocations]) { - val bmAddress = fetchFailed.shuffleLocation(0) + && fetchFailed.shuffleLocation.head.isInstanceOf[DefaultMapShuffleLocations]) { + val bmAddress = fetchFailed.shuffleLocation.head .asInstanceOf[DefaultMapShuffleLocations] .getBlockManagerId blacklistTracker.foreach(_.updateBlacklistForFetchFailure( 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 f84efd10663ba..de4cb361e931d 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/FetchFailedException.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/FetchFailedException.scala @@ -33,7 +33,7 @@ import org.apache.spark.util.Utils * (or risk triggering any other exceptions). See SPARK-19276. */ private[spark] class FetchFailedException( - shuffleLocations: Array[ShuffleLocation], + shuffleLocations: Seq[ShuffleLocation], shuffleId: Int, mapId: Int, reduceId: Int, @@ -42,7 +42,7 @@ private[spark] class FetchFailedException( extends Exception(message, cause) { def this( - shuffleLocations: Array[ShuffleLocation], + shuffleLocations: Seq[ShuffleLocation], shuffleId: Int, mapId: Int, reduceId: Int, diff --git a/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala index 60385a0a55830..5ef25e4381585 100644 --- a/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala +++ b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala @@ -22,6 +22,7 @@ import java.nio.ByteBuffer import java.util.concurrent.LinkedBlockingQueue import javax.annotation.concurrent.GuardedBy +import scala.collection.JavaConverters._ import scala.collection.mutable import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet, Queue} @@ -581,7 +582,7 @@ final class ShuffleBlockFetcherIterator( blockId match { case ShuffleBlockId(shufId, mapId, reduceId) => throw new FetchFailedException( - DefaultMapShuffleLocations.get(address).getLocationsForBlock(reduceId), + DefaultMapShuffleLocations.get(address).getLocationsForBlock(reduceId).asScala, shufId.toInt, mapId.toInt, reduceId, e) case _ => throw new SparkException( 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 ac1d8dc269ae1..e534ef8938218 100644 --- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -21,6 +21,7 @@ import java.util.{Properties, UUID} import scala.collection.JavaConverters._ import scala.collection.Map + import com.fasterxml.jackson.databind.ObjectMapper import com.fasterxml.jackson.module.scala.DefaultScalaModule import org.json4s.DefaultFormats @@ -439,9 +440,9 @@ private[spark] object JsonProtocol { ("Reason" -> reason) ~ json } - def shuffleLocationsToJson(shuffleLocations: Array[ShuffleLocation]): JValue = { + def shuffleLocationsToJson(shuffleLocations: Seq[ShuffleLocation]): JValue = { if (shuffleLocations != null && shuffleLocations.nonEmpty) { - if (shuffleLocations(0).isInstanceOf[DefaultMapShuffleLocations]) { + if (shuffleLocations.head.isInstanceOf[DefaultMapShuffleLocations]) { val array = JArray(shuffleLocations.map(location => { val blockManagerId = location.asInstanceOf[DefaultMapShuffleLocations].getBlockManagerId blockManagerIdToJson(blockManagerId) @@ -1015,13 +1016,13 @@ private[spark] object JsonProtocol { } } - def shuffleLocationsFromJson(json: JValue): Array[ShuffleLocation] = { + def shuffleLocationsFromJson(json: JValue): Seq[ShuffleLocation] = { val shuffleType = (json \ "type").extract[String] if (shuffleType == "Default") { (json \ "data").children.map(value => { val block = blockManagerIdFromJson(value) DefaultMapShuffleLocations.get(block) - }).toArray + }) } else { null } diff --git a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala index 149e61e56c4dd..98371930710c7 100644 --- a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala @@ -692,8 +692,8 @@ class SparkContextSuite extends SparkFunSuite with LocalSparkContext with Eventu // Make the first task in the first stage attempt fail. throw new FetchFailedException( shuffleLocations = - Array(DefaultMapShuffleLocations.get(SparkEnv.get.blockManager.blockManagerId)), - shuffleId = 0, mapId = 0, reduceId = 0, cause = new java.io.IOException("fake")) + Seq(DefaultMapShuffleLocations.get(SparkEnv.get.blockManager.blockManagerId)), + 0, 0, 0, new java.io.IOException("fake")) } else { // Make the second task in the first stage attempt sleep to generate a zombie task Thread.sleep(60000) diff --git a/core/src/test/scala/org/apache/spark/executor/ExecutorSuite.scala b/core/src/test/scala/org/apache/spark/executor/ExecutorSuite.scala index 65621a418134c..511966a8bded3 100644 --- a/core/src/test/scala/org/apache/spark/executor/ExecutorSuite.scala +++ b/core/src/test/scala/org/apache/spark/executor/ExecutorSuite.scala @@ -439,7 +439,7 @@ class FetchFailureThrowingRDD(sc: SparkContext) extends RDD[Int](sc, Nil) { override def hasNext: Boolean = true override def next(): Int = { throw new FetchFailedException( - Array(DefaultMapShuffleLocations.get(BlockManagerId("1", "hostA", 1234))), + Seq(DefaultMapShuffleLocations.get(BlockManagerId("1", "hostA", 1234))), shuffleId = 0, mapId = 0, reduceId = 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 3d06e34bd6fb8..ffcecce3b58be 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -475,7 +475,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi // reduce stage fails with a fetch failure from one host complete(taskSets(2), Seq( - (FetchFailed(makeShuffleLocationArray("exec-hostA2", "hostA", 12345), + (FetchFailed(makeShuffleLocationSeq("exec-hostA2", "hostA", 12345), firstShuffleId, 0, 0, "ignored"), null) )) @@ -721,7 +721,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi // the 2nd ResultTask failed complete(taskSets(1), Seq( (Success, 42), - (FetchFailed(makeShuffleLocationArray("hostA"), shuffleId, 0, 0, "ignored"), null))) + (FetchFailed(makeShuffleLocationSeq("hostA"), shuffleId, 0, 0, "ignored"), null))) // this will get called // blockManagerMaster.removeExecutor("exec-hostA") // ask the scheduler to try it again @@ -840,7 +840,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi val stageAttempt = taskSets.last checkStageId(stageId, attemptIdx, stageAttempt) complete(stageAttempt, stageAttempt.tasks.zipWithIndex.map { case (task, idx) => - (FetchFailed(makeShuffleLocationArray("hostA"), + (FetchFailed(makeShuffleLocationSeq("hostA"), shuffleDep.shuffleId, 0, idx, "ignored"), null) }.toSeq) } @@ -1076,7 +1076,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi // The first result task fails, with a fetch failure for the output from the first mapper. runEvent(makeCompletionEvent( taskSets(1).tasks(0), - FetchFailed(makeShuffleLocationArray("hostA"), shuffleId, 0, 0, "ignored"), + FetchFailed(makeShuffleLocationSeq("hostA"), shuffleId, 0, 0, "ignored"), null)) sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS) assert(sparkListener.failedStages.contains(1)) @@ -1084,7 +1084,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi // The second ResultTask fails, with a fetch failure for the output from the second mapper. runEvent(makeCompletionEvent( taskSets(1).tasks(0), - FetchFailed(makeShuffleLocationArray("hostA"), shuffleId, 1, 1, "ignored"), + FetchFailed(makeShuffleLocationSeq("hostA"), shuffleId, 1, 1, "ignored"), null)) // The SparkListener should not receive redundant failure events. sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS) @@ -1105,7 +1105,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi // The first result task fails, with a fetch failure for the output from the first mapper. runEvent(makeCompletionEvent( taskSets(1).tasks(0), - FetchFailed(makeShuffleLocationArray("hostA"), shuffleId, 0, 0, "ignored"), + FetchFailed(makeShuffleLocationSeq("hostA"), shuffleId, 0, 0, "ignored"), null)) assert(mapOutputTracker.findMissingPartitions(shuffleId) === Some(Seq(0, 1))) @@ -1212,7 +1212,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi // The first result task fails, with a fetch failure for the output from the first mapper. runEvent(makeCompletionEvent( taskSets(1).tasks(0), - FetchFailed(makeShuffleLocationArray("hostA"), shuffleId, 0, 0, "ignored"), + FetchFailed(makeShuffleLocationSeq("hostA"), shuffleId, 0, 0, "ignored"), null)) sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS) assert(sparkListener.failedStages.contains(1)) @@ -1227,7 +1227,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi // The second ResultTask fails, with a fetch failure for the output from the second mapper. runEvent(makeCompletionEvent( taskSets(1).tasks(1), - FetchFailed(makeShuffleLocationArray("hostB"), shuffleId, 1, 1, "ignored"), + FetchFailed(makeShuffleLocationSeq("hostB"), shuffleId, 1, 1, "ignored"), null)) // Another ResubmitFailedStages event should not result in another attempt for the map @@ -1276,7 +1276,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi // The first result task fails, with a fetch failure for the output from the first mapper. runEvent(makeCompletionEvent( taskSets(1).tasks(0), - FetchFailed(makeShuffleLocationArray("hostA"), shuffleId, 0, 0, "ignored"), + FetchFailed(makeShuffleLocationSeq("hostA"), shuffleId, 0, 0, "ignored"), null)) // Trigger resubmission of the failed map stage and finish the re-started map task. @@ -1292,7 +1292,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi // A late FetchFailed arrives from the second task in the original reduce stage. runEvent(makeCompletionEvent( taskSets(1).tasks(1), - FetchFailed(makeShuffleLocationArray("hostB"), shuffleId, 1, 1, "ignored"), + FetchFailed(makeShuffleLocationSeq("hostB"), shuffleId, 1, 1, "ignored"), null)) // Running ResubmitFailedStages shouldn't result in any more attempts for the map stage, because @@ -1752,7 +1752,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi // lets say there is a fetch failure in this task set, which makes us go back and // run stage 0, attempt 1 complete(taskSets(1), Seq( - (FetchFailed(makeShuffleLocationArray("hostA"), + (FetchFailed(makeShuffleLocationSeq("hostA"), shuffleDep1.shuffleId, 0, 0, "ignored"), null))) scheduler.resubmitFailedStages() @@ -1834,7 +1834,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi (Success, makeMapStatus("hostC", 1)))) // fail the third stage because hostA went down complete(taskSets(2), Seq( - (FetchFailed(makeShuffleLocationArray("hostA"), + (FetchFailed(makeShuffleLocationSeq("hostA"), shuffleDepTwo.shuffleId, 0, 0, "ignored"), null))) // TODO assert this: // blockManagerMaster.removeExecutor("exec-hostA") @@ -1866,7 +1866,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi (Success, makeMapStatus("hostB", 1)))) // pretend stage 2 failed because hostA went down complete(taskSets(2), Seq( - (FetchFailed(makeShuffleLocationArray("hostA"), + (FetchFailed(makeShuffleLocationSeq("hostA"), shuffleDepTwo.shuffleId, 0, 0, "ignored"), null))) // TODO assert this: // blockManagerMaster.removeExecutor("exec-hostA") @@ -2228,7 +2228,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi submit(reduceRdd, Array(0, 1)) complete(taskSets(1), Seq( (Success, 42), - (FetchFailed(makeShuffleLocationArray("hostA"), shuffleId, 0, 0, "ignored"), null))) + (FetchFailed(makeShuffleLocationSeq("hostA"), shuffleId, 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() @@ -2287,7 +2287,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi assert(taskSets(1).stageId === 1) complete(taskSets(1), Seq( (Success, makeMapStatus("hostA", rdd2.partitions.length)), - (FetchFailed(makeShuffleLocationArray("hostA"), dep1.shuffleId, 0, 0, "ignored"), null))) + (FetchFailed(makeShuffleLocationSeq("hostA"), dep1.shuffleId, 0, 0, "ignored"), null))) scheduler.resubmitFailedStages() assert(listener2.results.size === 0) // Second stage listener should not have a result yet @@ -2314,7 +2314,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi assert(taskSets(4).stageId === 2) complete(taskSets(4), Seq( (Success, 52), - (FetchFailed(makeShuffleLocationArray("hostD"), dep2.shuffleId, 0, 0, "ignored"), null))) + (FetchFailed(makeShuffleLocationSeq("hostD"), dep2.shuffleId, 0, 0, "ignored"), null))) scheduler.resubmitFailedStages() // TaskSet 5 will rerun stage 1's lost task, then TaskSet 6 will rerun stage 2 @@ -2352,7 +2352,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi assert(taskSets(1).stageId === 1) complete(taskSets(1), Seq( (Success, makeMapStatus("hostC", rdd2.partitions.length)), - (FetchFailed(makeShuffleLocationArray("hostA"), dep1.shuffleId, 0, 0, "ignored"), null))) + (FetchFailed(makeShuffleLocationSeq("hostA"), dep1.shuffleId, 0, 0, "ignored"), null))) scheduler.resubmitFailedStages() // Stage1 listener should not have a result yet assert(listener2.results.size === 0) @@ -2487,7 +2487,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi rdd1.map { case (x, _) if (x == 1) => throw new FetchFailedException( - makeShuffleLocationArray("1", "1", 1), shuffleHandle.shuffleId, 0, 0, "test") + makeShuffleLocationSeq("1", "1", 1), shuffleHandle.shuffleId, 0, 0, "test") case (x, _) => x }.count() } @@ -2500,7 +2500,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi rdd1.map { case (x, _) if (x == 1) && FailThisAttempt._fail.getAndSet(false) => throw new FetchFailedException( - makeShuffleLocationArray("1", "1", 1), shuffleHandle.shuffleId, 0, 0, "test") + makeShuffleLocationSeq("1", "1", 1), shuffleHandle.shuffleId, 0, 0, "test") } } @@ -2554,7 +2554,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi assert(taskSets(1).stageId === 1 && taskSets(1).stageAttemptId === 0) runEvent(makeCompletionEvent( taskSets(1).tasks(0), - FetchFailed(makeShuffleLocationArray("hostA"), shuffleIdA, 0, 0, + FetchFailed(makeShuffleLocationSeq("hostA"), shuffleIdA, 0, 0, "Fetch failure of task: stageId=1, stageAttempt=0, partitionId=0"), result = null)) @@ -2742,7 +2742,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi // The first task of the final stage failed with fetch failure runEvent(makeCompletionEvent( taskSets(2).tasks(0), - FetchFailed(makeShuffleLocationArray("hostC"), shuffleId2, 0, 0, "ignored"), + FetchFailed(makeShuffleLocationSeq("hostC"), shuffleId2, 0, 0, "ignored"), null)) val failedStages = scheduler.failedStages.toSeq @@ -2761,7 +2761,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi // The first task of the `shuffleMapRdd2` failed with fetch failure runEvent(makeCompletionEvent( taskSets(3).tasks(0), - FetchFailed(makeShuffleLocationArray("hostA"), shuffleId1, 0, 0, "ignored"), + FetchFailed(makeShuffleLocationSeq("hostA"), shuffleId1, 0, 0, "ignored"), null)) // The job should fail because Spark can't rollback the shuffle map stage. @@ -2786,7 +2786,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi // Fail the second task with FetchFailed. runEvent(makeCompletionEvent( taskSets.last.tasks(1), - FetchFailed(makeShuffleLocationArray("hostA"), shuffleId, 0, 0, "ignored"), + FetchFailed(makeShuffleLocationSeq("hostA"), shuffleId, 0, 0, "ignored"), null)) // The job should fail because Spark can't rollback the result stage. @@ -2829,7 +2829,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi // Fail the second task with FetchFailed. runEvent(makeCompletionEvent( taskSets.last.tasks(1), - FetchFailed(makeShuffleLocationArray("hostA"), shuffleId, 0, 0, "ignored"), + FetchFailed(makeShuffleLocationSeq("hostA"), shuffleId, 0, 0, "ignored"), null)) assert(failure == null, "job should not fail") @@ -2928,12 +2928,8 @@ object DAGSchedulerSuite { DefaultMapShuffleLocations.get(makeBlockManagerId(host)) } - def makeShuffleLocationArray(host: String): Array[ShuffleLocation] = { - DefaultMapShuffleLocations.get(makeBlockManagerId(host)).getLocationsForBlock(0) - } - - def makeShuffleLocationArray(execId: String, host: String, port: Int): Array[ShuffleLocation] = { - DefaultMapShuffleLocations.get(makeBlockManagerId(host)).getLocationsForBlock(0) + def makeShuffleLocationSeq(execId: String, host: String, port: Int): Seq[ShuffleLocation] = { + Seq(DefaultMapShuffleLocations.get(makeBlockManagerId(host))) } def makeShuffleLocationSeq(host: String): Seq[ShuffleLocation] = { diff --git a/core/src/test/scala/org/apache/spark/scheduler/OutputCommitCoordinatorSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/OutputCommitCoordinatorSuite.scala index 262487f8befe7..a5c3b21e3e611 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/OutputCommitCoordinatorSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/OutputCommitCoordinatorSuite.scala @@ -260,7 +260,7 @@ class OutputCommitCoordinatorSuite extends SparkFunSuite with BeforeAndAfter { if (ctx.stageAttemptNumber() == 0) { throw new FetchFailedException( shuffleLocations = - Array(DefaultMapShuffleLocations.get(SparkEnv.get.blockManager.blockManagerId)), + Seq(DefaultMapShuffleLocations.get(SparkEnv.get.blockManager.blockManagerId)), shuffleId = 1, mapId = 1, reduceId = 1, cause = new Exception("Failure for test.")) } else { ctx.stageId() diff --git a/core/src/test/scala/org/apache/spark/scheduler/SchedulerIntegrationSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/SchedulerIntegrationSuite.scala index a94b677d59023..d60004b9c1f97 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/SchedulerIntegrationSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/SchedulerIntegrationSuite.scala @@ -627,7 +627,7 @@ class BasicSchedulerIntegrationSuite extends SchedulerIntegrationSuite[SingleCor backend.taskSuccess(taskDescription, DAGSchedulerSuite.makeMapStatus("hostA", 10)) case (1, 0, 0) => val fetchFailed = FetchFailed( - DAGSchedulerSuite.makeShuffleLocationArray("hostA"), shuffleId, 0, 0, "ignored") + DAGSchedulerSuite.makeShuffleLocationSeq("hostA"), shuffleId, 0, 0, "ignored") backend.taskFailed(taskDescription, fetchFailed) case (1, _, partition) => backend.taskSuccess(taskDescription, 42 + partition) 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 cfcb2a4c546f1..f9b81ef1ef3e3 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala @@ -1247,7 +1247,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg // now fail those tasks tsmSpy.handleFailedTask(taskDescs(0).taskId, TaskState.FAILED, - FetchFailed(Array(DefaultMapShuffleLocations.get( + FetchFailed(Seq(DefaultMapShuffleLocations.get( BlockManagerId(taskDescs(0).executorId, "host1", 12345))), 0, 0, 0, "ignored")) tsmSpy.handleFailedTask(taskDescs(1).taskId, TaskState.FAILED, ExecutorLostFailure(taskDescs(1).executorId, exitCausedByApp = false, reason = None)) @@ -1289,7 +1289,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg // Fail the task with fetch failure tsm.handleFailedTask(taskDescs(0).taskId, TaskState.FAILED, FetchFailed( - Array(DefaultMapShuffleLocations.get( + Seq(DefaultMapShuffleLocations.get( BlockManagerId(taskDescs(0).executorId, "host1", 12345))), 0, 0, 0, "ignored")) assert(blacklistTracker.isNodeBlacklisted("host1")) 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 dc8ca509ee8e4..339e7844bd5b0 100644 --- a/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala @@ -317,8 +317,7 @@ class UISeleniumSuite extends SparkFunSuite with WebBrowser with Matchers with B val reduceId = taskContext.partitionId() val message = "Simulated fetch failure" throw new FetchFailedException( - shuffleLocations = Array(DefaultMapShuffleLocations.get(bmAddress)), - shuffleId = shuffleId, mapId = mapId, reduceId = reduceId, message = message) + Seq(DefaultMapShuffleLocations.get(bmAddress)), shuffleId, mapId, reduceId, message) } else { x } 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 9ff5f8ed1b487..1bc27bdf45a41 100644 --- a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala @@ -171,7 +171,7 @@ class JsonProtocolSuite extends SparkFunSuite { // TaskEndReason val fetchFailed = FetchFailed( - Array(DefaultMapShuffleLocations.get(BlockManagerId("With or", "without you", 15))), + Seq(DefaultMapShuffleLocations.get(BlockManagerId("With or", "without you", 15))), 17, 18, 19, "Some exception") val fetchMetadataFailed = new MetadataFetchFailedException(17, 19, "metadata Fetch failed exception").toTaskFailedReason @@ -289,14 +289,12 @@ class JsonProtocolSuite extends SparkFunSuite { test("FetchFailed backwards compatibility") { // FetchFailed in Spark 1.1.0 does not have a "Message" property. val fetchFailed = FetchFailed( - DefaultMapShuffleLocations.get(BlockManagerId("With or", "without you", 15)) - .getLocationsForBlock(0), + Seq(DefaultMapShuffleLocations.get(BlockManagerId("With or", "without you", 15))), 17, 18, 19, "ignored") val oldEvent = JsonProtocol.taskEndReasonToJson(fetchFailed) .removeField({ _._1 == "Message" }) val expectedFetchFailed = FetchFailed( - DefaultMapShuffleLocations.get(BlockManagerId("With or", "without you", 15)) - .getLocationsForBlock(0), + Seq(DefaultMapShuffleLocations.get(BlockManagerId("With or", "without you", 15))), 17, 18, 19, "Unknown reason") assert(expectedFetchFailed === JsonProtocol.taskEndReasonFromJson(oldEvent)) } From ba78f9af7df47a5efff060f15054f6d669c6550d Mon Sep 17 00:00:00 2001 From: Yifei Huang Date: Tue, 14 May 2019 15:04:24 -0700 Subject: [PATCH 14/27] fix serialization --- .../spark/shuffle/sort/DefaultMapShuffleLocations.java | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/DefaultMapShuffleLocations.java b/core/src/main/java/org/apache/spark/shuffle/sort/DefaultMapShuffleLocations.java index 374de7a96eb1b..4f076cd802f29 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/DefaultMapShuffleLocations.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/DefaultMapShuffleLocations.java @@ -17,17 +17,17 @@ package org.apache.spark.shuffle.sort; +import com.fasterxml.jackson.annotation.JsonIgnore; import com.google.common.cache.CacheBuilder; import com.google.common.cache.CacheLoader; import com.google.common.cache.LoadingCache; -import com.google.common.collect.ImmutableList; +import com.google.common.collect.Lists; import org.apache.spark.api.java.Optional; import org.apache.spark.api.shuffle.MapShuffleLocations; import org.apache.spark.api.shuffle.ShuffleLocation; import org.apache.spark.storage.BlockManagerId; -import java.util.ArrayList; import java.util.List; import java.util.Objects; @@ -49,11 +49,12 @@ public DefaultMapShuffleLocations load(BlockManagerId blockManagerId) { }); private final BlockManagerId location; + @JsonIgnore private final List locationsArray; public DefaultMapShuffleLocations(BlockManagerId blockManagerId) { this.location = blockManagerId; - this.locationsArray = ImmutableList.of(this); + this.locationsArray = Lists.newArrayList(this); } public static DefaultMapShuffleLocations get(BlockManagerId blockManagerId) { From f50a3093dbb41311f7a24ea3795afac17fe143ed Mon Sep 17 00:00:00 2001 From: Yifei Huang Date: Tue, 14 May 2019 15:47:53 -0700 Subject: [PATCH 15/27] fix javadoc --- .../apache/spark/api/shuffle/MapShuffleLocations.java | 11 +++++++++-- 1 file changed, 9 insertions(+), 2 deletions(-) diff --git a/core/src/main/java/org/apache/spark/api/shuffle/MapShuffleLocations.java b/core/src/main/java/org/apache/spark/api/shuffle/MapShuffleLocations.java index 87932e57a4232..3ac5c78203eb9 100644 --- a/core/src/main/java/org/apache/spark/api/shuffle/MapShuffleLocations.java +++ b/core/src/main/java/org/apache/spark/api/shuffle/MapShuffleLocations.java @@ -40,8 +40,15 @@ public interface MapShuffleLocations extends Serializable { List getLocationsForBlock(int reduceId); /** - * Deletes a host or a host/port combination from this MapShuffleLocations. - * Returns true if the removal of this ShuffleLocation results in missing partitions. + * Mark a location for a block in this map output as unreachable, and thus partitions can no + * longer be fetched from that location. + *

+ * This is called by the scheduler when it detects that a block could not be fetched from the + * file server located at this host and port. + *

+ * This should return true if there exists a data loss from the removal of this shuffle + * location. Otherwise, if all partitions can still be fetched from alternative locations, + * this should return false. */ boolean removeShuffleLocation(String host, Optional port); } From b7c1e2a3031f9489fbb9ab1c010074ddfd222c30 Mon Sep 17 00:00:00 2001 From: Yifei Huang Date: Wed, 15 May 2019 15:44:43 -0700 Subject: [PATCH 16/27] write async scheduler tests --- .../apache/spark/scheduler/DAGScheduler.scala | 2 +- .../scheduler/DAGSchedulerAsyncSuite.scala | 213 ++++++++++++++++++ .../spark/scheduler/DAGSchedulerSuite.scala | 12 +- 3 files changed, 220 insertions(+), 7 deletions(-) create mode 100644 core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerAsyncSuite.scala 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 706fc5a9bdf27..96205e76d854b 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -1634,7 +1634,7 @@ private[spark] class DAGScheduler( env.blockManager.externalShuffleServiceEnabled && unRegisterOutputOnHostOnFetchFailure } else { - true // always remove for remote shuffle storage + unRegisterOutputOnHostOnFetchFailure } shuffleLocations.foreach(location => { diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerAsyncSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerAsyncSuite.scala new file mode 100644 index 0000000000000..bb21493e4aaf5 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerAsyncSuite.scala @@ -0,0 +1,213 @@ +/* + * 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 +import java.util.function.Predicate + +import scala.collection.mutable.{HashSet, Map} +import com.google.common.collect.Lists +import scala.collection.mutable + +import org.apache.spark.{FetchFailed, HashPartitioner, ShuffleDependency, SparkConf, Success} +import org.apache.spark.api.java.Optional +import org.apache.spark.api.shuffle.{MapShuffleLocations, ShuffleLocation} +import org.apache.spark.internal.config +import org.apache.spark.rdd.RDD +import org.apache.spark.storage.BlockManagerId + +class DAGSchedulerAsyncSuite extends DAGSchedulerSuite { + + class AsyncShuffleLocation(hostname: String, portInt: Int, exec: String) extends ShuffleLocation { + override def host(): String = hostname + override def port(): Int = portInt + override def execId(): Optional[String] = Optional.of(exec) + } + + class DFSShuffleLocation extends ShuffleLocation { + override def host(): String = "hdfs" + override def port(): Int = 1234 + } + + val dfsLocation = new DFSShuffleLocation + + class AsyncMapShuffleLocation(asyncLocation: AsyncShuffleLocation) + extends MapShuffleLocations { + val locations = Lists.newArrayList(asyncLocation, dfsLocation) + + override def getLocationsForBlock(reduceId: Int): util.List[ShuffleLocation] = + locations + + override def removeShuffleLocation(host: String, port: Optional[Integer]): Boolean = { + var missingPartition = false + locations.removeIf(new Predicate[ShuffleLocation] { + override def test(loc: ShuffleLocation): Boolean = + loc.host() === host && (!port.isPresent || loc.port() === port.get()) + }) + if (locations.isEmpty) { + missingPartition = true + } + missingPartition + } + + } + + private def setupTest(): (RDD[_], Int) = { + afterEach() + val conf = new SparkConf() + // unregistering all outputs on a host is disabled for async case + conf.set(config.UNREGISTER_OUTPUT_ON_HOST_ON_FETCH_FAILURE, false) + init(conf) + val shuffleMapRdd = new MyRDD(sc, 2, Nil) + val shuffleDep = new ShuffleDependency(shuffleMapRdd, new HashPartitioner(2)) + val shuffleId = shuffleDep.shuffleId + val reduceRdd = new MyRDD(sc, 2, List(shuffleDep), tracker = mapOutputTracker) + (reduceRdd, shuffleId) + } + + test("Test async simple shuffle success") { + val (reduceRdd, shuffleId) = setupTest() + submit(reduceRdd, Array(0, 1)) + + // Perform map task + val mapStatus1 = makeAsyncMapStatus("hostA", reduceRdd.partitions.length) + val mapStatus2 = makeAsyncMapStatus("hostB", reduceRdd.partitions.length) + complete(taskSets(0), Seq((Success, mapStatus1), (Success, mapStatus2))) + assertMapOutputTrackerContains(shuffleId, Seq(mapStatus1, mapStatus2)) + + // perform reduce task + complete(taskSets(1), Seq((Success, 42), (Success, 43))) + assert(results === Map(0 -> 42, 1 -> 43)) + assertDataStructuresEmpty() + } + + test("test async fetch failed - different host") { + val (reduceRdd, shuffleId) = setupTest() + submit(reduceRdd, Array(0, 1)) + + // Perform map task + val mapStatus1 = makeAsyncMapStatus("hostA", reduceRdd.partitions.length) + val mapStatus2 = makeAsyncMapStatus("hostB", reduceRdd.partitions.length) + complete(taskSets(0), Seq((Success, mapStatus1), (Success, mapStatus2))) + + // The 2nd ResultTask reduce task failed. This will remove that shuffle location, + // but the other shuffle block is still available + complete(taskSets(1), Seq( + (Success, 42), + (FetchFailed(Seq(makeAsyncShuffleLocation("hostA"), dfsLocation), + shuffleId, 0, 0, "ignored"), null))) + assert(scheduler.failedStages.size > 0) + assert(mapOutputTracker.getNumAvailableOutputs(shuffleId) == 1) + assertMapOutputTrackerContains(shuffleId, Seq(null, mapStatus2)) + + // submit the mapper once more + scheduler.resubmitFailedStages() + complete(taskSets(2), Seq((Success, mapStatus1))) + assertMapOutputTrackerContains(shuffleId, Seq(mapStatus1, mapStatus2)) + + // submit last reduce task + complete(taskSets(3), Seq((Success, 43))) + assert(results === Map(0 -> 42, 1 -> 43)) + assertDataStructuresEmpty() + } + + test("test async fetch failed - same execId") { + val (reduceRdd, shuffleId) = setupTest() + submit(reduceRdd, Array(0, 1)) + + val mapStatus = makeAsyncMapStatus("hostA", reduceRdd.partitions.length) + complete(taskSets(0), Seq((Success, mapStatus), (Success, mapStatus))) + + // the 2nd ResultTask failed. This removes both results because mappers + // are on the same execId + complete(taskSets(1), Seq( + (Success, 42), + (FetchFailed(Seq(makeAsyncShuffleLocation("hostA"), dfsLocation), + shuffleId, 0, 0, "ignored"), null))) + assert(scheduler.failedStages.size > 0) + assert(mapOutputTracker.getNumAvailableOutputs(shuffleId) == 0) + assertMapOutputTrackerContains(shuffleId, Seq(null, null)) + + // submit both mappers once more + scheduler.resubmitFailedStages() + complete(taskSets(2), Seq((Success, mapStatus), (Success, mapStatus))) + assert(mapOutputTracker.getNumAvailableOutputs(shuffleId) == 2) + assertMapOutputTrackerContains(shuffleId, Seq(mapStatus, mapStatus)) + + // submit last reduce task + complete(taskSets(3), Seq((Success, 43))) + assert(results === Map(0 -> 42, 1 -> 43)) + assertDataStructuresEmpty() + } + + test("test async fetch failed - same host but different execId") { + val (reduceRdd, shuffleId) = setupTest() + submit(reduceRdd, Array(0, 1)) + + val mapStatus1 = makeAsyncMapStatus("hostA", reduceRdd.partitions.length) + val mapStatus2 = MapStatus( + BlockManagerId("other-exec", "hostA", 1234), + new AsyncMapShuffleLocation(new AsyncShuffleLocation("hostA", 1234, "other-exec")), + Array.fill[Long](reduceRdd.partitions.length)(2) + ) + complete(taskSets(0), Seq((Success, mapStatus1), (Success, mapStatus2))) + + // the 2nd ResultTask failed. This only removes the first shuffle location because + // the second location was written by a different executor + complete(taskSets(1), Seq( + (Success, 42), + (FetchFailed(Seq(makeAsyncShuffleLocation("hostA"), dfsLocation), + shuffleId, 0, 0, "ignored"), null))) + assert(scheduler.failedStages.size > 0) + assert(mapOutputTracker.getNumAvailableOutputs(shuffleId) == 1) + assertMapOutputTrackerContains(shuffleId, Seq(null, mapStatus2)) + + // submit the one mapper again + scheduler.resubmitFailedStages() + complete(taskSets(2), Seq((Success, mapStatus1))) + assert(mapOutputTracker.getNumAvailableOutputs(shuffleId) == 2) + assertMapOutputTrackerContains(shuffleId, Seq(mapStatus1, mapStatus2)) + + // submit last reduce task + complete(taskSets(3), Seq((Success, 43))) + assert(results === Map(0 -> 42, 1 -> 43)) + assertDataStructuresEmpty() + } + + def assertMapOutputTrackerContains( + shuffleId: Int, + set: Seq[MapStatus]): Unit = { + val actualShuffleLocations = mapOutputTracker.shuffleStatuses(shuffleId).mapStatuses + assert(set === actualShuffleLocations.toSeq) + } + + def makeAsyncMapStatus(host: String, reduces: Int, execId: Optional[String] = Optional.empty(), + sizes: Byte = 2): MapStatus = { + MapStatus(makeBlockManagerId(host), + new AsyncMapShuffleLocation(makeAsyncShuffleLocation(host)), + Array.fill[Long](reduces)(sizes)) + } + + def makeAsyncShuffleLocation(host: String): AsyncShuffleLocation = { + new AsyncShuffleLocation(host, 12345, "exec-" + host) + } + + def makeBlockManagerId(host: String): BlockManagerId = + BlockManagerId("exec-" + host, host, 12345) + +} 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 ffcecce3b58be..cd70fe580199d 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -238,7 +238,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi init(new SparkConf()) } - private def init(testConf: SparkConf): Unit = { + def init(testConf: SparkConf): Unit = { sc = new SparkContext("local[2]", "DAGSchedulerSuite", testConf) sparkListener.submittedStageInfos.clear() sparkListener.successfulStages.clear() @@ -308,7 +308,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi it.next.asInstanceOf[Tuple2[_, _]]._1 /** Send the given CompletionEvent messages for the tasks in the TaskSet. */ - private def complete(taskSet: TaskSet, results: Seq[(TaskEndReason, Any)]) { + def complete(taskSet: TaskSet, results: Seq[(TaskEndReason, Any)]) { assert(taskSet.tasks.size >= results.size) for ((result, i) <- results.zipWithIndex) { if (i < taskSet.tasks.size) { @@ -334,7 +334,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi } /** Submits a job to the scheduler and returns the job id. */ - private def submit( + def submit( rdd: RDD[_], partitions: Array[Int], func: (TaskContext, Iterator[_]) => _ = jobComputeFunc, @@ -430,8 +430,8 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi // reset the test context with the right shuffle service config afterEach() val conf = new SparkConf() - conf.set(config.SHUFFLE_SERVICE_ENABLED.key, "true") conf.set("spark.files.fetchFailure.unRegisterOutputOnHost", "true") + conf.set(config.SHUFFLE_SERVICE_ENABLED.key, "true") init(conf) runEvent(ExecutorAdded("exec-hostA1", "hostA")) runEvent(ExecutorAdded("exec-hostA2", "hostA")) @@ -731,7 +731,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi // we can see both result blocks now assert(mapOutputTracker .getMapSizesByShuffleLocation(shuffleId, 0) - .map(_._1(0).asInstanceOf[DefaultMapShuffleLocations].getBlockManagerId.host) + .map(_._1(0).host) .toSet === HashSet("hostA", "hostB")) complete(taskSets(3), Seq((Success, 43))) assert(results === Map(0 -> 42, 1 -> 43)) @@ -2875,7 +2875,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi } } - private def assertDataStructuresEmpty(): Unit = { + def assertDataStructuresEmpty(): Unit = { assert(scheduler.activeJobs.isEmpty) assert(scheduler.failedStages.isEmpty) assert(scheduler.jobIdToActiveJob.isEmpty) From de824d4ba7421fa9fb11c75d3f0478ceaf17a4c7 Mon Sep 17 00:00:00 2001 From: Yifei Huang Date: Wed, 15 May 2019 17:01:55 -0700 Subject: [PATCH 17/27] fix the async location tests --- .../api/shuffle/MapShuffleLocations.java | 12 ++ .../sort/DefaultMapShuffleLocations.java | 5 + .../org/apache/spark/MapOutputTracker.scala | 9 +- .../scheduler/DAGSchedulerAsyncSuite.scala | 94 ++++++++----- .../DAGSchedulerFileServerSuite.scala | 127 ++++++++++++++++++ 5 files changed, 214 insertions(+), 33 deletions(-) create mode 100644 core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerFileServerSuite.scala diff --git a/core/src/main/java/org/apache/spark/api/shuffle/MapShuffleLocations.java b/core/src/main/java/org/apache/spark/api/shuffle/MapShuffleLocations.java index 3ac5c78203eb9..dd0014cebd787 100644 --- a/core/src/main/java/org/apache/spark/api/shuffle/MapShuffleLocations.java +++ b/core/src/main/java/org/apache/spark/api/shuffle/MapShuffleLocations.java @@ -51,4 +51,16 @@ public interface MapShuffleLocations extends Serializable { * this should return false. */ boolean removeShuffleLocation(String host, Optional port); + + /** + * Mark all locations within this MapShuffleLocations with this execId as unreachable. + *

+ * This is called by the scheduler when it detects that an executor cannot be reached to + * fetch file data. + *

+ * This should return true if there exists a data loss from the removal of shuffle locations + * with this execId. Otherwise, if all partitions can still be fetched form alternative locaitons, + * this should return false. + */ + boolean removeShuffleLocation(String executorId); } diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/DefaultMapShuffleLocations.java b/core/src/main/java/org/apache/spark/shuffle/sort/DefaultMapShuffleLocations.java index 4f076cd802f29..1e171a903323d 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/DefaultMapShuffleLocations.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/DefaultMapShuffleLocations.java @@ -74,6 +74,11 @@ public boolean removeShuffleLocation(String host, Optional port) { return this.host().equals(host); } + @Override + public boolean removeShuffleLocation(String executorId) { + return location.executorId().equals(executorId); + } + public BlockManagerId getBlockManagerId() { return location; } diff --git a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala index f764873394935..da5b4a5466a95 100644 --- a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala +++ b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala @@ -144,7 +144,14 @@ private class ShuffleStatus(numPartitions: Int) { * still registered with that execId. */ def removeOutputsOnExecutor(execId: String): Unit = synchronized { - removeOutputsByFilter(x => x.executorId == execId) + for (mapId <- 0 until mapStatuses.length) { + if (mapStatuses(mapId) != null && + mapStatuses(mapId).mapShuffleLocations.removeShuffleLocation(execId)) { + _numAvailableOutputs -= 1 + mapStatuses(mapId) = null + invalidateSerializedMapOutputStatusCache() + } + } } /** diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerAsyncSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerAsyncSuite.scala index bb21493e4aaf5..2e2e2de04572a 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerAsyncSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerAsyncSuite.scala @@ -20,9 +20,9 @@ package org.apache.spark.scheduler import java.util import java.util.function.Predicate -import scala.collection.mutable.{HashSet, Map} +import scala.collection.mutable.Map + import com.google.common.collect.Lists -import scala.collection.mutable import org.apache.spark.{FetchFailed, HashPartitioner, ShuffleDependency, SparkConf, Success} import org.apache.spark.api.java.Optional @@ -46,25 +46,42 @@ class DAGSchedulerAsyncSuite extends DAGSchedulerSuite { val dfsLocation = new DFSShuffleLocation - class AsyncMapShuffleLocation(asyncLocation: AsyncShuffleLocation) + class AsyncMapShuffleLocations(asyncLocation: AsyncShuffleLocation) extends MapShuffleLocations { - val locations = Lists.newArrayList(asyncLocation, dfsLocation) + val locations : util.List[ShuffleLocation] = if (asyncLocation == null) { + Lists.newArrayList(dfsLocation) + } else { + Lists.newArrayList(asyncLocation, dfsLocation) + } override def getLocationsForBlock(reduceId: Int): util.List[ShuffleLocation] = locations override def removeShuffleLocation(host: String, port: Optional[Integer]): Boolean = { - var missingPartition = false - locations.removeIf(new Predicate[ShuffleLocation] { + removeIfPredicate(new Predicate[ShuffleLocation] { override def test(loc: ShuffleLocation): Boolean = loc.host() === host && (!port.isPresent || loc.port() === port.get()) }) + } + + override def removeShuffleLocation(executorId: String): Boolean = { + removeIfPredicate(new Predicate[ShuffleLocation] { + override def test(loc: ShuffleLocation): Boolean = + locations.get(0).execId().get().equals(executorId) + }) + } + + def removeIfPredicate(predicate: Predicate[ShuffleLocation]): Boolean = { + var missingPartition = false + if (locations.isEmpty) { + return missingPartition + } + locations.removeIf(predicate) if (locations.isEmpty) { missingPartition = true } missingPartition } - } private def setupTest(): (RDD[_], Int) = { @@ -85,10 +102,11 @@ class DAGSchedulerAsyncSuite extends DAGSchedulerSuite { submit(reduceRdd, Array(0, 1)) // Perform map task - val mapStatus1 = makeAsyncMapStatus("hostA", reduceRdd.partitions.length) - val mapStatus2 = makeAsyncMapStatus("hostB", reduceRdd.partitions.length) + val mapStatus1 = makeAsyncMapStatus("hostA") + val mapStatus2 = makeAsyncMapStatus("hostB") complete(taskSets(0), Seq((Success, mapStatus1), (Success, mapStatus2))) - assertMapOutputTrackerContains(shuffleId, Seq(mapStatus1, mapStatus2)) + assertMapOutputTrackerContains(shuffleId, Seq( + mapStatus1.mapShuffleLocations, mapStatus2.mapShuffleLocations)) // perform reduce task complete(taskSets(1), Seq((Success, 42), (Success, 43))) @@ -96,13 +114,13 @@ class DAGSchedulerAsyncSuite extends DAGSchedulerSuite { assertDataStructuresEmpty() } - test("test async fetch failed - different host") { + test("test async fetch failed - different hosts") { val (reduceRdd, shuffleId) = setupTest() submit(reduceRdd, Array(0, 1)) // Perform map task - val mapStatus1 = makeAsyncMapStatus("hostA", reduceRdd.partitions.length) - val mapStatus2 = makeAsyncMapStatus("hostB", reduceRdd.partitions.length) + val mapStatus1 = makeAsyncMapStatus("hostA") + val mapStatus2 = makeAsyncMapStatus("hostB") complete(taskSets(0), Seq((Success, mapStatus1), (Success, mapStatus2))) // The 2nd ResultTask reduce task failed. This will remove that shuffle location, @@ -113,12 +131,13 @@ class DAGSchedulerAsyncSuite extends DAGSchedulerSuite { shuffleId, 0, 0, "ignored"), null))) assert(scheduler.failedStages.size > 0) assert(mapOutputTracker.getNumAvailableOutputs(shuffleId) == 1) - assertMapOutputTrackerContains(shuffleId, Seq(null, mapStatus2)) + assertMapOutputTrackerContains(shuffleId, Seq(null, mapStatus2.mapShuffleLocations)) // submit the mapper once more scheduler.resubmitFailedStages() complete(taskSets(2), Seq((Success, mapStatus1))) - assertMapOutputTrackerContains(shuffleId, Seq(mapStatus1, mapStatus2)) + assertMapOutputTrackerContains(shuffleId, + Seq(mapStatus1.mapShuffleLocations, mapStatus2.mapShuffleLocations)) // submit last reduce task complete(taskSets(3), Seq((Success, 43))) @@ -126,28 +145,29 @@ class DAGSchedulerAsyncSuite extends DAGSchedulerSuite { assertDataStructuresEmpty() } - test("test async fetch failed - same execId") { + test("test async fetch failed - same host, same exec") { val (reduceRdd, shuffleId) = setupTest() submit(reduceRdd, Array(0, 1)) - val mapStatus = makeAsyncMapStatus("hostA", reduceRdd.partitions.length) + val mapStatus = makeAsyncMapStatus("hostA") complete(taskSets(0), Seq((Success, mapStatus), (Success, mapStatus))) - // the 2nd ResultTask failed. This removes both results because mappers - // are on the same execId + // the 2nd ResultTask failed. This removes the first executor, but the + // other task is still intact because it was uploaded to the remove dfs complete(taskSets(1), Seq( (Success, 42), (FetchFailed(Seq(makeAsyncShuffleLocation("hostA"), dfsLocation), shuffleId, 0, 0, "ignored"), null))) assert(scheduler.failedStages.size > 0) - assert(mapOutputTracker.getNumAvailableOutputs(shuffleId) == 0) - assertMapOutputTrackerContains(shuffleId, Seq(null, null)) + assert(mapOutputTracker.getNumAvailableOutputs(shuffleId) == 1) + assertMapOutputTrackerContains(shuffleId, Seq(null, new AsyncMapShuffleLocations(null))) // submit both mappers once more scheduler.resubmitFailedStages() - complete(taskSets(2), Seq((Success, mapStatus), (Success, mapStatus))) + complete(taskSets(2), Seq((Success, mapStatus))) assert(mapOutputTracker.getNumAvailableOutputs(shuffleId) == 2) - assertMapOutputTrackerContains(shuffleId, Seq(mapStatus, mapStatus)) + assertMapOutputTrackerContains(shuffleId, Seq( + mapStatus.mapShuffleLocations, mapStatus.mapShuffleLocations)) // submit last reduce task complete(taskSets(3), Seq((Success, 43))) @@ -159,11 +179,11 @@ class DAGSchedulerAsyncSuite extends DAGSchedulerSuite { val (reduceRdd, shuffleId) = setupTest() submit(reduceRdd, Array(0, 1)) - val mapStatus1 = makeAsyncMapStatus("hostA", reduceRdd.partitions.length) + val mapStatus1 = makeAsyncMapStatus("hostA") val mapStatus2 = MapStatus( BlockManagerId("other-exec", "hostA", 1234), - new AsyncMapShuffleLocation(new AsyncShuffleLocation("hostA", 1234, "other-exec")), - Array.fill[Long](reduceRdd.partitions.length)(2) + new AsyncMapShuffleLocations(new AsyncShuffleLocation("hostA", 1234, "other-exec")), + Array.fill[Long](2)(2) ) complete(taskSets(0), Seq((Success, mapStatus1), (Success, mapStatus2))) @@ -175,13 +195,14 @@ class DAGSchedulerAsyncSuite extends DAGSchedulerSuite { shuffleId, 0, 0, "ignored"), null))) assert(scheduler.failedStages.size > 0) assert(mapOutputTracker.getNumAvailableOutputs(shuffleId) == 1) - assertMapOutputTrackerContains(shuffleId, Seq(null, mapStatus2)) + assertMapOutputTrackerContains(shuffleId, Seq(null, mapStatus2.mapShuffleLocations)) // submit the one mapper again scheduler.resubmitFailedStages() complete(taskSets(2), Seq((Success, mapStatus1))) assert(mapOutputTracker.getNumAvailableOutputs(shuffleId) == 2) - assertMapOutputTrackerContains(shuffleId, Seq(mapStatus1, mapStatus2)) + assertMapOutputTrackerContains(shuffleId, + Seq(mapStatus1.mapShuffleLocations, mapStatus2.mapShuffleLocations)) // submit last reduce task complete(taskSets(3), Seq((Success, 43))) @@ -191,15 +212,24 @@ class DAGSchedulerAsyncSuite extends DAGSchedulerSuite { def assertMapOutputTrackerContains( shuffleId: Int, - set: Seq[MapStatus]): Unit = { + set: Seq[MapShuffleLocations]): Unit = { val actualShuffleLocations = mapOutputTracker.shuffleStatuses(shuffleId).mapStatuses + .map(mapStatus => { + if (mapStatus == null) { + return null + } + mapStatus.mapShuffleLocations + }) assert(set === actualShuffleLocations.toSeq) } - def makeAsyncMapStatus(host: String, reduces: Int, execId: Optional[String] = Optional.empty(), - sizes: Byte = 2): MapStatus = { + def makeAsyncMapStatus( + host: String, + reduces: Int = 2, + execId: Optional[String] = Optional.empty(), + sizes: Byte = 2): MapStatus = { MapStatus(makeBlockManagerId(host), - new AsyncMapShuffleLocation(makeAsyncShuffleLocation(host)), + new AsyncMapShuffleLocations(makeAsyncShuffleLocation(host)), Array.fill[Long](reduces)(sizes)) } diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerFileServerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerFileServerSuite.scala new file mode 100644 index 0000000000000..37ec23a959fff --- /dev/null +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerFileServerSuite.scala @@ -0,0 +1,127 @@ +/* + * 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 +import java.util.function.Predicate + +import com.google.common.collect.Lists +import scala.collection.mutable.Map + +import org.apache.spark.{HashPartitioner, ShuffleDependency, SparkConf, Success} +import org.apache.spark.api.java.Optional +import org.apache.spark.api.shuffle.{MapShuffleLocations, ShuffleLocation} +import org.apache.spark.internal.config +import org.apache.spark.rdd.RDD +import org.apache.spark.storage.BlockManagerId + +class DAGSchedulerFileServerSuite extends DAGSchedulerSuite { + + class FileServerShuffleLocation(hostname: String, portInt: Int) extends ShuffleLocation { + override def host(): String = hostname + override def port(): Int = portInt + } + + class FileServerMapShuffleLocations(mapShuffleLocations: Seq[util.List[ShuffleLocation]]) + extends MapShuffleLocations { + override def getLocationsForBlock(reduceId: Int): util.List[ShuffleLocation] = + mapShuffleLocations(reduceId) + + override def removeShuffleLocation(host: String, port: Optional[Integer]): Boolean = { + var missingPartition = false + for (locations <- mapShuffleLocations) { + locations.removeIf(new Predicate[ShuffleLocation] { + override def test(loc: ShuffleLocation): Boolean = { + loc.host() === host && (!port.isPresent || loc.port() === port.get()) + } + }) + if (locations.isEmpty) { + missingPartition = true + } + } + missingPartition + } + + override def removeShuffleLocation(executorId: String): Boolean = { + return false + } + } + + private def setupTest(): (RDD[_], Int) = { + afterEach() + val conf = new SparkConf() + // unregistering all outputs on a host is enabled for the individual file server case + conf.set(config.UNREGISTER_OUTPUT_ON_HOST_ON_FETCH_FAILURE, true) + init(conf) + val shuffleMapRdd = new MyRDD(sc, 2, Nil) + val shuffleDep = new ShuffleDependency(shuffleMapRdd, new HashPartitioner(2)) + val shuffleId = shuffleDep.shuffleId + val reduceRdd = new MyRDD(sc, 2, List(shuffleDep), tracker = mapOutputTracker) + (reduceRdd, shuffleId) + } + + test("Test simple shuffle success") { + val (reduceRdd, shuffleId) = setupTest() + submit(reduceRdd, Array(0, 1)) + + // Perform map task + val mapStatus1 = makeFileServerMapStatus("hostA", "hostB", "hostC", "hostD") + val mapStatus2 = makeFileServerMapStatus("hostA", "hostB", "hostC", "hostE") + complete(taskSets(0), Seq((Success, mapStatus1), (Success, mapStatus2))) + assertMapOutputTrackerContains(shuffleId, Seq(mapStatus1, mapStatus2)) + + // perform reduce task + complete(taskSets(1), Seq((Success, 42), (Success, 43))) + assert(results === Map(0 -> 42, 1 -> 43)) + assertDataStructuresEmpty() + } + + + def assertMapOutputTrackerContains( + shuffleId: Int, + set: Seq[MapStatus]): Unit = { + val actualShuffleLocations = mapOutputTracker.shuffleStatuses(shuffleId).mapStatuses + assert(set === actualShuffleLocations.toSeq) + } + + def makeFileServerMapStatus( + partition1Primary: String, + partition1Secondary: String, + partition2Primary: String, + partition2Secondary: String): MapStatus = { + val partition1List: util.List[ShuffleLocation] = Lists.newArrayList( + new FileServerShuffleLocation(partition1Primary, 1234), + new FileServerShuffleLocation(partition2Secondary, 1234)) + val partition2List: util.List[ShuffleLocation] = Lists.newArrayList( + new FileServerShuffleLocation(partition1Primary, 1234), + new FileServerShuffleLocation(partition2Secondary, 1234)) + makeFileServerMapStatus(partition1List, partition2List) + } + + def makeFileServerMapStatus(partition1Loc: util.List[ShuffleLocation], + partition2Loc: util.List[ShuffleLocation]): MapStatus = { + MapStatus( + makeBlockManagerId("executor-host"), + new FileServerMapShuffleLocations(Seq(partition1Loc, partition2Loc)), + Array.fill[Long](2)(2) + ) + } + + def makeBlockManagerId(host: String): BlockManagerId = + BlockManagerId("exec-" + host, host, 12345) +} From 7794b68c19bc32ecd55ecd2f2786e95ab26ab8f4 Mon Sep 17 00:00:00 2001 From: Yifei Huang Date: Wed, 15 May 2019 19:02:42 -0700 Subject: [PATCH 18/27] add tests for distributed file server case --- .../DAGSchedulerFileServerSuite.scala | 102 +++++++++++++++++- 1 file changed, 97 insertions(+), 5 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerFileServerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerFileServerSuite.scala index 37ec23a959fff..ced14df10427d 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerFileServerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerFileServerSuite.scala @@ -21,9 +21,10 @@ import java.util import java.util.function.Predicate import com.google.common.collect.Lists +import scala.collection.JavaConverters._ import scala.collection.mutable.Map -import org.apache.spark.{HashPartitioner, ShuffleDependency, SparkConf, Success} +import org.apache.spark.{FetchFailed, HashPartitioner, ShuffleDependency, SparkConf, Success} import org.apache.spark.api.java.Optional import org.apache.spark.api.shuffle.{MapShuffleLocations, ShuffleLocation} import org.apache.spark.internal.config @@ -67,6 +68,7 @@ class DAGSchedulerFileServerSuite extends DAGSchedulerSuite { val conf = new SparkConf() // unregistering all outputs on a host is enabled for the individual file server case conf.set(config.UNREGISTER_OUTPUT_ON_HOST_ON_FETCH_FAILURE, true) + conf.set(config.SHUFFLE_SERVICE_ENABLED, true) init(conf) val shuffleMapRdd = new MyRDD(sc, 2, Nil) val shuffleDep = new ShuffleDependency(shuffleMapRdd, new HashPartitioner(2)) @@ -83,7 +85,8 @@ class DAGSchedulerFileServerSuite extends DAGSchedulerSuite { val mapStatus1 = makeFileServerMapStatus("hostA", "hostB", "hostC", "hostD") val mapStatus2 = makeFileServerMapStatus("hostA", "hostB", "hostC", "hostE") complete(taskSets(0), Seq((Success, mapStatus1), (Success, mapStatus2))) - assertMapOutputTrackerContains(shuffleId, Seq(mapStatus1, mapStatus2)) + assertMapOutputTrackerContains(shuffleId, + Seq(mapStatus1.mapShuffleLocations, mapStatus2.mapShuffleLocations)) // perform reduce task complete(taskSets(1), Seq((Success, 42), (Success, 43))) @@ -91,11 +94,90 @@ class DAGSchedulerFileServerSuite extends DAGSchedulerSuite { assertDataStructuresEmpty() } + test("Test one mapper failed but another has blocks elsewhere") { + val (reduceRdd, shuffleId) = setupTest() + submit(reduceRdd, Array(0, 1)) + + // Perform map task + val mapStatus1 = makeFileServerMapStatus("hostA", "hostB", "hostC", "hostD") + val mapStatus2 = makeFileServerMapStatus("hostA", "hostE", "hostC", "hostE") + complete(taskSets(0), Seq((Success, mapStatus1), (Success, mapStatus2))) + assertMapOutputTrackerContains(shuffleId, + Seq(mapStatus1.mapShuffleLocations, mapStatus2.mapShuffleLocations)) + + Seq(1, 2, 3) + // perform reduce task + complete(taskSets(1), Seq((Success, 42), (FetchFailed( + Seq( + shuffleLocation("hostA"), + shuffleLocation("hostB"), + shuffleLocation("hostC"), + shuffleLocation("hostD")), + shuffleId, 0, 0, "ignored"), null))) + assert(scheduler.failedStages.size > 0) + assert(mapOutputTracker.getNumAvailableOutputs(shuffleId) == 1) + assertMapOutputTrackerContains(shuffleId, Seq(null, + new FileServerMapShuffleLocations(Seq( + shuffleLocationSeq("hostE"), + shuffleLocationSeq("hostE"))))) + + scheduler.resubmitFailedStages() + complete(taskSets(2), Seq((Success, mapStatus1))) + assert(mapOutputTracker.getNumAvailableOutputs(shuffleId) == 2) + assertMapOutputTrackerContains(shuffleId, Seq( + mapStatus1.mapShuffleLocations, mapStatus2.mapShuffleLocations)) + + complete(taskSets(3), Seq((Success, 43))) + assert(results === Map(0 -> 42, 1 -> 43)) + assertDataStructuresEmpty() + } + + test("Test one failed but other only has one partition replicated") { + val (reduceRdd, shuffleId) = setupTest() + submit(reduceRdd, Array(0, 1)) + + // Perform map task + val mapStatus1 = makeFileServerMapStatus("hostA", "hostB", "hostC", "hostD") + val mapStatus2 = makeFileServerMapStatus("hostA", "hostB", "hostC", "hostE") + complete(taskSets(0), Seq((Success, mapStatus1), (Success, mapStatus2))) + assertMapOutputTrackerContains(shuffleId, + Seq(mapStatus1.mapShuffleLocations, mapStatus2.mapShuffleLocations)) + + Seq(1, 2, 3) + // perform reduce task + complete(taskSets(1), Seq((Success, 42), (FetchFailed( + Seq( + shuffleLocation("hostA"), + shuffleLocation("hostB"), + shuffleLocation("hostC"), + shuffleLocation("hostD")), + shuffleId, 0, 0, "ignored"), null))) + assert(scheduler.failedStages.size > 0) + assert(mapOutputTracker.getNumAvailableOutputs(shuffleId) == 0) + assertMapOutputTrackerContains(shuffleId, Seq(null, null)) + + scheduler.resubmitFailedStages() + complete(taskSets(2), Seq((Success, mapStatus1), (Success, mapStatus2))) + assert(mapOutputTracker.getNumAvailableOutputs(shuffleId) == 2) + assertMapOutputTrackerContains(shuffleId, Seq( + mapStatus1.mapShuffleLocations, mapStatus2.mapShuffleLocations)) + + complete(taskSets(3), Seq((Success, 43))) + assert(results === Map(0 -> 42, 1 -> 43)) + assertDataStructuresEmpty() + } + def assertMapOutputTrackerContains( shuffleId: Int, - set: Seq[MapStatus]): Unit = { + set: Seq[MapShuffleLocations]): Unit = { val actualShuffleLocations = mapOutputTracker.shuffleStatuses(shuffleId).mapStatuses + .map(mapStatus => { + if (mapStatus == null) { + return null + } + mapStatus.mapShuffleLocations + }) assert(set === actualShuffleLocations.toSeq) } @@ -106,9 +188,9 @@ class DAGSchedulerFileServerSuite extends DAGSchedulerSuite { partition2Secondary: String): MapStatus = { val partition1List: util.List[ShuffleLocation] = Lists.newArrayList( new FileServerShuffleLocation(partition1Primary, 1234), - new FileServerShuffleLocation(partition2Secondary, 1234)) + new FileServerShuffleLocation(partition1Secondary, 1234)) val partition2List: util.List[ShuffleLocation] = Lists.newArrayList( - new FileServerShuffleLocation(partition1Primary, 1234), + new FileServerShuffleLocation(partition2Primary, 1234), new FileServerShuffleLocation(partition2Secondary, 1234)) makeFileServerMapStatus(partition1List, partition2List) } @@ -122,6 +204,16 @@ class DAGSchedulerFileServerSuite extends DAGSchedulerSuite { ) } + def shuffleLocationSeq(hosts: String*): util.List[ShuffleLocation] = { + hosts.map(host => + shuffleLocation(host) + ).asJava + } + + def shuffleLocation(host: String): ShuffleLocation = { + new FileServerShuffleLocation(host, 1234) + } + def makeBlockManagerId(host: String): BlockManagerId = BlockManagerId("exec-" + host, host, 12345) } From 9604889eab5d93440f5921c62104ccbbbe4763a2 Mon Sep 17 00:00:00 2001 From: Yifei Huang Date: Thu, 16 May 2019 16:32:26 -0700 Subject: [PATCH 19/27] use scala in tests --- .../api/shuffle/MapShuffleLocations.java | 4 +- .../sort/DefaultMapShuffleLocations.java | 4 +- .../org/apache/spark/MapOutputTracker.scala | 6 +-- .../scheduler/DAGSchedulerAsyncSuite.scala | 4 +- .../DAGSchedulerFileServerSuite.scala | 42 +++++++++---------- 5 files changed, 28 insertions(+), 32 deletions(-) diff --git a/core/src/main/java/org/apache/spark/api/shuffle/MapShuffleLocations.java b/core/src/main/java/org/apache/spark/api/shuffle/MapShuffleLocations.java index dd0014cebd787..a7b2c3006bb72 100644 --- a/core/src/main/java/org/apache/spark/api/shuffle/MapShuffleLocations.java +++ b/core/src/main/java/org/apache/spark/api/shuffle/MapShuffleLocations.java @@ -50,7 +50,7 @@ public interface MapShuffleLocations extends Serializable { * location. Otherwise, if all partitions can still be fetched from alternative locations, * this should return false. */ - boolean removeShuffleLocation(String host, Optional port); + boolean invalidateShuffleLocation(String host, Optional port); /** * Mark all locations within this MapShuffleLocations with this execId as unreachable. @@ -62,5 +62,5 @@ public interface MapShuffleLocations extends Serializable { * with this execId. Otherwise, if all partitions can still be fetched form alternative locaitons, * this should return false. */ - boolean removeShuffleLocation(String executorId); + boolean invalidateShuffleLocation(String executorId); } diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/DefaultMapShuffleLocations.java b/core/src/main/java/org/apache/spark/shuffle/sort/DefaultMapShuffleLocations.java index 1e171a903323d..ef7e4dab91543 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/DefaultMapShuffleLocations.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/DefaultMapShuffleLocations.java @@ -67,7 +67,7 @@ public List getLocationsForBlock(int reduceId) { } @Override - public boolean removeShuffleLocation(String host, Optional port) { + public boolean invalidateShuffleLocation(String host, Optional port) { if (port.isPresent()) { return this.host().equals(host) && this.port() == port.get(); } @@ -75,7 +75,7 @@ public boolean removeShuffleLocation(String host, Optional port) { } @Override - public boolean removeShuffleLocation(String executorId) { + public boolean invalidateShuffleLocation(String executorId) { return location.executorId().equals(executorId); } diff --git a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala index da5b4a5466a95..8acb8034620a0 100644 --- a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala +++ b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala @@ -112,7 +112,7 @@ private class ShuffleStatus(numPartitions: Int) { shuffleLocations.foreach { location => shouldDelete = mapStatuses(mapId) .mapShuffleLocations - .removeShuffleLocation(location.host(), Optional.of(location.port())) + .invalidateShuffleLocation(location.host(), Optional.of(location.port())) } } if (shouldDelete) { @@ -130,7 +130,7 @@ private class ShuffleStatus(numPartitions: Int) { def removeOutputsOnHost(host: String): Unit = { for (mapId <- 0 until mapStatuses.length) { if (mapStatuses(mapId) != null && - mapStatuses(mapId).mapShuffleLocations.removeShuffleLocation(host, Optional.empty())) { + mapStatuses(mapId).mapShuffleLocations.invalidateShuffleLocation(host, Optional.empty())) { _numAvailableOutputs -= 1 mapStatuses(mapId) = null invalidateSerializedMapOutputStatusCache() @@ -146,7 +146,7 @@ private class ShuffleStatus(numPartitions: Int) { def removeOutputsOnExecutor(execId: String): Unit = synchronized { for (mapId <- 0 until mapStatuses.length) { if (mapStatuses(mapId) != null && - mapStatuses(mapId).mapShuffleLocations.removeShuffleLocation(execId)) { + mapStatuses(mapId).mapShuffleLocations.invalidateShuffleLocation(execId)) { _numAvailableOutputs -= 1 mapStatuses(mapId) = null invalidateSerializedMapOutputStatusCache() diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerAsyncSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerAsyncSuite.scala index 2e2e2de04572a..73f19668a8fed 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerAsyncSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerAsyncSuite.scala @@ -57,14 +57,14 @@ class DAGSchedulerAsyncSuite extends DAGSchedulerSuite { override def getLocationsForBlock(reduceId: Int): util.List[ShuffleLocation] = locations - override def removeShuffleLocation(host: String, port: Optional[Integer]): Boolean = { + override def invalidateShuffleLocation(host: String, port: Optional[Integer]): Boolean = { removeIfPredicate(new Predicate[ShuffleLocation] { override def test(loc: ShuffleLocation): Boolean = loc.host() === host && (!port.isPresent || loc.port() === port.get()) }) } - override def removeShuffleLocation(executorId: String): Boolean = { + override def invalidateShuffleLocation(executorId: String): Boolean = { removeIfPredicate(new Predicate[ShuffleLocation] { override def test(loc: ShuffleLocation): Boolean = locations.get(0).execId().get().equals(executorId) diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerFileServerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerFileServerSuite.scala index ced14df10427d..fe6f1407e90f8 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerFileServerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerFileServerSuite.scala @@ -18,11 +18,11 @@ package org.apache.spark.scheduler import java.util -import java.util.function.Predicate -import com.google.common.collect.Lists import scala.collection.JavaConverters._ +import scala.collection.mutable.Buffer import scala.collection.mutable.Map +import scala.collection.mutable.Seq import org.apache.spark.{FetchFailed, HashPartitioner, ShuffleDependency, SparkConf, Success} import org.apache.spark.api.java.Optional @@ -37,20 +37,18 @@ class DAGSchedulerFileServerSuite extends DAGSchedulerSuite { override def host(): String = hostname override def port(): Int = portInt } - - class FileServerMapShuffleLocations(mapShuffleLocations: Seq[util.List[ShuffleLocation]]) + + class FileServerMapShuffleLocations(mapShuffleLocationsInput: Buffer[Buffer[ShuffleLocation]]) extends MapShuffleLocations { + val mapShuffleLocations = mapShuffleLocationsInput override def getLocationsForBlock(reduceId: Int): util.List[ShuffleLocation] = - mapShuffleLocations(reduceId) + mapShuffleLocations(reduceId).asJava - override def removeShuffleLocation(host: String, port: Optional[Integer]): Boolean = { + override def invalidateShuffleLocation(host: String, port: Optional[Integer]): Boolean = { var missingPartition = false - for (locations <- mapShuffleLocations) { - locations.removeIf(new Predicate[ShuffleLocation] { - override def test(loc: ShuffleLocation): Boolean = { - loc.host() === host && (!port.isPresent || loc.port() === port.get()) - } - }) + for ((locations, i) <- mapShuffleLocations.zipWithIndex) { + mapShuffleLocations(i) = locations.filter(loc => + loc.host() != host || (port.isPresent && loc.port() != port.get())) if (locations.isEmpty) { missingPartition = true } @@ -58,7 +56,7 @@ class DAGSchedulerFileServerSuite extends DAGSchedulerSuite { missingPartition } - override def removeShuffleLocation(executorId: String): Boolean = { + override def invalidateShuffleLocation(executorId: String): Boolean = { return false } } @@ -105,7 +103,6 @@ class DAGSchedulerFileServerSuite extends DAGSchedulerSuite { assertMapOutputTrackerContains(shuffleId, Seq(mapStatus1.mapShuffleLocations, mapStatus2.mapShuffleLocations)) - Seq(1, 2, 3) // perform reduce task complete(taskSets(1), Seq((Success, 42), (FetchFailed( Seq( @@ -117,7 +114,7 @@ class DAGSchedulerFileServerSuite extends DAGSchedulerSuite { assert(scheduler.failedStages.size > 0) assert(mapOutputTracker.getNumAvailableOutputs(shuffleId) == 1) assertMapOutputTrackerContains(shuffleId, Seq(null, - new FileServerMapShuffleLocations(Seq( + new FileServerMapShuffleLocations(Buffer( shuffleLocationSeq("hostE"), shuffleLocationSeq("hostE"))))) @@ -143,7 +140,6 @@ class DAGSchedulerFileServerSuite extends DAGSchedulerSuite { assertMapOutputTrackerContains(shuffleId, Seq(mapStatus1.mapShuffleLocations, mapStatus2.mapShuffleLocations)) - Seq(1, 2, 3) // perform reduce task complete(taskSets(1), Seq((Success, 42), (FetchFailed( Seq( @@ -186,28 +182,28 @@ class DAGSchedulerFileServerSuite extends DAGSchedulerSuite { partition1Secondary: String, partition2Primary: String, partition2Secondary: String): MapStatus = { - val partition1List: util.List[ShuffleLocation] = Lists.newArrayList( + val partition1List: Buffer[ShuffleLocation] = Buffer( new FileServerShuffleLocation(partition1Primary, 1234), new FileServerShuffleLocation(partition1Secondary, 1234)) - val partition2List: util.List[ShuffleLocation] = Lists.newArrayList( + val partition2List: Buffer[ShuffleLocation] = Buffer( new FileServerShuffleLocation(partition2Primary, 1234), new FileServerShuffleLocation(partition2Secondary, 1234)) makeFileServerMapStatus(partition1List, partition2List) } - def makeFileServerMapStatus(partition1Loc: util.List[ShuffleLocation], - partition2Loc: util.List[ShuffleLocation]): MapStatus = { + def makeFileServerMapStatus(partition1Loc: Buffer[ShuffleLocation], + partition2Loc: Buffer[ShuffleLocation]): MapStatus = { MapStatus( makeBlockManagerId("executor-host"), - new FileServerMapShuffleLocations(Seq(partition1Loc, partition2Loc)), + new FileServerMapShuffleLocations(Buffer(partition1Loc, partition2Loc)), Array.fill[Long](2)(2) ) } - def shuffleLocationSeq(hosts: String*): util.List[ShuffleLocation] = { + def shuffleLocationSeq(hosts: String*): Buffer[ShuffleLocation] = { hosts.map(host => shuffleLocation(host) - ).asJava + ).toBuffer } def shuffleLocation(host: String): ShuffleLocation = { From 58c8ef22ee932b71fc8468dcf944556264533611 Mon Sep 17 00:00:00 2001 From: Yifei Huang Date: Fri, 17 May 2019 10:31:53 -0700 Subject: [PATCH 20/27] more buffers --- .../scheduler/DAGSchedulerAsyncSuite.scala | 28 ++++++++----------- .../DAGSchedulerFileServerSuite.scala | 2 +- 2 files changed, 12 insertions(+), 18 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerAsyncSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerAsyncSuite.scala index 73f19668a8fed..b5f903ab3c389 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerAsyncSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerAsyncSuite.scala @@ -18,12 +18,11 @@ package org.apache.spark.scheduler import java.util -import java.util.function.Predicate +import scala.collection.JavaConverters._ +import scala.collection.mutable.Buffer import scala.collection.mutable.Map -import com.google.common.collect.Lists - import org.apache.spark.{FetchFailed, HashPartitioner, ShuffleDependency, SparkConf, Success} import org.apache.spark.api.java.Optional import org.apache.spark.api.shuffle.{MapShuffleLocations, ShuffleLocation} @@ -48,35 +47,30 @@ class DAGSchedulerAsyncSuite extends DAGSchedulerSuite { class AsyncMapShuffleLocations(asyncLocation: AsyncShuffleLocation) extends MapShuffleLocations { - val locations : util.List[ShuffleLocation] = if (asyncLocation == null) { - Lists.newArrayList(dfsLocation) + var locations : Buffer[ShuffleLocation] = if (asyncLocation == null) { + Buffer(dfsLocation) } else { - Lists.newArrayList(asyncLocation, dfsLocation) + Buffer(asyncLocation, dfsLocation) } override def getLocationsForBlock(reduceId: Int): util.List[ShuffleLocation] = - locations + locations.asJava override def invalidateShuffleLocation(host: String, port: Optional[Integer]): Boolean = { - removeIfPredicate(new Predicate[ShuffleLocation] { - override def test(loc: ShuffleLocation): Boolean = - loc.host() === host && (!port.isPresent || loc.port() === port.get()) - }) + removeIfPredicate(loc => + loc.host() === host && (!port.isPresent || loc.port() === port.get())) } override def invalidateShuffleLocation(executorId: String): Boolean = { - removeIfPredicate(new Predicate[ShuffleLocation] { - override def test(loc: ShuffleLocation): Boolean = - locations.get(0).execId().get().equals(executorId) - }) + removeIfPredicate(loc => !loc.execId().isPresent || !loc.execId().get().equals(executorId)) } - def removeIfPredicate(predicate: Predicate[ShuffleLocation]): Boolean = { + def removeIfPredicate(predicate: ShuffleLocation => Boolean): Boolean = { var missingPartition = false if (locations.isEmpty) { return missingPartition } - locations.removeIf(predicate) + locations = locations.filter(predicate) if (locations.isEmpty) { missingPartition = true } diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerFileServerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerFileServerSuite.scala index fe6f1407e90f8..20384b7a89deb 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerFileServerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerFileServerSuite.scala @@ -37,7 +37,7 @@ class DAGSchedulerFileServerSuite extends DAGSchedulerSuite { override def host(): String = hostname override def port(): Int = portInt } - + class FileServerMapShuffleLocations(mapShuffleLocationsInput: Buffer[Buffer[ShuffleLocation]]) extends MapShuffleLocations { val mapShuffleLocations = mapShuffleLocationsInput From 506a3f7030c1e1710909fe119b361fa97d2c5544 Mon Sep 17 00:00:00 2001 From: Yifei Huang Date: Fri, 17 May 2019 10:44:20 -0700 Subject: [PATCH 21/27] use Seq.empty instead of null --- core/src/main/scala/org/apache/spark/MapOutputTracker.scala | 2 +- core/src/main/scala/org/apache/spark/TaskEndReason.scala | 2 +- .../main/scala/org/apache/spark/scheduler/DAGScheduler.scala | 2 +- .../scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala | 2 +- .../org/apache/spark/scheduler/TaskSchedulerImplSuite.scala | 2 +- .../scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala | 2 +- 6 files changed, 6 insertions(+), 6 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala index 8acb8034620a0..ecf9d07325a38 100644 --- a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala +++ b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala @@ -106,7 +106,7 @@ private class ShuffleStatus(numPartitions: Int) { def removeMapOutput(mapId: Int, shuffleLocations: Seq[ShuffleLocation]): Unit = synchronized { if (mapStatuses(mapId) != null) { var shouldDelete = false - if (shuffleLocations == null) { + if (shuffleLocations.isEmpty) { shouldDelete = true } else { shuffleLocations.foreach { location => diff --git a/core/src/main/scala/org/apache/spark/TaskEndReason.scala b/core/src/main/scala/org/apache/spark/TaskEndReason.scala index d7ba6e3db36d9..f84b36e804549 100644 --- a/core/src/main/scala/org/apache/spark/TaskEndReason.scala +++ b/core/src/main/scala/org/apache/spark/TaskEndReason.scala @@ -82,7 +82,7 @@ case object Resubmitted extends TaskFailedReason { */ @DeveloperApi case class FetchFailed( - shuffleLocation: Seq[ShuffleLocation], // Note that shuffleLocation can be null + shuffleLocation: Seq[ShuffleLocation], // Note that shuffleLocation cannot be null shuffleId: Int, mapId: Int, reduceId: Int, 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 96205e76d854b..1c29803b9ecfa 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -1627,7 +1627,7 @@ private[spark] class DAGScheduler( } // TODO: mark the executor as failed only if there were lots of fetch failures on it - if (shuffleLocations != null) { + if (!shuffleLocations.isEmpty) { val toRemoveHost = if (env.conf.get(config.SHUFFLE_IO_PLUGIN_CLASS) == classOf[DefaultShuffleDataIO].getName) { 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 cd70fe580199d..ad1bd8d1b1bd6 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -1492,7 +1492,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi runEvent(ExecutorLost("exec-hostA", ExecutorKilled)) runEvent(makeCompletionEvent( taskSets(1).tasks(0), - FetchFailed(null, firstShuffleId, 2, 0, "Fetch failed"), + FetchFailed(Seq.empty, firstShuffleId, 2, 0, "Fetch failed"), null)) // so we resubmit stage 0, which completes happily diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala index 1a81f556e0612..3a3cf61657c7d 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala @@ -1137,7 +1137,7 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B assert(tsm.runningTasks === 10) // fail attempt tsm.handleFailedTask(tsm.taskAttempts.head.head.taskId, TaskState.FAILED, - FetchFailed(null, 0, 0, 0, "fetch failed")) + FetchFailed(Seq.empty, 0, 0, 0, "fetch failed")) // the attempt is a zombie, but the tasks are still running (this could be true even if // we actively killed those tasks, as killing is best-effort) assert(tsm.isZombie) 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 f9b81ef1ef3e3..830a6a77ab390 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala @@ -1397,7 +1397,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg // fail fetch taskSetManager1.handleFailedTask( taskSetManager1.taskAttempts.head.head.taskId, TaskState.FAILED, - FetchFailed(null, 0, 0, 0, "fetch failed")) + FetchFailed(Seq.empty, 0, 0, 0, "fetch failed")) assert(taskSetManager1.isZombie) assert(taskSetManager1.runningTasks === 9) From 118a2bf2b8ab7bb58569ec671163ed992a582049 Mon Sep 17 00:00:00 2001 From: Yifei Huang Date: Fri, 17 May 2019 12:00:38 -0700 Subject: [PATCH 22/27] add flag for removing other map statuses --- .../api/shuffle/ShuffleDriverComponents.java | 8 ++++ .../DefaultShuffleDriverComponents.java | 5 +++ .../scala/org/apache/spark/SparkContext.scala | 3 ++ .../apache/spark/scheduler/DAGScheduler.scala | 7 +++- .../scheduler/DAGSchedulerAsyncSuite.scala | 26 +++++++++++- .../DAGSchedulerFileServerSuite.scala | 40 +++++++++++++++---- 6 files changed, 78 insertions(+), 11 deletions(-) diff --git a/core/src/main/java/org/apache/spark/api/shuffle/ShuffleDriverComponents.java b/core/src/main/java/org/apache/spark/api/shuffle/ShuffleDriverComponents.java index 6a0ec8d44fd4f..4f2a45731bef4 100644 --- a/core/src/main/java/org/apache/spark/api/shuffle/ShuffleDriverComponents.java +++ b/core/src/main/java/org/apache/spark/api/shuffle/ShuffleDriverComponents.java @@ -30,4 +30,12 @@ public interface ShuffleDriverComponents { void cleanupApplication() throws IOException; void removeShuffleData(int shuffleId, boolean blocking) throws IOException; + + /** + * Whether to unregister other map statuses on the same hosts or executors + * when a shuffle task returns a {@link org.apache.spark.FetchFailed}. + */ + default boolean unregisterOtherMapStatusesOnFetchFailure() { + return false; + } } diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/lifecycle/DefaultShuffleDriverComponents.java b/core/src/main/java/org/apache/spark/shuffle/sort/lifecycle/DefaultShuffleDriverComponents.java index a3eddc8ec930e..d8d229e4dfd7e 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/lifecycle/DefaultShuffleDriverComponents.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/lifecycle/DefaultShuffleDriverComponents.java @@ -46,6 +46,11 @@ public void removeShuffleData(int shuffleId, boolean blocking) throws IOExceptio blockManagerMaster.removeShuffle(shuffleId, blocking); } + @Override + public boolean unregisterOtherMapStatusesOnFetchFailure() { + return true; + } + private void checkInitialized() { if (blockManagerMaster == null) { throw new IllegalStateException("Driver components must be initialized before using"); diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 999f180193d84..f9e15fb7f19ba 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -308,6 +308,9 @@ class SparkContext(config: SparkConf) extends SafeLogging { _dagScheduler = ds } + private[spark] def shuffleDriverComponents: ShuffleDriverComponents = + _shuffleDriverComponents + /** * A unique identifier for the Spark application. * Its format depends on the scheduler implementation. 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 1c29803b9ecfa..a63a6ed8520b2 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -229,6 +229,9 @@ private[spark] class DAGScheduler( private[spark] val eventProcessLoop = new DAGSchedulerEventProcessLoop(this) taskScheduler.setDAGScheduler(this) + private[spark] val unregisterOtherMapStatusesOnFetchFailure = sc.shuffleDriverComponents + .unregisterOtherMapStatusesOnFetchFailure() + /** * Called by the TaskSetManager to report task's starting. */ @@ -1627,14 +1630,14 @@ private[spark] class DAGScheduler( } // TODO: mark the executor as failed only if there were lots of fetch failures on it - if (!shuffleLocations.isEmpty) { + if (unregisterOtherMapStatusesOnFetchFailure && shuffleLocations.nonEmpty) { val toRemoveHost = if (env.conf.get(config.SHUFFLE_IO_PLUGIN_CLASS) == classOf[DefaultShuffleDataIO].getName) { env.blockManager.externalShuffleServiceEnabled && unRegisterOutputOnHostOnFetchFailure } else { - unRegisterOutputOnHostOnFetchFailure + true } shuffleLocations.foreach(location => { diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerAsyncSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerAsyncSuite.scala index b5f903ab3c389..6e9de03e3ebc1 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerAsyncSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerAsyncSuite.scala @@ -25,11 +25,33 @@ import scala.collection.mutable.Map import org.apache.spark.{FetchFailed, HashPartitioner, ShuffleDependency, SparkConf, Success} import org.apache.spark.api.java.Optional -import org.apache.spark.api.shuffle.{MapShuffleLocations, ShuffleLocation} +import org.apache.spark.api.shuffle.{MapShuffleLocations, ShuffleDataIO, ShuffleDriverComponents, ShuffleExecutorComponents, ShuffleLocation} import org.apache.spark.internal.config import org.apache.spark.rdd.RDD +import org.apache.spark.shuffle.sort.io.DefaultShuffleDataIO import org.apache.spark.storage.BlockManagerId +class AsyncShuffleDataIO(sparkConf: SparkConf) extends ShuffleDataIO { + val defaultShuffleDataIO = new DefaultShuffleDataIO(sparkConf) + + override def driver(): ShuffleDriverComponents = + new AsyncShuffleDriverComponents(defaultShuffleDataIO.driver()) + + override def executor(): ShuffleExecutorComponents = defaultShuffleDataIO.executor() +} + +class AsyncShuffleDriverComponents(default: ShuffleDriverComponents) + extends ShuffleDriverComponents { + override def initializeApplication(): util.Map[String, String] = default.initializeApplication() + + override def cleanupApplication(): Unit = default.cleanupApplication() + + override def removeShuffleData(shuffleId: Int, blocking: Boolean): Unit = + default.removeShuffleData(shuffleId, blocking) + + override def unregisterOtherMapStatusesOnFetchFailure(): Boolean = false +} + class DAGSchedulerAsyncSuite extends DAGSchedulerSuite { class AsyncShuffleLocation(hostname: String, portInt: Int, exec: String) extends ShuffleLocation { @@ -82,7 +104,7 @@ class DAGSchedulerAsyncSuite extends DAGSchedulerSuite { afterEach() val conf = new SparkConf() // unregistering all outputs on a host is disabled for async case - conf.set(config.UNREGISTER_OUTPUT_ON_HOST_ON_FETCH_FAILURE, false) + conf.set(config.SHUFFLE_IO_PLUGIN_CLASS, classOf[AsyncShuffleDataIO].getName) init(conf) val shuffleMapRdd = new MyRDD(sc, 2, Nil) val shuffleDep = new ShuffleDependency(shuffleMapRdd, new HashPartitioner(2)) diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerFileServerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerFileServerSuite.scala index 20384b7a89deb..376f7759f8f2f 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerFileServerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerFileServerSuite.scala @@ -26,11 +26,40 @@ import scala.collection.mutable.Seq import org.apache.spark.{FetchFailed, HashPartitioner, ShuffleDependency, SparkConf, Success} import org.apache.spark.api.java.Optional -import org.apache.spark.api.shuffle.{MapShuffleLocations, ShuffleLocation} +import org.apache.spark.api.shuffle.{MapShuffleLocations, ShuffleDataIO, ShuffleDriverComponents, ShuffleExecutorComponents, ShuffleLocation} import org.apache.spark.internal.config import org.apache.spark.rdd.RDD +import org.apache.spark.shuffle.sort.io.DefaultShuffleDataIO import org.apache.spark.storage.BlockManagerId +class FileServerShuffleDriverComponents(driverComponents: ShuffleDriverComponents) + extends ShuffleDriverComponents { + + override def initializeApplication(): util.Map[String, String] = + driverComponents.initializeApplication() + + override def cleanupApplication(): Unit = driverComponents.cleanupApplication() + + override def removeShuffleData(shuffleId: Int, blocking: Boolean): Unit = + driverComponents.removeShuffleData(shuffleId, blocking) + + override def unregisterOtherMapStatusesOnFetchFailure(): Boolean = true +} + +class FileServerShuffleDataIO(sparkConf: SparkConf) extends ShuffleDataIO { + val defaultShuffleDataIO = new DefaultShuffleDataIO(sparkConf) + override def driver(): ShuffleDriverComponents = + new FileServerShuffleDriverComponents(defaultShuffleDataIO.driver()) + + override def executor(): ShuffleExecutorComponents = defaultShuffleDataIO.executor() +} + +object FileServerShuffleDataIO { + def apply(sparkConf: SparkConf): Unit = { + new FileServerShuffleDataIO(sparkConf) + } +} + class DAGSchedulerFileServerSuite extends DAGSchedulerSuite { class FileServerShuffleLocation(hostname: String, portInt: Int) extends ShuffleLocation { @@ -49,24 +78,21 @@ class DAGSchedulerFileServerSuite extends DAGSchedulerSuite { for ((locations, i) <- mapShuffleLocations.zipWithIndex) { mapShuffleLocations(i) = locations.filter(loc => loc.host() != host || (port.isPresent && loc.port() != port.get())) - if (locations.isEmpty) { + if (mapShuffleLocations(i).isEmpty) { missingPartition = true } } missingPartition } - override def invalidateShuffleLocation(executorId: String): Boolean = { - return false - } + override def invalidateShuffleLocation(executorId: String): Boolean = false } private def setupTest(): (RDD[_], Int) = { afterEach() val conf = new SparkConf() // unregistering all outputs on a host is enabled for the individual file server case - conf.set(config.UNREGISTER_OUTPUT_ON_HOST_ON_FETCH_FAILURE, true) - conf.set(config.SHUFFLE_SERVICE_ENABLED, true) + conf.set(config.SHUFFLE_IO_PLUGIN_CLASS, classOf[FileServerShuffleDataIO].getName) init(conf) val shuffleMapRdd = new MyRDD(sc, 2, Nil) val shuffleDep = new ShuffleDependency(shuffleMapRdd, new HashPartitioner(2)) From bf8aeab07123d7bc4cd1efcece85b9d2b1526b76 Mon Sep 17 00:00:00 2001 From: Yifei Huang Date: Fri, 17 May 2019 16:43:47 -0700 Subject: [PATCH 23/27] small fixes --- .../apache/spark/scheduler/TaskSetManager.scala | 14 +++++++------- .../scheduler/DAGSchedulerFileServerSuite.scala | 6 ++---- 2 files changed, 9 insertions(+), 11 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 8b6d20dc08c89..3ae8064c1ed41 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala @@ -853,13 +853,13 @@ private[spark] class TaskSetManager( // locations shouldn't be running executors, so only fetches using the default Spark // implementation (DefaultMapShuffleLocations) of fetching from executor disk should result // in blacklistable executors. - if (fetchFailed.shuffleLocation != null && fetchFailed.shuffleLocation.nonEmpty - && fetchFailed.shuffleLocation.head.isInstanceOf[DefaultMapShuffleLocations]) { - val bmAddress = fetchFailed.shuffleLocation.head - .asInstanceOf[DefaultMapShuffleLocations] - .getBlockManagerId - blacklistTracker.foreach(_.updateBlacklistForFetchFailure( - bmAddress.host, bmAddress.executorId)) + if (fetchFailed.shuffleLocation.nonEmpty) { + fetchFailed.shuffleLocation.foreach(loc => { + if (loc.execId().isPresent) { + blacklistTracker.foreach(_.updateBlacklistForFetchFailure( + loc.host(), loc.execId().get())) + } + }) } None diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerFileServerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerFileServerSuite.scala index 376f7759f8f2f..e58c939db873e 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerFileServerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerFileServerSuite.scala @@ -124,7 +124,7 @@ class DAGSchedulerFileServerSuite extends DAGSchedulerSuite { // Perform map task val mapStatus1 = makeFileServerMapStatus("hostA", "hostB", "hostC", "hostD") - val mapStatus2 = makeFileServerMapStatus("hostA", "hostE", "hostC", "hostE") + val mapStatus2 = makeFileServerMapStatus("hostA", "hostE", "hostB", "hostE") complete(taskSets(0), Seq((Success, mapStatus1), (Success, mapStatus2))) assertMapOutputTrackerContains(shuffleId, Seq(mapStatus1.mapShuffleLocations, mapStatus2.mapShuffleLocations)) @@ -133,9 +133,7 @@ class DAGSchedulerFileServerSuite extends DAGSchedulerSuite { complete(taskSets(1), Seq((Success, 42), (FetchFailed( Seq( shuffleLocation("hostA"), - shuffleLocation("hostB"), - shuffleLocation("hostC"), - shuffleLocation("hostD")), + shuffleLocation("hostB")), shuffleId, 0, 0, "ignored"), null))) assert(scheduler.failedStages.size > 0) assert(mapOutputTracker.getNumAvailableOutputs(shuffleId) == 1) From 994ff5a42fc003cae8cb51afe9201ea3758dbd41 Mon Sep 17 00:00:00 2001 From: Yifei Huang Date: Mon, 20 May 2019 11:58:29 -0700 Subject: [PATCH 24/27] use seq empty --- core/src/main/scala/org/apache/spark/util/JsonProtocol.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 e534ef8938218..268f7d2848239 100644 --- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -1024,7 +1024,7 @@ private[spark] object JsonProtocol { DefaultMapShuffleLocations.get(block) }) } else { - null + Seq.empty } } From 7758a90b2c9def4a258025c7938dfa88a7f4b293 Mon Sep 17 00:00:00 2001 From: Yifei Huang Date: Mon, 20 May 2019 12:57:09 -0700 Subject: [PATCH 25/27] simplify async case --- .../scheduler/DAGSchedulerAsyncSuite.scala | 21 +++++-------------- 1 file changed, 5 insertions(+), 16 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerAsyncSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerAsyncSuite.scala index 6e9de03e3ebc1..7a2a4a5205cf8 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerAsyncSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerAsyncSuite.scala @@ -60,27 +60,16 @@ class DAGSchedulerAsyncSuite extends DAGSchedulerSuite { override def execId(): Optional[String] = Optional.of(exec) } - class DFSShuffleLocation extends ShuffleLocation { - override def host(): String = "hdfs" - override def port(): Int = 1234 - } - - val dfsLocation = new DFSShuffleLocation - class AsyncMapShuffleLocations(asyncLocation: AsyncShuffleLocation) extends MapShuffleLocations { - var locations : Buffer[ShuffleLocation] = if (asyncLocation == null) { - Buffer(dfsLocation) - } else { - Buffer(asyncLocation, dfsLocation) - } + var locations : Buffer[ShuffleLocation] = Buffer(asyncLocation) override def getLocationsForBlock(reduceId: Int): util.List[ShuffleLocation] = locations.asJava override def invalidateShuffleLocation(host: String, port: Optional[Integer]): Boolean = { removeIfPredicate(loc => - loc.host() === host && (!port.isPresent || loc.port() === port.get())) + loc.host() != host || (port.isPresent && loc.port() != port.get())) } override def invalidateShuffleLocation(executorId: String): Boolean = { @@ -143,7 +132,7 @@ class DAGSchedulerAsyncSuite extends DAGSchedulerSuite { // but the other shuffle block is still available complete(taskSets(1), Seq( (Success, 42), - (FetchFailed(Seq(makeAsyncShuffleLocation("hostA"), dfsLocation), + (FetchFailed(Seq(makeAsyncShuffleLocation("hostA")), shuffleId, 0, 0, "ignored"), null))) assert(scheduler.failedStages.size > 0) assert(mapOutputTracker.getNumAvailableOutputs(shuffleId) == 1) @@ -172,7 +161,7 @@ class DAGSchedulerAsyncSuite extends DAGSchedulerSuite { // other task is still intact because it was uploaded to the remove dfs complete(taskSets(1), Seq( (Success, 42), - (FetchFailed(Seq(makeAsyncShuffleLocation("hostA"), dfsLocation), + (FetchFailed(Seq(makeAsyncShuffleLocation("hostA")), shuffleId, 0, 0, "ignored"), null))) assert(scheduler.failedStages.size > 0) assert(mapOutputTracker.getNumAvailableOutputs(shuffleId) == 1) @@ -207,7 +196,7 @@ class DAGSchedulerAsyncSuite extends DAGSchedulerSuite { // the second location was written by a different executor complete(taskSets(1), Seq( (Success, 42), - (FetchFailed(Seq(makeAsyncShuffleLocation("hostA"), dfsLocation), + (FetchFailed(Seq(makeAsyncShuffleLocation("hostA")), shuffleId, 0, 0, "ignored"), null))) assert(scheduler.failedStages.size > 0) assert(mapOutputTracker.getNumAvailableOutputs(shuffleId) == 1) From cebe386d0412178bd40f875295bc90def5e6592c Mon Sep 17 00:00:00 2001 From: Yifei Huang Date: Mon, 20 May 2019 12:57:21 -0700 Subject: [PATCH 26/27] remove another null to Seq.empty --- .../scala/org/apache/spark/shuffle/FetchFailedException.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 de4cb361e931d..100e011700b36 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/FetchFailedException.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/FetchFailedException.scala @@ -71,4 +71,4 @@ private[spark] class MetadataFetchFailedException( shuffleId: Int, reduceId: Int, message: String) - extends FetchFailedException(null, shuffleId, -1, reduceId, message) + extends FetchFailedException(Seq.empty, shuffleId, -1, reduceId, message) From 056c87bb683c848b4637ea5ffa9c7f509c522f9f Mon Sep 17 00:00:00 2001 From: Yifei Huang Date: Mon, 20 May 2019 14:08:00 -0700 Subject: [PATCH 27/27] one more --- .../scala/org/apache/spark/scheduler/TaskContextSuite.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 27369759fad5d..29dfb8d437156 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskContextSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskContextSuite.scala @@ -176,7 +176,7 @@ class TaskContextSuite extends SparkFunSuite with BeforeAndAfter with LocalSpark if (stageAttemptNumber < 2) { // Throw FetchFailedException to explicitly trigger stage resubmission. A normal exception // will only trigger task resubmission in the same stage. - throw new FetchFailedException(null, 0, 0, 0, "Fake") + throw new FetchFailedException(Seq.empty, 0, 0, 0, "Fake") } Seq(stageAttemptNumber).iterator }.collect()