[GitHub] [spark] agrawaldevesh commented on a change in pull request #29422: [SPARK-32613][CORE] Fix regressions in DecommissionWorkerSuite

2020-08-16 Thread GitBox


agrawaldevesh commented on a change in pull request #29422:
URL: https://github.com/apache/spark/pull/29422#discussion_r471249082



##
File path: 
core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala
##
@@ -136,7 +139,21 @@ private[spark] class TaskSchedulerImpl(
   // IDs of the tasks running on each executor
   private val executorIdToRunningTaskIds = new HashMap[String, HashSet[Long]]
 
-  private val executorsPendingDecommission = new HashMap[String, 
ExecutorDecommissionInfo]
+  // We add executors here when we first get decommission notification for 
them. Executors can
+  // continue to run even after being asked to decommission, but they will 
eventually exit.
+  val executorsPendingDecommission = new HashMap[String, 
ExecutorDecommissionInfo]
+
+  // When they exit and we know of that via heartbeat failure, we will add 
them to this cache.
+  // This cache is consulted to know if a fetch failure is because a source 
executor was
+  // decommissioned.
+  lazy val decommissionedExecutorsRemoved = CacheBuilder.newBuilder()
+.expireAfterWrite(
+  conf.getLong("spark.decommissioningRememberAfterRemoval.seconds", 60L), 
TimeUnit.SECONDS)

Review comment:
   Yeah the fetch failure is most likely going to arrive well after the 
executor is removed because the reducer typically retries before giving up with 
a fetch failure. 
   
   The idea here is to be best effort: to act on fetch failures stemming from 
decommissioned mappers and purge out all of the other mappers on that host 
asap. Ofcourse, it is possible that the fetch failure arrives too late and we 
have forgotten that this was a decommissioned host. 
   
   We are not trying to promise 100% robustness with decommissioning, just 
trying to improve it a bit. 
   
   So now one question please: what should be the value of this timeout. I am 
thinking more like 5 minutes ? Obviously the smaller it is, the less the space 
overhead. But I think 5 minutes should be fine. how much of a delay have you 
seen in practice ? What is the delay in the RPC framework we are talking about ?





This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] [spark] agrawaldevesh commented on a change in pull request #29422: [SPARK-32613][CORE] Fix regressions in DecommissionWorkerSuite

2020-08-16 Thread GitBox


agrawaldevesh commented on a change in pull request #29422:
URL: https://github.com/apache/spark/pull/29422#discussion_r471247609



##
File path: 
core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala
##
@@ -294,10 +294,15 @@ private[spark] class CoarseGrainedExecutorBackend(
 override def run(): Unit = {
   var lastTaskRunningTime = System.nanoTime()
   val sleep_time = 1000 // 1s
-
+  // This config is internal and only used by unit tests to force an 
executor
+  // to hang around for longer when decommissioned.
+  val initialSleepMillis = env.conf.getInt(
+"spark.executor.decommission.initial.sleep.millis", sleep_time)

Review comment:
   Sure. good idea !





This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] [spark] agrawaldevesh commented on a change in pull request #29422: [SPARK-32613][CORE] Fix regressions in DecommissionWorkerSuite

2020-08-15 Thread GitBox


agrawaldevesh commented on a change in pull request #29422:
URL: https://github.com/apache/spark/pull/29422#discussion_r470889645



##
File path: core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala
##
@@ -2022,16 +2024,25 @@ private[spark] class DAGScheduler(
   blockManagerMaster.removeExecutor(execId)
   clearCacheLocs()
 }
-if (fileLost &&
-(!shuffleFileLostEpoch.contains(execId) || 
shuffleFileLostEpoch(execId) < currentEpoch)) {
-  shuffleFileLostEpoch(execId) = currentEpoch
-  hostToUnregisterOutputs match {
-case Some(host) =>
-  logInfo(s"Shuffle files lost for host: $host (epoch $currentEpoch)")
-  mapOutputTracker.removeOutputsOnHost(host)
-case None =>
-  logInfo(s"Shuffle files lost for executor: $execId (epoch 
$currentEpoch)")
-  mapOutputTracker.removeOutputsOnExecutor(execId)
+if (fileLost) {

Review comment:
   Will do. Good idea.
   
   Added a comment on the caller. The changes in this function are simply 
tweaking the existing logic to honor the newly added flag. So I thought it 
would be more interesting to describe why this unconditional forcing is 
required when a host is decommissioned.





This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] [spark] agrawaldevesh commented on a change in pull request #29422: [SPARK-32613][CORE] Fix regressions in DecommissionWorkerSuite

2020-08-14 Thread GitBox


agrawaldevesh commented on a change in pull request #29422:
URL: https://github.com/apache/spark/pull/29422#discussion_r470890206



##
File path: 
core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala
##
@@ -294,10 +294,13 @@ private[spark] class CoarseGrainedExecutorBackend(
 override def run(): Unit = {
   var lastTaskRunningTime = System.nanoTime()
   val sleep_time = 1000 // 1s
-
+  val initialSleepMillis = env.conf.getInt(

Review comment:
   sure





This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] [spark] agrawaldevesh commented on a change in pull request #29422: [SPARK-32613][CORE] Fix regressions in DecommissionWorkerSuite

2020-08-14 Thread GitBox


agrawaldevesh commented on a change in pull request #29422:
URL: https://github.com/apache/spark/pull/29422#discussion_r470890156



##
File path: 
core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala
##
@@ -323,6 +326,7 @@ private[spark] class CoarseGrainedExecutorBackend(
   // move forward.
   lastTaskRunningTime = System.nanoTime()
 }
+Thread.sleep(sleep_time)

Review comment:
   Yeah. No semantic change. We are still by default waiting for sleep_time 
the first time and the last time around (it is an infinite while loop that can 
only exit via an `exit(1)` -- via process death). I just wanted the first sleep 
interval to be configurable for testing. But no production change to the 
shutdown thread. 





This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] [spark] agrawaldevesh commented on a change in pull request #29422: [SPARK-32613][CORE] Fix regressions in DecommissionWorkerSuite

2020-08-14 Thread GitBox


agrawaldevesh commented on a change in pull request #29422:
URL: https://github.com/apache/spark/pull/29422#discussion_r470889735



##
File path: core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala
##
@@ -2012,7 +2013,8 @@ private[spark] class DAGScheduler(
   execId: String,
   fileLost: Boolean,
   hostToUnregisterOutputs: Option[String],
-  maybeEpoch: Option[Long] = None): Unit = {
+  maybeEpoch: Option[Long] = None,
+  ignoreShuffleVersion: Boolean = false): Unit = {

Review comment:
   Yeah I will make it be something like `ignoreShuffleFileLostEpoch` to be 
even more explicit





This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] [spark] agrawaldevesh commented on a change in pull request #29422: [SPARK-32613][CORE] Fix regressions in DecommissionWorkerSuite

2020-08-14 Thread GitBox


agrawaldevesh commented on a change in pull request #29422:
URL: https://github.com/apache/spark/pull/29422#discussion_r470889570



##
File path: 
core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala
##
@@ -1027,7 +1036,15 @@ private[spark] class TaskSchedulerImpl(
   }
 }
 
-executorsPendingDecommission -= executorId
+
+val decomInfo = executorsPendingDecommission.get(executorId)
+if (decomInfo.isDefined) {
+  val rememberSeconds =
+conf.getInt("spark.decommissioningRememberAfterRemoval.seconds", 60)
+  val gcSecond = TimeUnit.MILLISECONDS.toSeconds(clock.getTimeMillis()) + 
rememberSeconds
+  decommissioningExecutorsToGc.computeIfAbsent(gcSecond, _ => 
mutable.ArrayBuffer.empty) +=
+executorId
+}

Review comment:
   Hmm, no. the removal code only shares the piece about 
`TimeUnit.MILLISECONDS.toSeconds(clock.getTimeMillis())`, it does not share the 
rest of the code.





This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] [spark] agrawaldevesh commented on a change in pull request #29422: [SPARK-32613][CORE] Fix regressions in DecommissionWorkerSuite

2020-08-14 Thread GitBox


agrawaldevesh commented on a change in pull request #29422:
URL: https://github.com/apache/spark/pull/29422#discussion_r470889645



##
File path: core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala
##
@@ -2022,16 +2024,25 @@ private[spark] class DAGScheduler(
   blockManagerMaster.removeExecutor(execId)
   clearCacheLocs()
 }
-if (fileLost &&
-(!shuffleFileLostEpoch.contains(execId) || 
shuffleFileLostEpoch(execId) < currentEpoch)) {
-  shuffleFileLostEpoch(execId) = currentEpoch
-  hostToUnregisterOutputs match {
-case Some(host) =>
-  logInfo(s"Shuffle files lost for host: $host (epoch $currentEpoch)")
-  mapOutputTracker.removeOutputsOnHost(host)
-case None =>
-  logInfo(s"Shuffle files lost for executor: $execId (epoch 
$currentEpoch)")
-  mapOutputTracker.removeOutputsOnExecutor(execId)
+if (fileLost) {

Review comment:
   Will do. Good idea. 





This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] [spark] agrawaldevesh commented on a change in pull request #29422: [SPARK-32613][CORE] Fix regressions in DecommissionWorkerSuite

2020-08-14 Thread GitBox


agrawaldevesh commented on a change in pull request #29422:
URL: https://github.com/apache/spark/pull/29422#discussion_r470889426



##
File path: 
core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala
##
@@ -136,7 +137,9 @@ private[spark] class TaskSchedulerImpl(
   // IDs of the tasks running on each executor
   private val executorIdToRunningTaskIds = new HashMap[String, HashSet[Long]]
 
-  private val executorsPendingDecommission = new HashMap[String, 
ExecutorDecommissionInfo]
+  val executorsPendingDecommission = new HashMap[String, 
ExecutorDecommissionInfo]
+  // map of second to list of executors to clear form the above map
+  val decommissioningExecutorsToGc = new util.TreeMap[Long, 
mutable.ArrayBuffer[String]]()

Review comment:
   Sure. Any structure that lets me GC by time will do. I just wanted 
something lightweight and custom to this use case. 
   
   I expect the treemap to contain no more than 60 seconds worth of entries 
since things are keyed by the second, and they are also cleaned up on every 
check. The check happens on every executor loss and fetch failures. But yeah it 
is possible that if there are no failures then the entries could just sit there 
:-P. 
   
   I will change it to Cache. good idea. 





This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] [spark] agrawaldevesh commented on a change in pull request #29422: [SPARK-32613][CORE] Fix regressions in DecommissionWorkerSuite

2020-08-14 Thread GitBox


agrawaldevesh commented on a change in pull request #29422:
URL: https://github.com/apache/spark/pull/29422#discussion_r470888548



##
File path: 
core/src/test/scala/org/apache/spark/deploy/DecommissionWorkerSuite.scala
##
@@ -212,22 +226,27 @@ class DecommissionWorkerSuite
   override def handleRootTaskEnd(taskEnd: SparkListenerTaskEnd): Unit = {
 val taskInfo = taskEnd.taskInfo
 if (taskInfo.executorId == executorToDecom && taskInfo.attemptNumber 
== 0 &&
-  taskEnd.stageAttemptId == 0) {
+  taskEnd.stageAttemptId == 0 && taskEnd.stageId == 0) {
   decommissionWorkerOnMaster(workerToDecom,
 "decommission worker after task on it is done")
 }
   }
 }
-TestUtils.withListener(sc, listener) { _ =>
+withListener(sc, listener) { _ =>
   val jobResult = sc.parallelize(1 to 2, 2).mapPartitionsWithIndex((_, _) 
=> {
 val executorId = SparkEnv.get.executorId
-val sleepTimeSeconds = if (executorId == executorToDecom) 10 else 1
-Thread.sleep(sleepTimeSeconds * 1000L)
+val context = TaskContext.get()
+if (context.attemptNumber() == 0 && context.stageAttemptNumber() == 0) 
{
+  val sleepTimeSeconds = if (executorId == executorToDecom) 10 else 1
+  Thread.sleep(sleepTimeSeconds * 1000L)
+}

Review comment:
   Exactly. Got tired of waiting for the test to run and trying to cut 
slack.





This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org