Copilot commented on code in PR #3638:
URL: https://github.com/apache/celeborn/pull/3638#discussion_r3020470736


##########
client/src/test/scala/org/apache/celeborn/client/WorkerStatusTrackerSuite.scala:
##########
@@ -92,6 +96,68 @@ class WorkerStatusTrackerSuite extends CelebornFunSuite {
     
Assert.assertFalse(statusTracker.excludedWorkers.containsKey(mock("host1")))
   }
 
+  test("concurrent access to shuttingWorkers should not throw 
ConcurrentModificationException") {
+    val celebornConf = new CelebornConf()
+    val statusTracker = new WorkerStatusTracker(celebornConf, null)
+    val numWriters = 5
+    val numReaders = 5
+    val totalThreads = numWriters + numReaders
+    val executor = Executors.newFixedThreadPool(totalThreads)
+    val barrier = new CyclicBarrier(totalThreads)
+    val errors = new AtomicInteger(0)
+    val futures = new ArrayBuffer[Future[_]]()
+
+    // Pre-populate the set so iteration takes longer, increasing the
+    // window for concurrent modification to trigger a CME with HashSet.
+    (1 to 1000).foreach { i =>
+      statusTracker.shuttingWorkers.add(mock(s"pre-$i"))
+    }
+
+    try {
+      // Writers: concurrently add and remove workers
+      (1 to numWriters).foreach { i =>
+        futures += executor.submit(new Runnable {
+          override def run(): Unit = {
+            barrier.await()
+            (1 to 1000).foreach { j =>
+              val worker = mock(s"host-$i-$j")
+              statusTracker.shuttingWorkers.add(worker)
+              statusTracker.shuttingWorkers.remove(worker)
+            }
+          }
+        })
+      }
+
+      // Readers: iterate shuttingWorkers directly, mirroring how 
currentFailedWorkers iterates it
+      (1 to numReaders).foreach { i =>
+        futures += executor.submit(new Runnable {
+          override def run(): Unit = {
+            try {
+              barrier.await()
+              (1 to 1000).foreach { _ =>
+                statusTracker.shuttingWorkers.forEach(_ => ())
+              }
+            } catch {
+              case _: java.util.ConcurrentModificationException =>
+                errors.incrementAndGet()
+            }
+          }
+        })
+      }
+
+      // Surface any unexpected exceptions from threads
+      futures.foreach(_.get(30, TimeUnit.SECONDS))
+    } finally {
+      executor.shutdown()
+      Assert.assertTrue(executor.awaitTermination(30, TimeUnit.SECONDS))
+    }

Review Comment:
   `futures.foreach(_.get(30, TimeUnit.SECONDS))` applies the 30s timeout per 
future, so in a stuck/failing scenario this test can block for up to ~5 minutes 
(10 futures * 30s) before failing. To keep the suite responsive in CI, consider 
enforcing a single overall deadline (or using `invokeAll` with a global 
timeout) and also using timed `barrier.await(timeout, unit)` to avoid threads 
waiting indefinitely at the barrier.



##########
client/src/test/scala/org/apache/celeborn/client/WorkerStatusTrackerSuite.scala:
##########
@@ -92,6 +96,68 @@ class WorkerStatusTrackerSuite extends CelebornFunSuite {
     
Assert.assertFalse(statusTracker.excludedWorkers.containsKey(mock("host1")))
   }
 
+  test("concurrent access to shuttingWorkers should not throw 
ConcurrentModificationException") {
+    val celebornConf = new CelebornConf()
+    val statusTracker = new WorkerStatusTracker(celebornConf, null)
+    val numWriters = 5
+    val numReaders = 5
+    val totalThreads = numWriters + numReaders
+    val executor = Executors.newFixedThreadPool(totalThreads)
+    val barrier = new CyclicBarrier(totalThreads)
+    val errors = new AtomicInteger(0)
+    val futures = new ArrayBuffer[Future[_]]()
+
+    // Pre-populate the set so iteration takes longer, increasing the
+    // window for concurrent modification to trigger a CME with HashSet.
+    (1 to 1000).foreach { i =>
+      statusTracker.shuttingWorkers.add(mock(s"pre-$i"))
+    }
+
+    try {
+      // Writers: concurrently add and remove workers
+      (1 to numWriters).foreach { i =>
+        futures += executor.submit(new Runnable {
+          override def run(): Unit = {
+            barrier.await()
+            (1 to 1000).foreach { j =>
+              val worker = mock(s"host-$i-$j")
+              statusTracker.shuttingWorkers.add(worker)
+              statusTracker.shuttingWorkers.remove(worker)
+            }
+          }
+        })
+      }
+
+      // Readers: iterate shuttingWorkers directly, mirroring how 
currentFailedWorkers iterates it
+      (1 to numReaders).foreach { i =>
+        futures += executor.submit(new Runnable {
+          override def run(): Unit = {
+            try {
+              barrier.await()
+              (1 to 1000).foreach { _ =>
+                statusTracker.shuttingWorkers.forEach(_ => ())
+              }

Review Comment:
   This concurrency test is probabilistic (it relies on a race to happen during 
iteration). As written, the test may still pass even if `shuttingWorkers` 
regresses back to a non-thread-safe `HashSet`, which reduces its value as a 
regression test. Consider making the failure deterministic by coordinating a 
reader thread that creates an iterator / starts iterating, then a writer thread 
that mutates the set at a known point, and finally continuing iteration to 
assert whether a `ConcurrentModificationException` is thrown (HashSet) vs not 
thrown (ConcurrentHashMap keySet).



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

Reply via email to