This is an automated email from the ASF dual-hosted git repository.

danny0405 pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/hudi.git


The following commit(s) were added to refs/heads/master by this push:
     new 321b4299b2bd test: Fix flaky test 
ITTestHoodieFlinkCompactor#testHoodieFlinkCompactorService (#17677)
321b4299b2bd is described below

commit 321b4299b2bd2eb3fa55340a4cfa382b4a1368c4
Author: Shuo Cheng <[email protected]>
AuthorDate: Wed Dec 24 11:50:48 2025 +0800

    test: Fix flaky test 
ITTestHoodieFlinkCompactor#testHoodieFlinkCompactorService (#17677)
---
 .../hudi/sink/compact/ITTestHoodieFlinkCompactor.java |  8 +++++---
 .../test/java/org/apache/hudi/utils/TestUtils.java    | 19 +++++++++++++++++++
 2 files changed, 24 insertions(+), 3 deletions(-)

diff --git 
a/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/compact/ITTestHoodieFlinkCompactor.java
 
b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/compact/ITTestHoodieFlinkCompactor.java
index 2a6e2bcabe9e..f2646aba5269 100644
--- 
a/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/compact/ITTestHoodieFlinkCompactor.java
+++ 
b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/compact/ITTestHoodieFlinkCompactor.java
@@ -28,6 +28,7 @@ import org.apache.hudi.common.table.HoodieTableConfig;
 import org.apache.hudi.common.table.HoodieTableMetaClient;
 import org.apache.hudi.common.table.HoodieTableVersion;
 import org.apache.hudi.common.table.timeline.HoodieInstant;
+import org.apache.hudi.common.table.timeline.HoodieTimeline;
 import org.apache.hudi.common.util.CompactionUtils;
 import org.apache.hudi.common.util.Option;
 import org.apache.hudi.common.util.collection.Pair;
@@ -43,6 +44,7 @@ import org.apache.hudi.utils.FlinkMiniCluster;
 import org.apache.hudi.utils.TestConfigurations;
 import org.apache.hudi.utils.TestData;
 import org.apache.hudi.utils.TestSQL;
+import org.apache.hudi.utils.TestUtils;
 
 import org.apache.flink.api.common.typeinfo.TypeInformation;
 import org.apache.flink.configuration.Configuration;
@@ -313,9 +315,9 @@ public class ITTestHoodieFlinkCompactor {
     HoodieFlinkCompactor.AsyncCompactionService asyncCompactionService = new 
HoodieFlinkCompactor.AsyncCompactionService(cfg, conf);
     asyncCompactionService.start(null);
 
-    // wait for the asynchronous commit to finish
-    TimeUnit.SECONDS.sleep(10);
-
+    assertTrue(TestUtils.waitUntil(() ->
+            TestUtils.getLastCompleteInstant(tempFile.getAbsolutePath(), 
HoodieTimeline.COMMIT_ACTION) != null, 20),
+        "Timed out waiting for compaction commit");
     asyncCompactionService.shutDown();
 
     TestData.checkWrittenDataCOW(tempFile, EXPECTED2);
diff --git 
a/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/utils/TestUtils.java
 
b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/utils/TestUtils.java
index 17f11db950a7..adbee35dd7c6 100644
--- 
a/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/utils/TestUtils.java
+++ 
b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/utils/TestUtils.java
@@ -44,6 +44,8 @@ import javax.annotation.Nullable;
 
 import java.io.IOException;
 import java.nio.file.Files;
+import java.util.concurrent.TimeUnit;
+import java.util.function.BooleanSupplier;
 
 import static 
org.apache.hudi.common.testutils.HoodieTestUtils.INSTANT_GENERATOR;
 import static org.junit.jupiter.api.Assertions.assertTrue;
@@ -157,4 +159,21 @@ public class TestUtils {
     Files.move(sourcePath, newFilePath);
     return newCompletionTime;
   }
+
+  /**
+   * Waits for a condition to be met within a specific timeout.
+   *
+   * @param condition      The condition to poll.
+   * @param timeoutSeconds Maximum time to wait in seconds.
+   */
+  public static boolean waitUntil(BooleanSupplier condition, int 
timeoutSeconds) throws InterruptedException {
+    long limit = System.currentTimeMillis() + (timeoutSeconds * 1000L);
+    while (System.currentTimeMillis() < limit) {
+      if (condition.getAsBoolean()) {
+        return true;
+      }
+      TimeUnit.SECONDS.sleep(1);
+    }
+    return false;
+  }
 }

Reply via email to