This is an automated email from the ASF dual-hosted git repository. kturner 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 461d497189 Jenkins FateSummaryIT fix (again) (#4631) 461d497189 is described below commit 461d497189d9f5374660ac0fcf53b57f9cfde628 Author: Kevin Rathbun <43969518+kevinrr...@users.noreply.github.com> AuthorDate: Mon Jun 10 18:10:19 2024 -0400 Jenkins FateSummaryIT fix (again) (#4631) Rewrite of testFatePrintAndSummaryCommandsWithInProgressTxns Rewrote the test to directly test the method where the ZK NoNodeException would occur. This will not have the same Jenkins timeout issues that the test previously had but will be more narrow in scope. --- .../org/apache/accumulo/core/fate/AdminUtil.java | 6 +- .../org/apache/accumulo/test/FateSummaryIT.java | 149 ++++++++++----------- 2 files changed, 76 insertions(+), 79 deletions(-) diff --git a/core/src/main/java/org/apache/accumulo/core/fate/AdminUtil.java b/core/src/main/java/org/apache/accumulo/core/fate/AdminUtil.java index b7afea1b84..3ea322ef78 100644 --- a/core/src/main/java/org/apache/accumulo/core/fate/AdminUtil.java +++ b/core/src/main/java/org/apache/accumulo/core/fate/AdminUtil.java @@ -333,7 +333,7 @@ public class AdminUtil<T> { * @param waitingLocks populated list of locks held by transaction - or an empty map if none. * @return current fate and lock status */ - private FateStatus getTransactionStatus(ReadOnlyTStore<T> zs, Set<Long> filterTxid, + public static <T> FateStatus getTransactionStatus(ReadOnlyTStore<T> zs, Set<Long> filterTxid, EnumSet<TStatus> filterStatus, Map<Long,List<String>> heldLocks, Map<Long,List<String>> waitingLocks) { @@ -399,11 +399,11 @@ public class AdminUtil<T> { } - private boolean includeByStatus(TStatus status, EnumSet<TStatus> filterStatus) { + private static boolean includeByStatus(TStatus status, EnumSet<TStatus> filterStatus) { return (filterStatus == null) || filterStatus.contains(status); } - private boolean includeByTxid(Long tid, Set<Long> filterTxid) { + private static boolean includeByTxid(Long tid, Set<Long> filterTxid) { return (filterTxid == null) || filterTxid.isEmpty() || filterTxid.contains(tid); } diff --git a/test/src/main/java/org/apache/accumulo/test/FateSummaryIT.java b/test/src/main/java/org/apache/accumulo/test/FateSummaryIT.java index 9c3ee5b8a4..5f2543bc1b 100644 --- a/test/src/main/java/org/apache/accumulo/test/FateSummaryIT.java +++ b/test/src/main/java/org/apache/accumulo/test/FateSummaryIT.java @@ -19,6 +19,10 @@ package org.apache.accumulo.test; import static org.apache.accumulo.core.util.compaction.ExternalCompactionUtil.getCompactorAddrs; +import static org.easymock.EasyMock.anyLong; +import static org.easymock.EasyMock.expect; +import static org.easymock.EasyMock.replay; +import static org.easymock.EasyMock.verify; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertNotEquals; @@ -29,21 +33,30 @@ import static org.junit.jupiter.api.Assertions.fail; import java.time.Duration; import java.util.ArrayList; import java.util.EnumSet; +import java.util.HashMap; import java.util.HashSet; +import java.util.List; import java.util.Map; import java.util.Set; import java.util.SortedSet; import java.util.TreeSet; +import java.util.stream.Collectors; +import org.apache.accumulo.core.Constants; import org.apache.accumulo.core.client.Accumulo; import org.apache.accumulo.core.client.AccumuloClient; import org.apache.accumulo.core.client.IteratorSetting; import org.apache.accumulo.core.client.admin.NewTableConfiguration; -import org.apache.accumulo.core.iterators.IteratorUtil; +import org.apache.accumulo.core.fate.AdminUtil; +import org.apache.accumulo.core.fate.Repo; +import org.apache.accumulo.core.fate.ZooStore; +import org.apache.accumulo.core.fate.zookeeper.ZooReaderWriter; import org.apache.accumulo.core.iterators.IteratorUtil.IteratorScope; +import org.apache.accumulo.core.util.FastFormat; import org.apache.accumulo.minicluster.ServerType; import org.apache.accumulo.miniclusterImpl.MiniAccumuloClusterImpl.ProcessInfo; import org.apache.accumulo.miniclusterImpl.MiniAccumuloConfigImpl; +import org.apache.accumulo.server.ServerContext; import org.apache.accumulo.server.util.Admin; import org.apache.accumulo.server.util.fateCommand.FateSummaryReport; import org.apache.accumulo.test.functional.ConfigurableMacBase; @@ -52,6 +65,8 @@ import org.apache.accumulo.test.functional.SlowIterator; import org.apache.accumulo.test.util.Wait; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.io.Text; +import org.apache.zookeeper.KeeperException; +import org.easymock.EasyMock; import org.junit.jupiter.api.Test; public class FateSummaryIT extends ConfigurableMacBase { @@ -174,93 +189,75 @@ public class FateSummaryIT extends ConfigurableMacBase { // between getting the list and probing for info on that transaction, a NoNodeException would // occur causing the cmd to fail. This test ensures that this problem has been fixed (if the // tx no longer exists, it should just be ignored so the print/summary can complete). + ServerContext sctx = getCluster().getServerContext(); - String[] commandsToTest = {"--print", "--summary"}; - // We want to have enough transactions to give enough opportunity for a transaction to - // complete mid-print - final int numTxns = 250; - final String table = getUniqueNames(1)[0]; + // This error was occurring in AdminUtil.getTransactionStatus(). One of the methods that is + // called which may throw the NNE is top(), so we will mock this method to sometimes throw a + // NNE and ensure it is handled/ignored within getTransactionStatus() + ZooStore<String> zs = EasyMock.createMockBuilder(ZooStore.class) + .withConstructor(String.class, ZooReaderWriter.class) + .withArgs(sctx.getZooKeeperRoot() + Constants.ZFATE, sctx.getZooReaderWriter()) + .addMockedMethod("top").addMockedMethod("list").createMock(); + // Create 3 transactions, when iterating through the list of transactions in + // getTransactionStatus(), the 2nd transaction should cause a NNE which should be + // handled/ignored in getTransactionStatus(). The other two transactions should still + // be returned. + long tx1 = zs.create(); + long tx2 = zs.create(); + long tx3 = zs.create(); + // Mock list() to ensure same order every run + expect(zs.list()).andReturn(List.of(tx1, tx2, tx3)).once(); - // Occasionally, the summary/print cmds will see a COMMIT_COMPACTION transaction which was - // initiated on starting the manager, causing the test to fail. Stopping the compactor fixes - // this issue. - getCluster().getClusterControl().stopAllServers(ServerType.COMPACTOR); - Wait.waitFor(() -> getCompactorAddrs(getCluster().getServerContext()).isEmpty(), 60_000); + expect(zs.top(anyLong())).andReturn(new TestRepo()).once(); + expect(zs.top(anyLong())).andThrow(new RuntimeException(new KeeperException.NoNodeException())) + .once(); + expect(zs.top(anyLong())).andReturn(new TestRepo()).once(); + replay(zs); - try (AccumuloClient client = Accumulo.newClient().from(getClientProperties()).build()) { - for (String command : commandsToTest) { - IteratorSetting is = new IteratorSetting(1, SlowIterator.class); - is.addOption("sleepTime", "2"); - - NewTableConfiguration cfg = new NewTableConfiguration(); - cfg.attachIterator(is, EnumSet.of(IteratorUtil.IteratorScope.majc)); - client.tableOperations().create(table, cfg); - - ReadWriteIT.ingest(client, 10, 1, 10, 0, table); - client.tableOperations().flush(table, null, null, true); + AdminUtil.FateStatus status = null; + try { + status = AdminUtil.getTransactionStatus(zs, null, null, new HashMap<>(), new HashMap<>()); + } catch (Exception e) { + fail( + "Either an unexpected error occurred in getTransactionStatus() or the NoNodeException which" + + " is expected to be handled in getTransactionStatus() was not handled. Error:\n" + + e); + } + verify(zs); + assertNotNull(status); + assertEquals(2, status.getTransactions().size()); + assertTrue(status.getTransactions().stream().map(AdminUtil.TransactionStatus::getTxid) + .collect(Collectors.toList()) + .containsAll(List.of(FastFormat.toHexString(tx1), FastFormat.toHexString(tx3)))); + } - // validate no transactions - ProcessInfo p = execAdminFateCommand(command); - assertEquals(0, p.getProcess().waitFor()); - String result = p.readStdOut(); - assertTrue(noTransactions(result, command)); + private static class TestRepo implements Repo<String> { - // We want transactions which take some time to complete since we don't want them - // to complete before the call to print (hence the sleep time iterator), but we - // also don't want them to take too long to complete since in that case we - // may not see transactions complete mid-print + private static final long serialVersionUID = 1L; - // create 250 txns each taking >= 20ms to complete >= 5 seconds total - for (int i = 0; i < numTxns; i++) { - // Initiate compaction to create txn. This compaction will take >= 20ms to complete - // ((10 key values) * (2ms sleep time / key value)) - client.tableOperations().compact(table, null, null, false, false); - } + @Override + public long isReady(long tid, String environment) throws Exception { + return 0; + } - // Keep printing until we see a transaction complete mid-print or until we run out of - // transactions (they all complete). - // Realistically, should only take 1 or 2 iterations to see a transaction complete - // mid-print. - do { - // Execute the command when transactions are currently running and may complete mid-print - p = execAdminFateCommand(command); - // Previously, this check could fail due to a ZK NoNodeException - assertEquals(0, p.getProcess().waitFor()); - result = p.readStdOut(); - // A transaction should have completed mid-print and been ignored - } while (!result.contains("Tried to get info on a since completed transaction - ignoring") - && !noTransactions(result, command)); + @Override + public String getName() { + return null; + } - if (noTransactions(result, command)) { - // Fail since we printed until all transactions have completed and didn't see a - // transaction complete mid-print. - // This is highly unlikely to have occurred. - fail(); - } - // Otherwise, we saw 'Tried to get info on a since completed transaction - ignoring', so - // test passes - client.tableOperations().delete(table); - } + @Override + public Repo<String> call(long tid, String environment) throws Exception { + return null; } - } - private boolean noTransactions(String result, String command) { - if (command.equals("--print")) { - return result.contains(" 0 transactions"); - } else { // --summary - result = result.substring(result.indexOf("{"), result.lastIndexOf("}") + 1); - FateSummaryReport report = FateSummaryReport.fromJson(result); - return report != null && report.getReportTime() != 0 && report.getStatusCounts().isEmpty() - && report.getStepCounts().isEmpty() && report.getCmdCounts().isEmpty() - && report.getStatusFilterNames().isEmpty() && report.getFateDetails().isEmpty(); + @Override + public void undo(long tid, String environment) throws Exception { + } - } - private ProcessInfo execAdminFateCommand(String command) throws Exception { - if (command.equals("--print")) { - return getCluster().exec(Admin.class, "fate", command); - } else { // --summary - return getCluster().exec(Admin.class, "fate", command, "-j"); + @Override + public String getReturn() { + return null; } } }