You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@accumulo.apache.org by ed...@apache.org on 2022/06/28 16:09:57 UTC

[accumulo] branch main updated: Fix Fate print command and improve ShellServerIT test (#2755)

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

edcoleman 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 4327bee71f Fix Fate print command and improve ShellServerIT test (#2755)
4327bee71f is described below

commit 4327bee71f63bd7380d6ec2f209414adfe9335e2
Author: EdColeman <de...@etcoleman.com>
AuthorDate: Tue Jun 28 16:09:51 2022 +0000

    Fix Fate print command and improve ShellServerIT test (#2755)
    
    * modify filter conditions for status
    * Add additional tests to code provided by millerruntime, modified to add cluster accumulo.properties
---
 .../java/org/apache/accumulo/fate/AdminUtil.java   | 19 ++++---
 .../accumulo/shell/commands/FateCommand.java       | 16 +++---
 .../accumulo/shell/commands/FateCommandTest.java   | 12 +++--
 .../apache/accumulo/test/shell/ShellServerIT.java  | 59 ++++++++++++++++++++++
 4 files changed, 88 insertions(+), 18 deletions(-)

diff --git a/core/src/main/java/org/apache/accumulo/fate/AdminUtil.java b/core/src/main/java/org/apache/accumulo/fate/AdminUtil.java
index a1e8b8d017..dfd65d8494 100644
--- a/core/src/main/java/org/apache/accumulo/fate/AdminUtil.java
+++ b/core/src/main/java/org/apache/accumulo/fate/AdminUtil.java
@@ -252,14 +252,12 @@ public class AdminUtil<T> {
   public FateStatus getStatus(ReadOnlyTStore<T> zs, ZooReader zk,
       ServiceLock.ServiceLockPath lockPath, Set<Long> filterTxid, EnumSet<TStatus> filterStatus)
       throws KeeperException, InterruptedException {
-
     Map<Long,List<String>> heldLocks = new HashMap<>();
     Map<Long,List<String>> waitingLocks = new HashMap<>();
 
     findLocks(zk, lockPath, heldLocks, waitingLocks);
 
     return getTransactionStatus(zs, filterTxid, filterStatus, heldLocks, waitingLocks);
-
   }
 
   /**
@@ -390,17 +388,23 @@ public class AdminUtil<T> {
 
       zs.unreserve(tid, 0);
 
-      if ((filterTxid != null && !filterTxid.contains(tid))
-          || (filterStatus != null && !filterStatus.contains(status)))
-        continue;
-
-      statuses.add(new TransactionStatus(tid, status, debug, hlocks, wlocks, top, timeCreated));
+      if (includeByStatus(status, filterStatus) && includeByTxid(tid, filterTxid)) {
+        statuses.add(new TransactionStatus(tid, status, debug, hlocks, wlocks, top, timeCreated));
+      }
     }
 
     return new FateStatus(statuses, heldLocks, waitingLocks);
 
   }
 
+  private boolean includeByStatus(TStatus status, EnumSet<TStatus> filterStatus) {
+    return (filterStatus == null) || filterStatus.contains(status);
+  }
+
+  private boolean includeByTxid(Long tid, Set<Long> filterTxid) {
+    return (filterTxid == null) || filterTxid.isEmpty() || filterTxid.contains(tid);
+  }
+
   public void print(ReadOnlyTStore<T> zs, ZooReader zk, ServiceLock.ServiceLockPath lockPath)
       throws KeeperException, InterruptedException {
     print(zs, zk, lockPath, new Formatter(System.out), null, null);
@@ -409,7 +413,6 @@ public class AdminUtil<T> {
   public void print(ReadOnlyTStore<T> zs, ZooReader zk, ServiceLock.ServiceLockPath lockPath,
       Formatter fmt, Set<Long> filterTxid, EnumSet<TStatus> filterStatus)
       throws KeeperException, InterruptedException {
-
     FateStatus fateStatus = getStatus(zs, zk, lockPath, filterTxid, filterStatus);
 
     for (TransactionStatus txStatus : fateStatus.getTransactions()) {
diff --git a/shell/src/main/java/org/apache/accumulo/shell/commands/FateCommand.java b/shell/src/main/java/org/apache/accumulo/shell/commands/FateCommand.java
index db94d9f188..9a83dfadfd 100644
--- a/shell/src/main/java/org/apache/accumulo/shell/commands/FateCommand.java
+++ b/shell/src/main/java/org/apache/accumulo/shell/commands/FateCommand.java
@@ -218,10 +218,12 @@ public class FateCommand extends Command {
       throws InterruptedException, KeeperException, IOException {
     // Parse transaction ID filters for print display
     Set<Long> filterTxid = new HashSet<>();
-    if (args != null && args.length >= 2) {
-      for (int i = 1; i < args.length; i++) {
-        Long val = parseTxid(args[i]);
-        filterTxid.add(val);
+    if (args != null && args.length >= 1) {
+      for (int i = 0; i < args.length; i++) {
+        if (!args[i].isEmpty()) {
+          Long val = parseTxid(args[i]);
+          filterTxid.add(val);
+        }
       }
     }
 
@@ -343,12 +345,14 @@ public class FateCommand extends Command {
     delete.setArgs(Option.UNLIMITED_VALUES);
     delete.setOptionalArg(false);
 
-    list = new Option("list", "list", true, "print FaTE transaction information");
+    list = new Option("list", "list", true,
+        "print FaTE transaction information. Filter on id(s) with FATE[id] or id list ");
     list.setArgName("txid");
     list.setArgs(Option.UNLIMITED_VALUES);
     list.setOptionalArg(true);
 
-    print = new Option("print", "print", true, "print FaTE transaction information");
+    print = new Option("print", "print", true,
+        "print FaTE transaction information. Filter on id(s) with FATE[id] or id list ");
     print.setArgName("txid");
     print.setArgs(Option.UNLIMITED_VALUES);
     print.setOptionalArg(true);
diff --git a/shell/src/test/java/org/apache/accumulo/shell/commands/FateCommandTest.java b/shell/src/test/java/org/apache/accumulo/shell/commands/FateCommandTest.java
index 4307c6e8e4..5f526c6939 100644
--- a/shell/src/test/java/org/apache/accumulo/shell/commands/FateCommandTest.java
+++ b/shell/src/test/java/org/apache/accumulo/shell/commands/FateCommandTest.java
@@ -213,14 +213,18 @@ public class FateCommandTest {
 
     try {
       cmd.printTx(shell, helper, zs, zk, tableLocksPath, cli.getOptionValues("list"), cli);
+      cmd.printTx(shell, helper, zs, zk, tableLocksPath, cli.getOptionValues("list FATE[1]"), cli);
+      cmd.printTx(shell, helper, zs, zk, tableLocksPath, cli.getOptionValues("list 1234"), cli);
+      cmd.printTx(shell, helper, zs, zk, tableLocksPath, cli.getOptionValues("list 1234 2345"),
+          cli);
       cmd.printTx(shell, helper, zs, zk, tableLocksPath, cli.getOptionValues("print"), cli);
+      cmd.printTx(shell, helper, zs, zk, tableLocksPath, cli.getOptionValues("print FATE[1]"), cli);
+      cmd.printTx(shell, helper, zs, zk, tableLocksPath, cli.getOptionValues("print 1234"), cli);
+      cmd.printTx(shell, helper, zs, zk, tableLocksPath, cli.getOptionValues("print 1234 2345"),
+          cli);
       cmd.printTx(shell, helper, zs, zk, tableLocksPath, new String[] {""}, cli);
       cmd.printTx(shell, helper, zs, zk, tableLocksPath, new String[] {}, cli);
       cmd.printTx(shell, helper, zs, zk, tableLocksPath, null, cli);
-      cmd.printTx(shell, helper, zs, zk, tableLocksPath, new String[] {"list"}, cli);
-      cmd.printTx(shell, helper, zs, zk, tableLocksPath, new String[] {"list", "1234"}, cli);
-      cmd.printTx(shell, helper, zs, zk, tableLocksPath, new String[] {"print"}, cli);
-      cmd.printTx(shell, helper, zs, zk, tableLocksPath, new String[] {"print", "1234"}, cli);
     } finally {
       output.clear();
       System.setOut(out);
diff --git a/test/src/main/java/org/apache/accumulo/test/shell/ShellServerIT.java b/test/src/main/java/org/apache/accumulo/test/shell/ShellServerIT.java
index 493e1123c6..628879be51 100644
--- a/test/src/main/java/org/apache/accumulo/test/shell/ShellServerIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/shell/ShellServerIT.java
@@ -2082,4 +2082,63 @@ public class ShellServerIT extends SharedMiniClusterBase {
     // check that there are two files, with none having extra summary info
     assertMatches(output, "(?sm).*^.*total[:]2[,]\\s+missing[:]0[,]\\s+extra[:]0.*$.*");
   }
+
+  @Test
+  public void testFateCommandWithSlowCompaction() throws Exception {
+    final String table = getUniqueNames(1)[0];
+
+    String orgProps = System.getProperty("accumulo.properties");
+
+    System.setProperty("accumulo.properties",
+        "file://" + getCluster().getConfig().getAccumuloPropsFile().getCanonicalPath());
+    // compact
+    ts.exec("createtable " + table);
+
+    // setup SlowIterator to sleep for 10 seconds
+    ts.exec("config -t " + table
+        + " -s table.iterator.majc.slow=1,org.apache.accumulo.test.functional.SlowIterator");
+    ts.exec("config -t " + table + " -s table.iterator.majc.slow.opt.sleepTime=10000");
+
+    String tableId = getTableId(table);
+
+    // make two files
+    ts.exec("insert a1 b c v_a1");
+    ts.exec("insert a2 b c v_a2");
+    ts.exec("flush -w");
+    ts.exec("insert x1 b c v_x1");
+    ts.exec("insert x2 b c v_x2");
+    ts.exec("flush -w");
+    int oldCount = countFiles(tableId);
+
+    // no transactions running
+    ts.exec("fate -print", true, "0 transactions", true);
+
+    // merge two files into one
+    ts.exec("compact -t " + table);
+    Thread.sleep(1_000);
+    // start 2nd transaction
+    ts.exec("compact -t " + table);
+    Thread.sleep(3_000);
+
+    // 2 compactions should be running so parse the output to get one of the transaction ids
+    log.info("Calling fate print for table = {}", table);
+    String result = ts.exec("fate -print", true, "txid:", true);
+    String[] resultParts = result.split("txid: ");
+    String[] parts = resultParts[1].split(" ");
+    String txid = parts[0];
+    // test filters
+    ts.exec("fate -print -t IN_PROGRESS", true, "2 transactions", true);
+    ts.exec("fate -print " + txid + " -t IN_PROGRESS", true, "1 transactions", true);
+    ts.exec("fate -print " + txid + " -t FAILED", true, "0 transactions", true);
+    ts.exec("fate -print -t NEW", true, "0 transactions", true);
+    ts.exec("fate -print 1234", true, "0 transactions", true);
+    ts.exec("fate -print FATE[aaa] 1 2 3", true, "0 transactions", true);
+
+    ts.exec("deletetable -f " + table);
+
+    if (orgProps != null) {
+      System.setProperty("accumulo.properties", orgProps);
+    }
+
+  }
 }