Skip to content

Commit

Permalink
Fix flaky FateIT.testTransactionStatus() (apache#5121)
Browse files Browse the repository at this point in the history
Related to apache#2474 and apache#2550. The sync fix in apache#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.
  • Loading branch information
ctubbsii authored Nov 27, 2024
1 parent 0de01c3 commit 70e73bc
Showing 1 changed file with 28 additions and 15 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand All @@ -40,6 +40,7 @@
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;
Expand Down Expand Up @@ -243,27 +244,39 @@ public void testTransactionStatus() throws Exception {
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();
}
Expand Down

0 comments on commit 70e73bc

Please sign in to comment.