[GitHub] [spark] dongjoon-hyun commented on a diff in pull request #37924: [SPARK-40481][CORE] Ignore stage fetch failure caused by decommissioned executor

2022-10-09 Thread GitBox


dongjoon-hyun commented on code in PR #37924:
URL: https://github.com/apache/spark/pull/37924#discussion_r990930867


##
core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala:
##
@@ -2159,6 +2176,26 @@ private[spark] class DAGScheduler(
 }
   }
 
+  /**
+   * Whether executor is decommissioning or decommissioned.
+   * Return true when:
+   *  1. Waiting for decommission start
+   *  2. Under decommission process
+   * Return false when:
+   *  1. Stopped or terminated after finishing decommission
+   *  2. Under decommission process, then removed by driver with other reasons
+   */
+  private[scheduler] def isExecutorDecommissioningOrDecommissioned(
+  taskScheduler: TaskScheduler, bmAddress: BlockManagerId): Boolean = {
+if (bmAddress != null) {
+  taskScheduler
+.getExecutorDecommissionState(bmAddress.executorId)

Review Comment:
   Hi, @Ngone51 . It's included in the function description, isn't it?
   ```
   * Return false when:
   *  1. Stopped or terminated after finishing decommission
   *  2. Under decommission process, then removed by driver with other reasons
   */
   ```



-- 
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.

To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org

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] dongjoon-hyun commented on a diff in pull request #37924: [SPARK-40481][CORE] Ignore stage fetch failure caused by decommissioned executor

2022-09-22 Thread GitBox


dongjoon-hyun commented on code in PR #37924:
URL: https://github.com/apache/spark/pull/37924#discussion_r977436837


##
core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala:
##
@@ -1140,6 +1147,43 @@ class DAGSchedulerSuite extends SparkFunSuite with 
TempLocalSparkContext with Ti
 assertDataStructuresEmpty()
   }
 
+  test("SPARK-40481: Multiple consecutive stage fetch failures from 
decommissioned executor" +
+"should not fail job when ignoreOnDecommissionFetchFailure is enabled.") {

Review Comment:
   `ignoreOnDecommissionFetchFailure` -> `ignoreDecommissionFetchFailure`



-- 
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.

To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org

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] dongjoon-hyun commented on a diff in pull request #37924: [SPARK-40481][CORE] Ignore stage fetch failure caused by decommissioned executor

2022-09-22 Thread GitBox


dongjoon-hyun commented on code in PR #37924:
URL: https://github.com/apache/spark/pull/37924#discussion_r977436492


##
core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala:
##
@@ -1140,6 +1147,43 @@ class DAGSchedulerSuite extends SparkFunSuite with 
TempLocalSparkContext with Ti
 assertDataStructuresEmpty()
   }
 
+  test("SPARK-40481: Multiple consecutive stage fetch failures from 
decommissioned executor" +
+"should not fail job when ignoreOnDecommissionFetchFailure is enabled.") {

Review Comment:
   We need a space, `"should` -> `" should`.



-- 
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.

To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org

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] dongjoon-hyun commented on a diff in pull request #37924: [SPARK-40481][CORE] Ignore stage fetch failure caused by decommissioned executor

2022-09-22 Thread GitBox


dongjoon-hyun commented on code in PR #37924:
URL: https://github.com/apache/spark/pull/37924#discussion_r977435531


##
core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala:
##
@@ -2159,6 +2176,24 @@ private[spark] class DAGScheduler(
 }
   }
 
+  /**
+   * Whether executor is decommissioned. Return true when executors are in 
below cases:
+   *  1. Waiting for decommission start
+   *  2. Under decommission process
+   *  3. Stopped or terminated after finishing decommission
+   *  4. Under decommission process, then removed by driver with other reasons
+   */
+  private[scheduler] def isExecutorDecommissioned(

Review Comment:
   The method name should be revised according to your new comments.



-- 
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.

To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org

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] dongjoon-hyun commented on a diff in pull request #37924: [SPARK-40481][CORE] Ignore stage fetch failure caused by decommissioned executor

2022-09-22 Thread GitBox


dongjoon-hyun commented on code in PR #37924:
URL: https://github.com/apache/spark/pull/37924#discussion_r977435182


##
core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala:
##
@@ -2159,6 +2176,24 @@ private[spark] class DAGScheduler(
 }
   }
 
+  /**
+   * Whether executor is decommissioned. Return true when executors are in 
below cases:
+   *  1. Waiting for decommission start
+   *  2. Under decommission process
+   *  3. Stopped or terminated after finishing decommission
+   *  4. Under decommission process, then removed by driver with other reasons

Review Comment:
   I hope we can add comments that this is a best-effort approach and there is 
a corner case like (4) which we cannot ignore.



-- 
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.

To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org

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] dongjoon-hyun commented on a diff in pull request #37924: [SPARK-40481][CORE] Ignore stage fetch failure caused by decommissioned executor

2022-09-22 Thread GitBox


dongjoon-hyun commented on code in PR #37924:
URL: https://github.com/apache/spark/pull/37924#discussion_r977433871


##
core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala:
##
@@ -2159,6 +2176,24 @@ private[spark] class DAGScheduler(
 }
   }
 
+  /**
+   * Whether executor is decommissioned. Return true when executors are in 
below cases:
+   *  1. Waiting for decommission start
+   *  2. Under decommission process
+   *  3. Stopped or terminated after finishing decommission
+   *  4. Under decommission process, then removed by driver with other reasons

Review Comment:
   Thank you for updates, but what I mean was that this AS-IS method is not 
aware of Case (4), @warrenzhu25 .



-- 
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.

To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org

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] dongjoon-hyun commented on a diff in pull request #37924: [SPARK-40481][CORE] Ignore stage fetch failure caused by decommissioned executor

2022-09-22 Thread GitBox


dongjoon-hyun commented on code in PR #37924:
URL: https://github.com/apache/spark/pull/37924#discussion_r977433871


##
core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala:
##
@@ -2159,6 +2176,24 @@ private[spark] class DAGScheduler(
 }
   }
 
+  /**
+   * Whether executor is decommissioned. Return true when executors are in 
below cases:
+   *  1. Waiting for decommission start
+   *  2. Under decommission process
+   *  3. Stopped or terminated after finishing decommission
+   *  4. Under decommission process, then removed by driver with other reasons

Review Comment:
   Thank you for updates, but what I mean was that this AS-IS method is not 
aware of Case (3) and (4), @warrenzhu25 .



-- 
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.

To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org

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] dongjoon-hyun commented on a diff in pull request #37924: [SPARK-40481][CORE] Ignore stage fetch failure caused by decommissioned executor

2022-09-19 Thread GitBox


dongjoon-hyun commented on code in PR #37924:
URL: https://github.com/apache/spark/pull/37924#discussion_r974893716


##
core/src/main/scala/org/apache/spark/internal/config/package.scala:
##
@@ -2221,6 +2221,14 @@ package object config {
   .checkValue(_ >= 0, "needs to be a non-negative value")
   .createWithDefault(5)
 
+  private[spark] val STAGE_IGNORE_DECOMMISSION_FETCH_FAILURE =
+ConfigBuilder("spark.stage.ignoreOnDecommissionFetchFailure")

Review Comment:
   Shall we remove the redundant `On`? It would be better because the variable 
name already have no `On`.
   ```
   - ignoreOnDecommissionFetchFailure
   + ignoreDecommissionFetchFailure
   ```



-- 
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.

To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org

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] dongjoon-hyun commented on a diff in pull request #37924: [SPARK-40481][CORE] Ignore stage fetch failure caused by decommissioned executor

2022-09-19 Thread GitBox


dongjoon-hyun commented on code in PR #37924:
URL: https://github.com/apache/spark/pull/37924#discussion_r974893716


##
core/src/main/scala/org/apache/spark/internal/config/package.scala:
##
@@ -2221,6 +2221,14 @@ package object config {
   .checkValue(_ >= 0, "needs to be a non-negative value")
   .createWithDefault(5)
 
+  private[spark] val STAGE_IGNORE_DECOMMISSION_FETCH_FAILURE =
+ConfigBuilder("spark.stage.ignoreOnDecommissionFetchFailure")

Review Comment:
   Shall we remove the redundant `On`?
   ```
   - ignoreOnDecommissionFetchFailure
   + ignoreDecommissionFetchFailure
   ```



-- 
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.

To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org

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] dongjoon-hyun commented on a diff in pull request #37924: [SPARK-40481][CORE] Ignore stage fetch failure caused by decommissioned executor

2022-09-19 Thread GitBox


dongjoon-hyun commented on code in PR #37924:
URL: https://github.com/apache/spark/pull/37924#discussion_r974491025


##
core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala:
##
@@ -2159,6 +2171,16 @@ private[spark] class DAGScheduler(
 }
   }
 
+  private def isExecutorDecommissioned(bmAddress: BlockManagerId) = {

Review Comment:
   If you don't mind, please add `taskScheduler` as a parameter here and use it 
inside function body.



-- 
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.

To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org

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] dongjoon-hyun commented on a diff in pull request #37924: [SPARK-40481][CORE] Ignore stage fetch failure caused by decommissioned executor

2022-09-19 Thread GitBox


dongjoon-hyun commented on code in PR #37924:
URL: https://github.com/apache/spark/pull/37924#discussion_r974490653


##
core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala:
##
@@ -1860,8 +1863,17 @@ private[spark] class DAGScheduler(
 s"(attempt ${failedStage.latestInfo.attemptNumber}) running")
 } else {
   failedStage.failedAttemptIds.add(task.stageAttemptId)
+  val ignoreStageFailure = ignoreDecommissionFetchFailure &&
+isExecutorDecommissioned(bmAddress)
+  if (ignoreStageFailure) {
+logInfo("Ignoring fetch failure from $task of $failedStage 
attempt" +

Review Comment:
   This log message might be misleading or wrong when 
`disallowStageRetryForTest` is true.



-- 
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.

To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org

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] dongjoon-hyun commented on a diff in pull request #37924: [SPARK-40481][CORE] Ignore stage fetch failure caused by decommissioned executor

2022-09-19 Thread GitBox


dongjoon-hyun commented on code in PR #37924:
URL: https://github.com/apache/spark/pull/37924#discussion_r974486551


##
docs/configuration.md:
##
@@ -2605,6 +2605,15 @@ Apart from these, the following properties are also 
available, and may be useful
   
   2.2.0
 
+
+  spark.stage.attempt.ignoreOnDecommissionFetchFailure
+  false
+  
+Whether ignore stage fetch failure caused by executor decommission when 

Review Comment:
   Please remove the trailing space at the end of this line.



-- 
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.

To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org

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] dongjoon-hyun commented on a diff in pull request #37924: [SPARK-40481][CORE] Ignore stage fetch failure caused by decommissioned executor

2022-09-19 Thread GitBox


dongjoon-hyun commented on code in PR #37924:
URL: https://github.com/apache/spark/pull/37924#discussion_r974483075


##
core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala:
##
@@ -1860,8 +1863,17 @@ private[spark] class DAGScheduler(
 s"(attempt ${failedStage.latestInfo.attemptNumber}) running")
 } else {
   failedStage.failedAttemptIds.add(task.stageAttemptId)
+  val ignoreStageFailure = ignoreDecommissionFetchFailure &&
+isExecutorDecommissioned(bmAddress)
+  if (ignoreStageFailure) {
+logInfo("Ignoring fetch failure from $task of $failedStage 
attempt" +
+  s"${task.stageAttemptId} as executor ${bmAddress.executorId} is 
decommissioned and " +
+  s" ${config.STAGE_IGNORE_DECOMMISSION_FETCH_FAILURE.key}=true")
+  }
+
   val shouldAbortStage =
-failedStage.failedAttemptIds.size >= maxConsecutiveStageAttempts ||
+(!ignoreStageFailure &&
+  failedStage.failedAttemptIds.size >= 
maxConsecutiveStageAttempts)||

Review Comment:
   `)||` -> `) ||`?



-- 
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.

To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org

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] dongjoon-hyun commented on a diff in pull request #37924: [SPARK-40481][CORE] Ignore stage fetch failure caused by decommissioned executor

2022-09-19 Thread GitBox


dongjoon-hyun commented on code in PR #37924:
URL: https://github.com/apache/spark/pull/37924#discussion_r974479715


##
core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala:
##
@@ -230,6 +230,9 @@ private[spark] class DAGScheduler(
 sc.getConf.getInt("spark.stage.maxConsecutiveAttempts",
   DAGScheduler.DEFAULT_MAX_CONSECUTIVE_STAGE_ATTEMPTS)
 
+  private[scheduler] val ignoreDecommissionFetchFailure =

Review Comment:
   Could you add a comment like the following?
   
https://github.com/apache/spark/blob/2d6d5e2c44861745e4a7f5a4c15f74337f651b03/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala#L226-L228



-- 
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.

To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org

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] dongjoon-hyun commented on a diff in pull request #37924: [SPARK-40481][CORE] Ignore stage fetch failure caused by decommissioned executor

2022-09-18 Thread GitBox


dongjoon-hyun commented on code in PR #37924:
URL: https://github.com/apache/spark/pull/37924#discussion_r973809002


##
core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala:
##
@@ -1140,6 +1146,38 @@ class DAGSchedulerSuite extends SparkFunSuite with 
TempLocalSparkContext with Ti
 assertDataStructuresEmpty()
   }
 
+  test("SPARK-40481: Multiple consecutive stage fetch failures should not fail 
job " +
+"when ignoreOnDecommissionFetchFailure is enabled.") {
+conf.set(config.STAGE_IGNORE_DECOMMISSION_FETCH_FAILURE.key, "true")
+
+setupStageAbortTest(sc)
+val parts = 2
+val shuffleMapRdd = new MyRDD(sc, parts, Nil)
+val shuffleDep = new ShuffleDependency(shuffleMapRdd, new 
HashPartitioner(parts))
+val reduceRdd = new MyRDD(sc, parts, List(shuffleDep), tracker = 
mapOutputTracker)
+submit(reduceRdd, (0 until parts).toArray)
+
+for (attempt <- 0 until scheduler.maxConsecutiveStageAttempts) {
+  // Complete all the tasks for the current attempt of stage 0 successfully
+  completeShuffleMapStageSuccessfully(0, attempt, numShufflePartitions = 
parts,
+Seq("hostA", "hostB"))
+
+  taskScheduler.executorDecommission("hostA-exec", 
ExecutorDecommissionInfo(""))
+  // Now we should have a new taskSet, for a new attempt of stage 1.
+  // Fail all these tasks with FetchFailure
+  completeNextStageWithFetchFailure(1, attempt, shuffleDep)
+
+  // this will trigger a resubmission of stage 0, since we've lost some of 
its
+  // map output, for the next iteration through the loop
+  scheduler.resubmitFailedStages()
+   }
+
+// Confirm job finished successfully
+sc.listenerBus.waitUntilEmpty()
+assert(scheduler.runningStages.nonEmpty)
+assert(!ended)
+

Review Comment:
   Shall we clean up this redundant empty line?



-- 
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.

To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org

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] dongjoon-hyun commented on a diff in pull request #37924: [SPARK-40481][CORE] Ignore stage fetch failure caused by decommissioned executor

2022-09-18 Thread GitBox


dongjoon-hyun commented on code in PR #37924:
URL: https://github.com/apache/spark/pull/37924#discussion_r973808911


##
core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala:
##
@@ -1140,6 +1146,38 @@ class DAGSchedulerSuite extends SparkFunSuite with 
TempLocalSparkContext with Ti
 assertDataStructuresEmpty()
   }
 
+  test("SPARK-40481: Multiple consecutive stage fetch failures should not fail 
job " +
+"when ignoreOnDecommissionFetchFailure is enabled.") {

Review Comment:
   Could you revise the test case name? We are not ignoring the fetch failure 
from the normal executors.



-- 
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.

To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org

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] dongjoon-hyun commented on a diff in pull request #37924: [SPARK-40481][CORE] Ignore stage fetch failure caused by decommissioned executor

2022-09-18 Thread GitBox


dongjoon-hyun commented on code in PR #37924:
URL: https://github.com/apache/spark/pull/37924#discussion_r973808705


##
core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala:
##
@@ -1853,15 +1856,31 @@ private[spark] class DAGScheduler(
   case FetchFailed(bmAddress, shuffleId, _, mapIndex, reduceId, 
failureMessage) =>
 val failedStage = stageIdToStage(task.stageId)
 val mapStage = shuffleIdToMapStage(shuffleId)
+val isExecutorDecommissioned =

Review Comment:
   We should not have this overhead here. Could you move this to the `else` 
statement which needs this variable?



-- 
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.

To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org

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] dongjoon-hyun commented on a diff in pull request #37924: [SPARK-40481][CORE] Ignore stage fetch failure caused by decommissioned executor

2022-09-18 Thread GitBox


dongjoon-hyun commented on code in PR #37924:
URL: https://github.com/apache/spark/pull/37924#discussion_r973808498


##
core/src/main/scala/org/apache/spark/internal/config/package.scala:
##
@@ -2221,6 +2221,14 @@ package object config {
   .checkValue(_ >= 0, "needs to be a non-negative value")
   .createWithDefault(5)
 
+  private[spark] val STAGE_IGNORE_DECOMMISSION_FETCH_FAILURE =
+ConfigBuilder("spark.stage.attempt.ignoreOnDecommissionFetchFailure")

Review Comment:
   We need a new proper name. Please ignore Amazon configuration name.



-- 
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.

To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org

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] dongjoon-hyun commented on a diff in pull request #37924: [SPARK-40481][CORE] Ignore stage fetch failure caused by decommissioned executor

2022-09-18 Thread GitBox


dongjoon-hyun commented on code in PR #37924:
URL: https://github.com/apache/spark/pull/37924#discussion_r973807828


##
core/src/main/scala/org/apache/spark/internal/config/package.scala:
##
@@ -2221,6 +2221,14 @@ package object config {
   .checkValue(_ >= 0, "needs to be a non-negative value")
   .createWithDefault(5)
 
+  private[spark] val STAGE_IGNORE_DECOMMISSION_FETCH_FAILURE =
+ConfigBuilder("spark.stage.attempt.ignoreOnDecommissionFetchFailure")
+  .doc("Whether ignore stage fetch failure caused by executor decommission 
when " +
+  "count spark.stage.maxConsecutiveAttempts")
+  .version("3.4.0")
+  .booleanConf
+  .createWithDefault(false)

Review Comment:
   Thank you for starting this with `false` first. We can enable this after one 
release cycle.



-- 
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.

To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org

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] dongjoon-hyun commented on a diff in pull request #37924: [SPARK-40481][CORE] Ignore stage fetch failure caused by decommissioned executor

2022-09-18 Thread GitBox


dongjoon-hyun commented on code in PR #37924:
URL: https://github.com/apache/spark/pull/37924#discussion_r973807726


##
core/src/main/scala/org/apache/spark/internal/config/package.scala:
##
@@ -2221,6 +2221,14 @@ package object config {
   .checkValue(_ >= 0, "needs to be a non-negative value")
   .createWithDefault(5)
 
+  private[spark] val STAGE_IGNORE_DECOMMISSION_FETCH_FAILURE =
+ConfigBuilder("spark.stage.attempt.ignoreOnDecommissionFetchFailure")
+  .doc("Whether ignore stage fetch failure caused by executor decommission 
when " +
+  "count spark.stage.maxConsecutiveAttempts")

Review Comment:
   indentation?



-- 
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.

To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org

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