[GitHub] [spark] mridulm commented on a diff in pull request #40393: [SPARK-40082] Schedule mergeFinalize when push merge shuffleMapStage retry but no running tasks

2023-03-20 Thread via GitHub


mridulm commented on code in PR #40393:
URL: https://github.com/apache/spark/pull/40393#discussion_r1142572022


##
core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala:
##
@@ -4595,6 +4595,184 @@ class DAGSchedulerSuite extends SparkFunSuite with 
TempLocalSparkContext with Ti
 }
   }
 
+  test("SPARK-40082: recomputation of shuffle map stage with no pending 
partitions" +
+"should not hang") {
+
+initPushBasedShuffleConfs(conf)
+conf.set(config.SHUFFLE_MERGER_LOCATIONS_MIN_STATIC_THRESHOLD, 3)
+DAGSchedulerSuite.clearMergerLocs()
+DAGSchedulerSuite.addMergerLocs(Seq("host1", "host2", "host3"))
+
+val rddA = new MyRDD(sc, 2, Nil)
+val shuffleDepA = new ShuffleDependency(rddA, new HashPartitioner(2))
+val shuffleIdA = shuffleDepA.shuffleId
+
+val rddB = new MyRDD(sc, 2, List(shuffleDepA), tracker = mapOutputTracker)
+val shuffleDepB = new ShuffleDependency(rddB, new HashPartitioner(2))
+
+val rddC = new MyRDD(sc, 2, List(shuffleDepB), tracker = mapOutputTracker)
+
+submit(rddC, Array(0, 1))
+
+completeShuffleMapStageSuccessfully(0, 0, 2, Seq("hostA", "hostA"))
+
+// Fetch failed
+runEvent(makeCompletionEvent(
+  taskSets(1).tasks(0),
+  FetchFailed(makeBlockManagerId("hostC"), shuffleIdA, 0L, 0, 0,
+"Fetch failure of task: stageId=1, stageAttempt=0, partitionId=0"),
+  result = null))
+
+// long running task complete
+completeShuffleMapStageSuccessfully(1, 0, 2, Seq("hostA", "hostA"))
+assert(!shuffleDepB.shuffleMergeFinalized)
+
+// stage1`s tasks have all completed
+val shuffleStage1 = 
scheduler.stageIdToStage(1).asInstanceOf[ShuffleMapStage]
+assert(shuffleStage1.pendingPartitions.isEmpty)
+
+// resubmit
+scheduler.resubmitFailedStages()
+
+// complete parentStage0
+completeShuffleMapStageSuccessfully(0, 0, 2, Seq("hostA", "hostA"))
+
+// stage1 should be shuffleMergeFinalized
+assert(shuffleDepB.shuffleMergeFinalized)
+  }
+
+  for (pushBasedShuffleEnabled <- Seq(true, false)) {
+test("SPARK-40082: recomputation of shuffle map stage with no pending 
partitions should not " +
+  s"hang. pushBasedShuffleEnabled = $pushBasedShuffleEnabled") {

Review Comment:
   Good question - I have not tried that :-) This is a pattern used for other 
tests as well when we want to do a config sweep.
   Does specifying pushBasedShuffleEnabled = true in that string work ?



-- 
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] mridulm commented on a diff in pull request #40393: [SPARK-40082] Schedule mergeFinalize when push merge shuffleMapStage retry but no running tasks

2023-03-20 Thread via GitHub


mridulm commented on code in PR #40393:
URL: https://github.com/apache/spark/pull/40393#discussion_r1142572022


##
core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala:
##
@@ -4595,6 +4595,184 @@ class DAGSchedulerSuite extends SparkFunSuite with 
TempLocalSparkContext with Ti
 }
   }
 
+  test("SPARK-40082: recomputation of shuffle map stage with no pending 
partitions" +
+"should not hang") {
+
+initPushBasedShuffleConfs(conf)
+conf.set(config.SHUFFLE_MERGER_LOCATIONS_MIN_STATIC_THRESHOLD, 3)
+DAGSchedulerSuite.clearMergerLocs()
+DAGSchedulerSuite.addMergerLocs(Seq("host1", "host2", "host3"))
+
+val rddA = new MyRDD(sc, 2, Nil)
+val shuffleDepA = new ShuffleDependency(rddA, new HashPartitioner(2))
+val shuffleIdA = shuffleDepA.shuffleId
+
+val rddB = new MyRDD(sc, 2, List(shuffleDepA), tracker = mapOutputTracker)
+val shuffleDepB = new ShuffleDependency(rddB, new HashPartitioner(2))
+
+val rddC = new MyRDD(sc, 2, List(shuffleDepB), tracker = mapOutputTracker)
+
+submit(rddC, Array(0, 1))
+
+completeShuffleMapStageSuccessfully(0, 0, 2, Seq("hostA", "hostA"))
+
+// Fetch failed
+runEvent(makeCompletionEvent(
+  taskSets(1).tasks(0),
+  FetchFailed(makeBlockManagerId("hostC"), shuffleIdA, 0L, 0, 0,
+"Fetch failure of task: stageId=1, stageAttempt=0, partitionId=0"),
+  result = null))
+
+// long running task complete
+completeShuffleMapStageSuccessfully(1, 0, 2, Seq("hostA", "hostA"))
+assert(!shuffleDepB.shuffleMergeFinalized)
+
+// stage1`s tasks have all completed
+val shuffleStage1 = 
scheduler.stageIdToStage(1).asInstanceOf[ShuffleMapStage]
+assert(shuffleStage1.pendingPartitions.isEmpty)
+
+// resubmit
+scheduler.resubmitFailedStages()
+
+// complete parentStage0
+completeShuffleMapStageSuccessfully(0, 0, 2, Seq("hostA", "hostA"))
+
+// stage1 should be shuffleMergeFinalized
+assert(shuffleDepB.shuffleMergeFinalized)
+  }
+
+  for (pushBasedShuffleEnabled <- Seq(true, false)) {
+test("SPARK-40082: recomputation of shuffle map stage with no pending 
partitions should not " +
+  s"hang. pushBasedShuffleEnabled = $pushBasedShuffleEnabled") {

Review Comment:
   Good question - I have not tried that :-)
   Does specifying pushBasedShuffleEnabled = true in that string work ?



-- 
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] mridulm commented on a diff in pull request #40393: [SPARK-40082] Schedule mergeFinalize when push merge shuffleMapStage retry but no running tasks

2023-03-20 Thread via GitHub


mridulm commented on code in PR #40393:
URL: https://github.com/apache/spark/pull/40393#discussion_r1142362530


##
core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala:
##
@@ -4595,6 +4595,184 @@ class DAGSchedulerSuite extends SparkFunSuite with 
TempLocalSparkContext with Ti
 }
   }
 
+  test("SPARK-40082: recomputation of shuffle map stage with no pending 
partitions" +
+"should not hang") {
+
+initPushBasedShuffleConfs(conf)
+conf.set(config.SHUFFLE_MERGER_LOCATIONS_MIN_STATIC_THRESHOLD, 3)
+DAGSchedulerSuite.clearMergerLocs()
+DAGSchedulerSuite.addMergerLocs(Seq("host1", "host2", "host3"))
+
+val rddA = new MyRDD(sc, 2, Nil)
+val shuffleDepA = new ShuffleDependency(rddA, new HashPartitioner(2))
+val shuffleIdA = shuffleDepA.shuffleId
+
+val rddB = new MyRDD(sc, 2, List(shuffleDepA), tracker = mapOutputTracker)
+val shuffleDepB = new ShuffleDependency(rddB, new HashPartitioner(2))
+
+val rddC = new MyRDD(sc, 2, List(shuffleDepB), tracker = mapOutputTracker)
+
+submit(rddC, Array(0, 1))
+
+completeShuffleMapStageSuccessfully(0, 0, 2, Seq("hostA", "hostA"))

Review Comment:
   nit: For consistency, let us make it `hostA` and `hostB` ... and have 
`hostA` (say) fail.



##
core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala:
##
@@ -4595,6 +4595,184 @@ class DAGSchedulerSuite extends SparkFunSuite with 
TempLocalSparkContext with Ti
 }
   }
 
+  test("SPARK-40082: recomputation of shuffle map stage with no pending 
partitions" +
+"should not hang") {
+
+initPushBasedShuffleConfs(conf)
+conf.set(config.SHUFFLE_MERGER_LOCATIONS_MIN_STATIC_THRESHOLD, 3)
+DAGSchedulerSuite.clearMergerLocs()
+DAGSchedulerSuite.addMergerLocs(Seq("host1", "host2", "host3"))
+
+val rddA = new MyRDD(sc, 2, Nil)
+val shuffleDepA = new ShuffleDependency(rddA, new HashPartitioner(2))
+val shuffleIdA = shuffleDepA.shuffleId
+
+val rddB = new MyRDD(sc, 2, List(shuffleDepA), tracker = mapOutputTracker)
+val shuffleDepB = new ShuffleDependency(rddB, new HashPartitioner(2))
+
+val rddC = new MyRDD(sc, 2, List(shuffleDepB), tracker = mapOutputTracker)
+
+submit(rddC, Array(0, 1))
+
+completeShuffleMapStageSuccessfully(0, 0, 2, Seq("hostA", "hostA"))
+
+// Fetch failed
+runEvent(makeCompletionEvent(
+  taskSets(1).tasks(0),
+  FetchFailed(makeBlockManagerId("hostC"), shuffleIdA, 0L, 0, 0,

Review Comment:
   nit: Since map tasks did not run on `hostC`, let us change it to `hostA` as 
per comment above.



##
pom.xml:
##
@@ -114,7 +114,7 @@
 1.8
 ${java.version}
 ${java.version}
-3.8.7
+3.6.3

Review Comment:
   Revert this ?



##
core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala:
##
@@ -4595,6 +4595,184 @@ class DAGSchedulerSuite extends SparkFunSuite with 
TempLocalSparkContext with Ti
 }
   }
 
+  test("SPARK-40082: recomputation of shuffle map stage with no pending 
partitions" +
+"should not hang") {
+
+initPushBasedShuffleConfs(conf)
+conf.set(config.SHUFFLE_MERGER_LOCATIONS_MIN_STATIC_THRESHOLD, 3)
+DAGSchedulerSuite.clearMergerLocs()
+DAGSchedulerSuite.addMergerLocs(Seq("host1", "host2", "host3"))
+
+val rddA = new MyRDD(sc, 2, Nil)
+val shuffleDepA = new ShuffleDependency(rddA, new HashPartitioner(2))
+val shuffleIdA = shuffleDepA.shuffleId
+
+val rddB = new MyRDD(sc, 2, List(shuffleDepA), tracker = mapOutputTracker)
+val shuffleDepB = new ShuffleDependency(rddB, new HashPartitioner(2))
+
+val rddC = new MyRDD(sc, 2, List(shuffleDepB), tracker = mapOutputTracker)
+
+submit(rddC, Array(0, 1))
+
+completeShuffleMapStageSuccessfully(0, 0, 2, Seq("hostA", "hostA"))
+
+// Fetch failed
+runEvent(makeCompletionEvent(
+  taskSets(1).tasks(0),
+  FetchFailed(makeBlockManagerId("hostC"), shuffleIdA, 0L, 0, 0,
+"Fetch failure of task: stageId=1, stageAttempt=0, partitionId=0"),
+  result = null))
+
+// long running task complete
+completeShuffleMapStageSuccessfully(1, 0, 2, Seq("hostA", "hostA"))

Review Comment:
   nit: let us change the host to `hostB` for successes



-- 
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] mridulm commented on a diff in pull request #40393: [SPARK-40082] Schedule mergeFinalize when push merge shuffleMapStage retry but no running tasks

2023-03-20 Thread via GitHub


mridulm commented on code in PR #40393:
URL: https://github.com/apache/spark/pull/40393#discussion_r1141789869


##
core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala:
##
@@ -4595,6 +4595,183 @@ class DAGSchedulerSuite extends SparkFunSuite with 
TempLocalSparkContext with Ti
 }
   }
 
+  test("SPARK-40082: recomputation of shuffle map stage with no pending 
partitions should not hang") {

Review Comment:
   This line is exceeding 100 chars, and is probably responsible for the build 
failure.



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