aboutsummaryrefslogtreecommitdiff
path: root/core/src/main
diff options
context:
space:
mode:
authorEric Liang <ekl@databricks.com>2016-09-07 12:33:50 -0700
committerJosh Rosen <joshrosen@databricks.com>2016-09-07 12:33:50 -0700
commit649fa4bf1d6fc9271ae56b6891bc93ebf57858d1 (patch)
tree90abb06091264b81769757b18a84e710a2ea7989 /core/src/main
parent76ad89e9241fb2dece95dd445661dd95ee4ef699 (diff)
downloadspark-649fa4bf1d6fc9271ae56b6891bc93ebf57858d1.tar.gz
spark-649fa4bf1d6fc9271ae56b6891bc93ebf57858d1.tar.bz2
spark-649fa4bf1d6fc9271ae56b6891bc93ebf57858d1.zip
[SPARK-17370] Shuffle service files not invalidated when a slave is lost
## What changes were proposed in this pull request? DAGScheduler invalidates shuffle files when an executor loss event occurs, but not when the external shuffle service is enabled. This is because when shuffle service is on, the shuffle file lifetime can exceed the executor lifetime. However, it also doesn't invalidate shuffle files when the shuffle service itself is lost (due to whole slave loss). This can cause long hangs when slaves are lost since the file loss is not detected until a subsequent stage attempts to read the shuffle files. The proposed fix is to also invalidate shuffle files when an executor is lost due to a `SlaveLost` event. ## How was this patch tested? Unit tests, also verified on an actual cluster that slave loss invalidates shuffle files immediately as expected. cc mateiz Author: Eric Liang <ekl@databricks.com> Closes #14931 from ericl/sc-4439.
Diffstat (limited to 'core/src/main')
-rw-r--r--core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala2
-rw-r--r--core/src/main/scala/org/apache/spark/deploy/client/StandaloneAppClient.scala4
-rw-r--r--core/src/main/scala/org/apache/spark/deploy/client/StandaloneAppClientListener.scala3
-rw-r--r--core/src/main/scala/org/apache/spark/deploy/master/Master.scala4
-rw-r--r--core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala24
-rw-r--r--core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala3
-rw-r--r--core/src/main/scala/org/apache/spark/scheduler/ExecutorLossReason.scala6
-rw-r--r--core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala9
-rw-r--r--core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala5
9 files changed, 38 insertions, 22 deletions
diff --git a/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala b/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala
index 34c0696bfc..ac09c6c497 100644
--- a/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala
@@ -135,7 +135,7 @@ private[deploy] object DeployMessages {
}
case class ExecutorUpdated(id: Int, state: ExecutorState, message: Option[String],
- exitStatus: Option[Int])
+ exitStatus: Option[Int], workerLost: Boolean)
case class ApplicationRemoved(message: String)
diff --git a/core/src/main/scala/org/apache/spark/deploy/client/StandaloneAppClient.scala b/core/src/main/scala/org/apache/spark/deploy/client/StandaloneAppClient.scala
index 7a60f08aad..93f58ce637 100644
--- a/core/src/main/scala/org/apache/spark/deploy/client/StandaloneAppClient.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/client/StandaloneAppClient.scala
@@ -174,12 +174,12 @@ private[spark] class StandaloneAppClient(
cores))
listener.executorAdded(fullId, workerId, hostPort, cores, memory)
- case ExecutorUpdated(id, state, message, exitStatus) =>
+ case ExecutorUpdated(id, state, message, exitStatus, workerLost) =>
val fullId = appId + "/" + id
val messageText = message.map(s => " (" + s + ")").getOrElse("")
logInfo("Executor updated: %s is now %s%s".format(fullId, state, messageText))
if (ExecutorState.isFinished(state)) {
- listener.executorRemoved(fullId, message.getOrElse(""), exitStatus)
+ listener.executorRemoved(fullId, message.getOrElse(""), exitStatus, workerLost)
}
case MasterChanged(masterRef, masterWebUiUrl) =>
diff --git a/core/src/main/scala/org/apache/spark/deploy/client/StandaloneAppClientListener.scala b/core/src/main/scala/org/apache/spark/deploy/client/StandaloneAppClientListener.scala
index 370b16ce42..64255ec92b 100644
--- a/core/src/main/scala/org/apache/spark/deploy/client/StandaloneAppClientListener.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/client/StandaloneAppClientListener.scala
@@ -36,5 +36,6 @@ private[spark] trait StandaloneAppClientListener {
def executorAdded(
fullId: String, workerId: String, hostPort: String, cores: Int, memory: Int): Unit
- def executorRemoved(fullId: String, message: String, exitStatus: Option[Int]): Unit
+ def executorRemoved(
+ fullId: String, message: String, exitStatus: Option[Int], workerLost: Boolean): Unit
}
diff --git a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala
index dfffc47703..dcf41638e7 100644
--- a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala
@@ -252,7 +252,7 @@ private[deploy] class Master(
appInfo.resetRetryCount()
}
- exec.application.driver.send(ExecutorUpdated(execId, state, message, exitStatus))
+ exec.application.driver.send(ExecutorUpdated(execId, state, message, exitStatus, false))
if (ExecutorState.isFinished(state)) {
// Remove this executor from the worker and app
@@ -766,7 +766,7 @@ private[deploy] class Master(
for (exec <- worker.executors.values) {
logInfo("Telling app of lost executor: " + exec.id)
exec.application.driver.send(ExecutorUpdated(
- exec.id, ExecutorState.LOST, Some("worker lost"), None))
+ exec.id, ExecutorState.LOST, Some("worker lost"), None, workerLost = true))
exec.state = ExecutorState.LOST
exec.application.removeExecutor(exec)
}
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 4eb7c81f9e..dd47c1dbbe 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala
@@ -239,8 +239,8 @@ class DAGScheduler(
/**
* Called by TaskScheduler implementation when an executor fails.
*/
- def executorLost(execId: String): Unit = {
- eventProcessLoop.post(ExecutorLost(execId))
+ def executorLost(execId: String, reason: ExecutorLossReason): Unit = {
+ eventProcessLoop.post(ExecutorLost(execId, reason))
}
/**
@@ -1281,7 +1281,7 @@ class DAGScheduler(
// TODO: mark the executor as failed only if there were lots of fetch failures on it
if (bmAddress != null) {
- handleExecutorLost(bmAddress.executorId, fetchFailed = true, Some(task.epoch))
+ handleExecutorLost(bmAddress.executorId, filesLost = true, Some(task.epoch))
}
}
@@ -1306,15 +1306,16 @@ class DAGScheduler(
* modify the scheduler's internal state. Use executorLost() to post a loss event from outside.
*
* We will also assume that we've lost all shuffle blocks associated with the executor if the
- * executor serves its own blocks (i.e., we're not using external shuffle) OR a FetchFailed
- * occurred, in which case we presume all shuffle data related to this executor to be lost.
+ * executor serves its own blocks (i.e., we're not using external shuffle), the entire slave
+ * is lost (likely including the shuffle service), or a FetchFailed occurred, in which case we
+ * presume all shuffle data related to this executor to be lost.
*
* Optionally the epoch during which the failure was caught can be passed to avoid allowing
* stray fetch failures from possibly retriggering the detection of a node as lost.
*/
private[scheduler] def handleExecutorLost(
execId: String,
- fetchFailed: Boolean,
+ filesLost: Boolean,
maybeEpoch: Option[Long] = None) {
val currentEpoch = maybeEpoch.getOrElse(mapOutputTracker.getEpoch)
if (!failedEpoch.contains(execId) || failedEpoch(execId) < currentEpoch) {
@@ -1322,7 +1323,8 @@ class DAGScheduler(
logInfo("Executor lost: %s (epoch %d)".format(execId, currentEpoch))
blockManagerMaster.removeExecutor(execId)
- if (!env.blockManager.externalShuffleServiceEnabled || fetchFailed) {
+ if (filesLost || !env.blockManager.externalShuffleServiceEnabled) {
+ logInfo("Shuffle files lost for executor: %s (epoch %d)".format(execId, currentEpoch))
// TODO: This will be really slow if we keep accumulating shuffle map stages
for ((shuffleId, stage) <- shuffleIdToMapStage) {
stage.removeOutputsOnExecutor(execId)
@@ -1624,8 +1626,12 @@ private[scheduler] class DAGSchedulerEventProcessLoop(dagScheduler: DAGScheduler
case ExecutorAdded(execId, host) =>
dagScheduler.handleExecutorAdded(execId, host)
- case ExecutorLost(execId) =>
- dagScheduler.handleExecutorLost(execId, fetchFailed = false)
+ case ExecutorLost(execId, reason) =>
+ val filesLost = reason match {
+ case SlaveLost(_, true) => true
+ case _ => false
+ }
+ dagScheduler.handleExecutorLost(execId, filesLost)
case BeginEvent(task, taskInfo) =>
dagScheduler.handleBeginEvent(task, taskInfo)
diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala
index 8c76112482..03781a2a2b 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala
@@ -77,7 +77,8 @@ private[scheduler] case class CompletionEvent(
private[scheduler] case class ExecutorAdded(execId: String, host: String) extends DAGSchedulerEvent
-private[scheduler] case class ExecutorLost(execId: String) extends DAGSchedulerEvent
+private[scheduler] case class ExecutorLost(execId: String, reason: ExecutorLossReason)
+ extends DAGSchedulerEvent
private[scheduler]
case class TaskSetFailed(taskSet: TaskSet, reason: String, exception: Option[Throwable])
diff --git a/core/src/main/scala/org/apache/spark/scheduler/ExecutorLossReason.scala b/core/src/main/scala/org/apache/spark/scheduler/ExecutorLossReason.scala
index 642bf81ac0..46a35b6a2e 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/ExecutorLossReason.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/ExecutorLossReason.scala
@@ -51,6 +51,10 @@ private[spark] object ExecutorKilled extends ExecutorLossReason("Executor killed
*/
private [spark] object LossReasonPending extends ExecutorLossReason("Pending loss reason.")
+/**
+ * @param _message human readable loss reason
+ * @param workerLost whether the worker is confirmed lost too (i.e. including shuffle service)
+ */
private[spark]
-case class SlaveLost(_message: String = "Slave lost")
+case class SlaveLost(_message: String = "Slave lost", workerLost: Boolean = false)
extends ExecutorLossReason(_message)
diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala
index 7d905538c6..ee5cbfeb47 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala
@@ -346,6 +346,7 @@ private[spark] class TaskSchedulerImpl(
def statusUpdate(tid: Long, state: TaskState, serializedData: ByteBuffer) {
var failedExecutor: Option[String] = None
+ var reason: Option[ExecutorLossReason] = None
synchronized {
try {
if (state == TaskState.LOST && taskIdToExecutorId.contains(tid)) {
@@ -353,8 +354,9 @@ private[spark] class TaskSchedulerImpl(
val execId = taskIdToExecutorId(tid)
if (executorIdToTaskCount.contains(execId)) {
- removeExecutor(execId,
+ reason = Some(
SlaveLost(s"Task $tid was lost, so marking the executor as lost as well."))
+ removeExecutor(execId, reason.get)
failedExecutor = Some(execId)
}
}
@@ -387,7 +389,8 @@ private[spark] class TaskSchedulerImpl(
}
// Update the DAGScheduler without holding a lock on this, since that can deadlock
if (failedExecutor.isDefined) {
- dagScheduler.executorLost(failedExecutor.get)
+ assert(reason.isDefined)
+ dagScheduler.executorLost(failedExecutor.get, reason.get)
backend.reviveOffers()
}
}
@@ -513,7 +516,7 @@ private[spark] class TaskSchedulerImpl(
}
// Call dagScheduler.executorLost without holding the lock on this to prevent deadlock
if (failedExecutor.isDefined) {
- dagScheduler.executorLost(failedExecutor.get)
+ dagScheduler.executorLost(failedExecutor.get, reason)
backend.reviveOffers()
}
}
diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala
index 5068bf2e66..04d40e2907 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala
@@ -150,10 +150,11 @@ private[spark] class StandaloneSchedulerBackend(
fullId, hostPort, cores, Utils.megabytesToString(memory)))
}
- override def executorRemoved(fullId: String, message: String, exitStatus: Option[Int]) {
+ override def executorRemoved(
+ fullId: String, message: String, exitStatus: Option[Int], workerLost: Boolean) {
val reason: ExecutorLossReason = exitStatus match {
case Some(code) => ExecutorExited(code, exitCausedByApp = true, message)
- case None => SlaveLost(message)
+ case None => SlaveLost(message, workerLost = workerLost)
}
logInfo("Executor %s removed: %s".format(fullId, message))
removeExecutor(fullId.split("/")(1), reason)