Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Make removing map output by exec id configurable #608

Merged
merged 3 commits into from
Oct 3, 2019
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,12 @@

public interface ShuffleDriverComponents {

enum MapOutputUnregistrationStrategy {
MAP_OUTPUT_ONLY,
EXECUTOR,
HOST,
}

/**
* @return additional SparkConf values necessary for the executors.
*/
Expand All @@ -33,7 +39,7 @@ default void registerShuffle(int shuffleId) throws IOException {}

default void removeShuffle(int shuffleId, boolean blocking) throws IOException {}

default boolean shouldUnregisterOutputOnHostOnFetchFailure() {
return false;
default MapOutputUnregistrationStrategy unregistrationStrategyOnFetchFailure() {
return MapOutputUnregistrationStrategy.EXECUTOR;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -48,8 +48,11 @@ public void removeShuffle(int shuffleId, boolean blocking) {
}

@Override
public boolean shouldUnregisterOutputOnHostOnFetchFailure() {
return shouldUnregisterOutputOnHostOnFetchFailure;
public MapOutputUnregistrationStrategy unregistrationStrategyOnFetchFailure() {
if (shouldUnregisterOutputOnHostOnFetchFailure) {
return MapOutputUnregistrationStrategy.HOST;
}
return MapOutputUnregistrationStrategy.EXECUTOR;
}

private void checkInitialized() {
Expand Down
14 changes: 10 additions & 4 deletions core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala
Original file line number Diff line number Diff line change
Expand Up @@ -43,6 +43,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.api.ShuffleDriverComponents.MapOutputUnregistrationStrategy
import org.apache.spark.storage._
import org.apache.spark.storage.BlockManagerMessages.BlockManagerHeartbeat
import org.apache.spark.util._
Expand Down Expand Up @@ -1672,7 +1673,8 @@ private[spark] class DAGScheduler(
// TODO: mark the executor as failed only if there were lots of fetch failures on it
if (bmAddress != null) {
if (bmAddress.executorId == null) {
if (shuffleDriverComponents.shouldUnregisterOutputOnHostOnFetchFailure()) {
if (shuffleDriverComponents.unregistrationStrategyOnFetchFailure() ==
MapOutputUnregistrationStrategy.HOST) {
val currentEpoch = task.epoch
val host = bmAddress.host
logInfo("Shuffle files lost for host: %s (epoch %d)".format(host, currentEpoch))
Expand All @@ -1681,7 +1683,8 @@ private[spark] class DAGScheduler(
}
} else {
val hostToUnregisterOutputs =
if (shuffleDriverComponents.shouldUnregisterOutputOnHostOnFetchFailure()) {
if (shuffleDriverComponents.unregistrationStrategyOnFetchFailure() ==
MapOutputUnregistrationStrategy.HOST) {
// We had a fetch failure with the external shuffle service, so we
// assume all shuffle data on the node is bad.
Some(bmAddress.host)
Expand Down Expand Up @@ -1863,8 +1866,11 @@ private[spark] class DAGScheduler(
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))
mapOutputTracker.removeOutputsOnExecutor(execId)
if (shuffleDriverComponents.unregistrationStrategyOnFetchFailure() ==
MapOutputUnregistrationStrategy.EXECUTOR) {
logInfo("Shuffle files lost for executor: %s (epoch %d)".format(execId, currentEpoch))
mapOutputTracker.removeOutputsOnExecutor(execId)
}
}
clearCacheLocs()

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,7 @@ import org.apache.spark.{FetchFailed, HashPartitioner, ShuffleDependency, SparkC
import org.apache.spark.internal.config
import org.apache.spark.rdd.RDD
import org.apache.spark.shuffle.api.{ShuffleDataIO, ShuffleDriverComponents, ShuffleExecutorComponents}
import org.apache.spark.shuffle.api.ShuffleDriverComponents.MapOutputUnregistrationStrategy
import org.apache.spark.shuffle.sort.io.LocalDiskShuffleDataIO
import org.apache.spark.storage.BlockManagerId

Expand All @@ -44,16 +45,42 @@ class PluginShuffleDriverComponents(delegate: ShuffleDriverComponents)
override def removeShuffle(shuffleId: Int, blocking: Boolean): Unit =
delegate.removeShuffle(shuffleId, blocking)

override def shouldUnregisterOutputOnHostOnFetchFailure(): Boolean = true
override def unregistrationStrategyOnFetchFailure():
ShuffleDriverComponents.MapOutputUnregistrationStrategy =
MapOutputUnregistrationStrategy.HOST
}

class AsyncShuffleDataIO(sparkConf: SparkConf) extends ShuffleDataIO {
val localDiskShuffleDataIO = new LocalDiskShuffleDataIO(sparkConf)
override def driver(): ShuffleDriverComponents =
new AsyncShuffleDriverComponents(localDiskShuffleDataIO.driver())

override def executor(): ShuffleExecutorComponents = localDiskShuffleDataIO.executor()
}

class AsyncShuffleDriverComponents(delegate: ShuffleDriverComponents)
extends ShuffleDriverComponents {
override def initializeApplication(): util.Map[String, String] =
delegate.initializeApplication()

override def cleanupApplication(): Unit =
delegate.cleanupApplication()

override def removeShuffle(shuffleId: Int, blocking: Boolean): Unit =
delegate.removeShuffle(shuffleId, blocking)

override def unregistrationStrategyOnFetchFailure():
ShuffleDriverComponents.MapOutputUnregistrationStrategy =
MapOutputUnregistrationStrategy.MAP_OUTPUT_ONLY
}

class DAGSchedulerShufflePluginSuite extends DAGSchedulerSuite {

private def setupTest(): (RDD[_], Int) = {
private def setupTest(pluginClass: Class[_]): (RDD[_], Int) = {
afterEach()
val conf = new SparkConf()
// unregistering all outputs on a host is enabled for the individual file server case
conf.set(config.SHUFFLE_IO_PLUGIN_CLASS, classOf[PluginShuffleDataIO].getName)
conf.set(config.SHUFFLE_IO_PLUGIN_CLASS, pluginClass.getName)
init(conf)
val shuffleMapRdd = new MyRDD(sc, 2, Nil)
val shuffleDep = new ShuffleDependency(shuffleMapRdd, new HashPartitioner(2))
Expand All @@ -62,8 +89,32 @@ class DAGSchedulerShufflePluginSuite extends DAGSchedulerSuite {
(reduceRdd, shuffleId)
}

test("Test async") {
val (reduceRdd, shuffleId) = setupTest(classOf[AsyncShuffleDataIO])
submit(reduceRdd, Array(0, 1))

// Perform map task
val mapStatus1 = makeMapStatus("exec1", "hostA")
val mapStatus2 = makeMapStatus("exec1", "hostA")
complete(taskSets(0), Seq((Success, mapStatus1), (Success, mapStatus2)))
assertMapShuffleLocations(shuffleId, Seq(mapStatus1, mapStatus2))


// perform reduce task
complete(taskSets(1), Seq((Success, 42),
(FetchFailed(BlockManagerId("exec1", "hostA", 1234), shuffleId, 1, 0, "ignored"), null)))
assertMapShuffleLocations(shuffleId, Seq(mapStatus1, null))

scheduler.resubmitFailedStages()
complete(taskSets(2), Seq((Success, mapStatus2)))

complete(taskSets(3), Seq((Success, 43)))
assert(results === Map(0 -> 42, 1 -> 43))
assertDataStructuresEmpty()
}

test("Test simple file server") {
val (reduceRdd, shuffleId) = setupTest()
val (reduceRdd, shuffleId) = setupTest(classOf[PluginShuffleDataIO])
submit(reduceRdd, Array(0, 1))

// Perform map task
Expand All @@ -79,7 +130,7 @@ class DAGSchedulerShufflePluginSuite extends DAGSchedulerSuite {
}

test("Test simple file server fetch failure") {
val (reduceRdd, shuffleId) = setupTest()
val (reduceRdd, shuffleId) = setupTest(classOf[PluginShuffleDataIO])
submit(reduceRdd, Array(0, 1))

// Perform map task
Expand All @@ -100,7 +151,7 @@ class DAGSchedulerShufflePluginSuite extends DAGSchedulerSuite {
}

test("Test simple file fetch server - duplicate host") {
val (reduceRdd, shuffleId) = setupTest()
val (reduceRdd, shuffleId) = setupTest(classOf[PluginShuffleDataIO])
submit(reduceRdd, Array(0, 1))

// Perform map task
Expand All @@ -121,7 +172,7 @@ class DAGSchedulerShufflePluginSuite extends DAGSchedulerSuite {
}

test("Test DFS case - empty BlockManagerId") {
val (reduceRdd, shuffleId) = setupTest()
val (reduceRdd, shuffleId) = setupTest(classOf[PluginShuffleDataIO])
submit(reduceRdd, Array(0, 1))

val mapStatus = makeEmptyMapStatus()
Expand All @@ -135,7 +186,7 @@ class DAGSchedulerShufflePluginSuite extends DAGSchedulerSuite {
}

test("Test DFS case - fetch failure") {
val (reduceRdd, shuffleId) = setupTest()
val (reduceRdd, shuffleId) = setupTest(classOf[PluginShuffleDataIO])
submit(reduceRdd, Array(0, 1))

// Perform map task
Expand Down