This is an automated email from the ASF dual-hosted git repository. ctubbsii pushed a commit to branch 2.1 in repository https://gitbox.apache.org/repos/asf/accumulo.git
The following commit(s) were added to refs/heads/2.1 by this push: new 70e73bc386 Fix flaky FateIT.testTransactionStatus() (#5121) 70e73bc386 is described below commit 70e73bc386d419e3f6a5a37e52a6ed5a0d52b4ea Author: Christopher Tubbs <ctubb...@apache.org> AuthorDate: Wed Nov 27 18:07:22 2024 -0500 Fix flaky FateIT.testTransactionStatus() (#5121) Related to #2474 and #2550. The sync fix in #2550 helped ensure that the ZooKeeper client had the updated status, it is still possible for the node to be deleted before the SUCCESSFUL transaction status is observed. This change makes FateIT more stable by accepting that as a valid possible outcome, and no longer fails when that happens. --- .../accumulo/test/fate/zookeeper/FateIT.java | 43 ++++++++++++++-------- 1 file changed, 28 insertions(+), 15 deletions(-) diff --git a/test/src/main/java/org/apache/accumulo/test/fate/zookeeper/FateIT.java b/test/src/main/java/org/apache/accumulo/test/fate/zookeeper/FateIT.java index 2dde8fabca..bc92312d41 100644 --- a/test/src/main/java/org/apache/accumulo/test/fate/zookeeper/FateIT.java +++ b/test/src/main/java/org/apache/accumulo/test/fate/zookeeper/FateIT.java @@ -19,12 +19,12 @@ package org.apache.accumulo.test.fate.zookeeper; import static java.nio.charset.StandardCharsets.UTF_8; +import static java.util.concurrent.TimeUnit.SECONDS; import static org.apache.accumulo.core.fate.ReadOnlyTStore.TStatus.FAILED; import static org.apache.accumulo.core.fate.ReadOnlyTStore.TStatus.FAILED_IN_PROGRESS; import static org.apache.accumulo.core.fate.ReadOnlyTStore.TStatus.IN_PROGRESS; import static org.apache.accumulo.core.fate.ReadOnlyTStore.TStatus.NEW; import static org.apache.accumulo.core.fate.ReadOnlyTStore.TStatus.SUBMITTED; -import static org.apache.accumulo.core.fate.ReadOnlyTStore.TStatus.SUCCESSFUL; import static org.apache.accumulo.harness.AccumuloITBase.ZOOKEEPER_TESTING_SERVER; import static org.easymock.EasyMock.createMock; import static org.easymock.EasyMock.expect; @@ -40,6 +40,7 @@ import java.util.ArrayList; import java.util.List; import java.util.UUID; import java.util.concurrent.CountDownLatch; +import java.util.concurrent.atomic.AtomicBoolean; import org.apache.accumulo.core.Constants; import org.apache.accumulo.core.clientImpl.thrift.TableOperation; @@ -243,27 +244,39 @@ public class FateIT { assertEquals(IN_PROGRESS, getTxStatus(zk, txid)); // tell the op to exit the method finishCall.countDown(); - // Check that it transitions to SUCCESSFUL - TStatus s = getTxStatus(zk, txid); - while (s != SUCCESSFUL) { - s = getTxStatus(zk, txid); - Thread.sleep(10); - } - // Check that it gets removed - boolean errorSeen = false; - while (!errorSeen) { + // Check that it transitions to SUCCESSFUL and gets removed + final var sawSuccess = new AtomicBoolean(false); + Wait.waitFor(() -> { + TStatus s; try { - s = getTxStatus(zk, txid); - Thread.sleep(10); + switch (s = getTxStatus(zk, txid)) { + 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; + default: + fail("Saw unexpected status: " + s); + } } catch (KeeperException e) { if (e.code() == KeeperException.Code.NONODE) { - errorSeen = true; + if (!sawSuccess.get()) { + LOG.debug("Never saw transaction status change to SUCCESSFUL, but that's okay"); + } + return true; } else { fail("Unexpected error thrown: " + e.getMessage()); } } - } - + // keep waiting for NoNode + return false; + }, SECONDS.toMillis(30), 10); } finally { fate.shutdown(); }