kevinjqliu opened a new issue, #16359: URL: https://github.com/apache/iceberg/issues/16359
## Summary Some `spark-extensions` serializable-isolation concurrency tests wait on `Future.get()` without a timeout while worker threads loop up to `Integer.MAX_VALUE`. If the expected `ValidationException` is not thrown, the test has no in-method upper bound and can continue running until CI timeout, disk exhaustion, or manual interruption. ## Example Representative source: https://github.com/apache/iceberg/blob/6976e020b894f6a6777704df2b8c4458cb291ae9/spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMerge.java#L1538-L1628 The merge thread loops nearly unboundedly: ```java for (int numOperations = 0; numOperations < Integer.MAX_VALUE; numOperations++) { ... sql("MERGE INTO ... WHEN MATCHED THEN UPDATE SET dep = 'x'", commitTarget()); barrier.incrementAndGet(); } ``` The test then waits without a timeout: ```java assertThatThrownBy(mergeFuture::get) .isInstanceOf(ExecutionException.class) .cause() .isInstanceOf(ValidationException.class) .hasMessageContaining("Found conflicting files that can contain"); ``` Cleanup is in `finally`, so it is not reached unless `mergeFuture.get()` returns or throws. ## CI evidence This is not only theoretical. In PR #16303, CI was retriggered multiple times before the root cause was understood. The failure was later identified as a legitimate runaway-loop issue: the test kept running until GitHub Actions reported `No space left on device`. ## Affected area The pattern appears in serializable-isolation row-level operation tests, including: - `TestMerge.testMergeWithSerializableIsolation` - `TestDelete.testDeleteWithSerializableIsolation` - `TestUpdate.testUpdateWithSerializableIsolation` These tests are inherited by the corresponding copy-on-write / merge-on-read test classes where present. ## Suggested fix Add an explicit upper bound so these tests fail fast with useful diagnostics instead of relying on CI timeouts or disk exhaustion. Possible options: - use `Future.get(timeout, unit)`; - add a JUnit timeout; - bound the operation loop to a reasonable retry count; - ensure both futures are cancelled and the executor is shut down on timeout. -- 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]
