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

ctubbsii pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/accumulo.git


The following commit(s) were added to refs/heads/main by this push:
     new 7e6ad2d363 Update FateIT with changes from #5121
7e6ad2d363 is described below

commit 7e6ad2d3636e361a402aae207666a4bf690a1fd6
Author: Christopher Tubbs <ctubb...@apache.org>
AuthorDate: Thu Nov 28 05:19:35 2024 -0500

    Update FateIT with changes from #5121
    
    Add more strict transition checks to FateIT.testTransactionStatus in the
    main (elastic) branch that were done in #5121 for 2.1 and 3.1 to
    stabilize FateIT, to ensure the fate transactions transition from
    IN_PROGRESS to SUCCESSFUL (may not be observed by the test) to UNKNOWN.
---
 .../java/org/apache/accumulo/test/fate/FateIT.java | 32 ++++++++++++++++++++--
 1 file changed, 30 insertions(+), 2 deletions(-)

diff --git a/test/src/main/java/org/apache/accumulo/test/fate/FateIT.java 
b/test/src/main/java/org/apache/accumulo/test/fate/FateIT.java
index d128f24a65..e8a77bd330 100644
--- a/test/src/main/java/org/apache/accumulo/test/fate/FateIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/fate/FateIT.java
@@ -18,6 +18,7 @@
  */
 package org.apache.accumulo.test.fate;
 
+import static java.util.concurrent.TimeUnit.SECONDS;
 import static org.apache.accumulo.core.fate.ReadOnlyFateStore.TStatus.FAILED;
 import static 
org.apache.accumulo.core.fate.ReadOnlyFateStore.TStatus.FAILED_IN_PROGRESS;
 import static 
org.apache.accumulo.core.fate.ReadOnlyFateStore.TStatus.IN_PROGRESS;
@@ -29,6 +30,7 @@ import static org.junit.jupiter.api.Assertions.assertEquals;
 import static org.junit.jupiter.api.Assertions.assertFalse;
 import static org.junit.jupiter.api.Assertions.assertThrows;
 import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.junit.jupiter.api.Assertions.fail;
 
 import java.util.ArrayList;
 import java.util.HashSet;
@@ -36,6 +38,7 @@ import java.util.List;
 import java.util.Set;
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicLong;
 
@@ -245,8 +248,33 @@ public abstract class FateIT extends SharedMiniClusterBase 
implements FateTestRu
       assertEquals(IN_PROGRESS, getTxStatus(sctx, fateId));
       // tell the op to exit the method
       finishCall.countDown();
-
-      Wait.waitFor(() -> getTxStatus(sctx, fateId) == UNKNOWN);
+      // Check that it transitions to SUCCESSFUL and then removed (UNKNOWN)
+      final var sawSuccess = new AtomicBoolean(false);
+      Wait.waitFor(() -> {
+        TStatus s;
+        switch (s = getTxStatus(sctx, fateId)) {
+          case IN_PROGRESS:
+            if (sawSuccess.get()) {
+              fail("Should never see IN_PROGRESS after seeing SUCCESSFUL");
+            }
+            break;
+          case SUCCESSFUL:
+            // expected, but might be too quick to be detected
+            if (sawSuccess.compareAndSet(false, true)) {
+              LOG.debug("Saw expected transaction status change to 
SUCCESSFUL");
+            }
+            break;
+          case UNKNOWN:
+            if (!sawSuccess.get()) {
+              LOG.debug("Never saw transaction status change to SUCCESSFUL, 
but that's okay");
+            }
+            return true;
+          default:
+            fail("Saw unexpected status: " + s);
+        }
+        // keep waiting for UNKNOWN
+        return false;
+      }, SECONDS.toMillis(30), 10);
     } finally {
       fate.shutdown(10, TimeUnit.MINUTES);
     }

Reply via email to