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


##########
core/src/test/scala/org/apache/spark/storage/BlockManagerDecommissionIntegrationSuite.scala:
##########
@@ -353,4 +355,78 @@ class BlockManagerDecommissionIntegrationSuite extends 
SparkFunSuite with LocalS
     import scala.language.reflectiveCalls
     assert(listener.removeReasonValidated)
   }
+
+  test("SPARK-46957: Migrated shuffle files should be able to cleanup from 
executor") {
+
+    val sparkTempDir = System.getProperty("java.io.tmpdir")
+
+    def shuffleFiles: Seq[File] = {
+      FileUtils
+        .listFiles(new File(sparkTempDir), Array("data", "index"), true)
+        .asScala
+        .toSeq
+    }
+
+    val existingShuffleFiles = shuffleFiles
+
+    val conf = new SparkConf()
+      .setAppName("SPARK-46957")
+      .setMaster("local-cluster[2,1,1024]")
+      .set(config.DECOMMISSION_ENABLED, true)
+      .set(config.STORAGE_DECOMMISSION_ENABLED, true)
+      .set(config.STORAGE_DECOMMISSION_SHUFFLE_BLOCKS_ENABLED, true)
+    sc = new SparkContext(conf)
+    TestUtils.waitUntilExecutorsUp(sc, 2, 60000)
+    val shuffleBlockUpdates = new ArrayBuffer[BlockId]()
+    var isDecommissionedExecutorRemoved = false
+    val execToDecommission = sc.getExecutorIds().head
+    sc.addSparkListener(new SparkListener {
+      override def onBlockUpdated(blockUpdated: SparkListenerBlockUpdated): 
Unit = {
+        if (blockUpdated.blockUpdatedInfo.blockId.isShuffle) {
+          shuffleBlockUpdates += blockUpdated.blockUpdatedInfo.blockId
+        }
+      }
+
+      override def onExecutorRemoved(executorRemoved: 
SparkListenerExecutorRemoved): Unit = {
+        assert(execToDecommission === executorRemoved.executorId)
+        isDecommissionedExecutorRemoved = true
+      }
+    })
+
+    // Run a job to create shuffle data
+    val result = sc.parallelize(1 to 1000, 10)
+      .map { i => (i % 2, i) }
+      .reduceByKey(_ + _).collect()
+
+    assert(result.head === (0, 250500))
+    assert(result.tail.head === (1, 250000))
+    sc.schedulerBackend
+      .asInstanceOf[StandaloneSchedulerBackend]
+      .decommissionExecutor(
+        execToDecommission,
+        ExecutorDecommissionInfo("test", None),
+        adjustTargetNumExecutors = true
+      )
+
+    eventually(timeout(1.minute), interval(10.milliseconds)) {
+      assert(isDecommissionedExecutorRemoved)
+      // Ensure there are shuffle data have been migrated
+      assert(shuffleBlockUpdates.size >= 2)
+    }
+
+    val shuffleId = shuffleBlockUpdates
+      .find(_.isInstanceOf[ShuffleIndexBlockId])
+      .map(_.asInstanceOf[ShuffleIndexBlockId].shuffleId)
+      .get
+
+    val newShuffleFiles = shuffleFiles.diff(existingShuffleFiles)
+    assert(newShuffleFiles.size >= shuffleBlockUpdates.size)

Review Comment:
   Unfortunately, this seems to introduce a flakiness.
   - https://github.com/apache/spark/actions/runs/11378094259/job/31653317304
   ```
   [info] - SPARK-46957: Migrated shuffle files should be able to cleanup from 
executor *** FAILED *** (36 seconds, 137 milliseconds)
   [info]   0 was not greater than or equal to 6 
(BlockManagerDecommissionIntegrationSuite.scala:423)
   ```



-- 
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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to