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

dlmarion 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 16dba3d3c9 Move fate summary IT from ShellServerIT to its own class 
(#2950)
16dba3d3c9 is described below

commit 16dba3d3c9716c3c085b6707a48431d2302fab0f
Author: Dave Marion <dlmar...@apache.org>
AuthorDate: Thu Sep 22 09:46:50 2022 -0400

    Move fate summary IT from ShellServerIT to its own class (#2950)
    
    Closes #2948
---
 .../org/apache/accumulo/server/util/Admin.java     |   2 +-
 .../server/util/fateCommand/FateTxnDetails.java    |   4 +
 .../org/apache/accumulo/test/FateSummaryIT.java    | 156 +++++++++++++++++++++
 .../apache/accumulo/test/shell/ShellServerIT.java  |  97 -------------
 4 files changed, 161 insertions(+), 98 deletions(-)

diff --git 
a/server/base/src/main/java/org/apache/accumulo/server/util/Admin.java 
b/server/base/src/main/java/org/apache/accumulo/server/util/Admin.java
index 8bf5274ad7..1741911d48 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/util/Admin.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/util/Admin.java
@@ -251,7 +251,7 @@ public class Admin implements KeywordExecutable {
     boolean printJson;
 
     @Parameter(names = {"-s", "--state"},
-        description = "<state>[ <state>...] Print transactions in the state(s) 
{NEW, IN_PROGRESS, FAILED_IN_PROGRESS, FAILED, SUCCESSFUL}")
+        description = "<state>[ -s <state>...] Print transactions in the 
state(s) {NEW, IN_PROGRESS, FAILED_IN_PROGRESS, FAILED, SUCCESSFUL}")
     List<String> states = new ArrayList<>();
   }
 
diff --git 
a/server/base/src/main/java/org/apache/accumulo/server/util/fateCommand/FateTxnDetails.java
 
b/server/base/src/main/java/org/apache/accumulo/server/util/fateCommand/FateTxnDetails.java
index 6fc4fccbb6..d2022c9f3b 100644
--- 
a/server/base/src/main/java/org/apache/accumulo/server/util/fateCommand/FateTxnDetails.java
+++ 
b/server/base/src/main/java/org/apache/accumulo/server/util/fateCommand/FateTxnDetails.java
@@ -95,6 +95,10 @@ public class FateTxnDetails implements 
Comparable<FateTxnDetails> {
     return formattedLocks;
   }
 
+  public String getTxnId() {
+    return txnId;
+  }
+
   /**
    * Sort by running time in reverse (oldest txn first). txid is unique as 
used to break times and
    * so that compareTo remains consistent with hashCode and equals methods.
diff --git a/test/src/main/java/org/apache/accumulo/test/FateSummaryIT.java 
b/test/src/main/java/org/apache/accumulo/test/FateSummaryIT.java
new file mode 100644
index 0000000000..d868e5e8bb
--- /dev/null
+++ b/test/src/main/java/org/apache/accumulo/test/FateSummaryIT.java
@@ -0,0 +1,156 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   https://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.accumulo.test;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertNotEquals;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+
+import java.time.Duration;
+import java.util.ArrayList;
+import java.util.EnumSet;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.SortedSet;
+import java.util.TreeSet;
+
+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.IteratorScope;
+import org.apache.accumulo.miniclusterImpl.MiniAccumuloClusterImpl.ProcessInfo;
+import org.apache.accumulo.miniclusterImpl.MiniAccumuloConfigImpl;
+import org.apache.accumulo.server.util.Admin;
+import org.apache.accumulo.server.util.fateCommand.FateSummaryReport;
+import org.apache.accumulo.test.functional.ConfigurableMacBase;
+import org.apache.accumulo.test.functional.ReadWriteIT;
+import org.apache.accumulo.test.functional.SlowIterator;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.Text;
+import org.junit.jupiter.api.Test;
+
+public class FateSummaryIT extends ConfigurableMacBase {
+
+  @Override
+  protected Duration defaultTimeout() {
+    return Duration.ofMinutes(2);
+  }
+
+  @Override
+  public void configure(MiniAccumuloConfigImpl cfg, Configuration 
hadoopCoreSite) {}
+
+  @Test
+  public void testFateSummaryCommandWithSlowCompaction() throws Exception {
+    try (AccumuloClient client = 
Accumulo.newClient().from(getClientProperties()).build()) {
+      String namespace = "ns1";
+      final String table = namespace + "." + getUniqueNames(1)[0];
+      client.namespaceOperations().create(namespace);
+
+      SortedSet<Text> splits = new TreeSet<Text>();
+      splits.add(new Text("h"));
+      splits.add(new Text("m"));
+      splits.add(new Text("r"));
+      splits.add(new Text("w"));
+      IteratorSetting is = new IteratorSetting(1, SlowIterator.class);
+      is.addOption("sleepTime", "10000");
+
+      NewTableConfiguration cfg = new NewTableConfiguration();
+      cfg.withSplits(splits);
+      cfg.attachIterator(is, EnumSet.of(IteratorScope.majc));
+      client.tableOperations().create(table, cfg);
+
+      ReadWriteIT.ingest(client, 10, 10, 10, 0, table);
+      client.tableOperations().flush(table);
+
+      // validate blank report, compactions have not started yet
+      ProcessInfo p = getCluster().exec(Admin.class, "fate", "--summary", 
"-j", "-s", "NEW", "-s",
+          "IN_PROGRESS", "-s", "FAILED");
+      assertEquals(0, p.getProcess().waitFor());
+      String result = p.readStdOut();
+      result = result.substring(result.indexOf("{"), result.lastIndexOf("}") + 
1);
+      FateSummaryReport report = FateSummaryReport.fromJson(result);
+      assertNotNull(report);
+      assertNotEquals(0, report.getReportTime());
+      Set<String> expected = new HashSet<>();
+      expected.add("FAILED");
+      expected.add("IN_PROGRESS");
+      expected.add("NEW");
+      assertEquals(expected, report.getStatusFilterNames());
+      assertEquals(Map.of(), report.getStatusCounts());
+      assertEquals(Map.of(), report.getStepCounts());
+      assertEquals(Map.of(), report.getCmdCounts());
+
+      // create Fate transactions
+      client.tableOperations().compact(table, null, null, false, false);
+      client.tableOperations().compact(table, null, null, false, false);
+
+      // validate no filters
+      p = getCluster().exec(Admin.class, "fate", "--summary", "-j");
+      assertEquals(0, p.getProcess().waitFor());
+      result = p.readStdOut();
+      result = result.substring(result.indexOf("{"), result.lastIndexOf("}") + 
1);
+      report = FateSummaryReport.fromJson(result);
+      assertNotNull(report);
+      assertNotEquals(0, report.getReportTime());
+      assertEquals(Set.of(), report.getStatusFilterNames());
+      assertFalse(report.getStatusCounts().isEmpty());
+      assertFalse(report.getStepCounts().isEmpty());
+      assertFalse(report.getCmdCounts().isEmpty());
+      assertEquals(2, report.getFateDetails().size());
+      ArrayList<String> txns = new ArrayList<>();
+      report.getFateDetails().forEach((d) -> {
+        txns.add(d.getTxnId());
+      });
+      assertEquals(2, txns.size());
+
+      // validate tx ids
+      p = getCluster().exec(Admin.class, "fate", txns.get(0), txns.get(1), 
"--summary", "-j");
+      assertEquals(0, p.getProcess().waitFor());
+      result = p.readStdOut();
+      result = result.substring(result.indexOf("{"), result.lastIndexOf("}") + 
1);
+      report = FateSummaryReport.fromJson(result);
+      assertNotNull(report);
+      assertNotEquals(0, report.getReportTime());
+      assertEquals(Set.of(), report.getStatusFilterNames());
+      assertFalse(report.getStatusCounts().isEmpty());
+      assertFalse(report.getStepCounts().isEmpty());
+      assertFalse(report.getCmdCounts().isEmpty());
+      assertEquals(2, report.getFateDetails().size());
+
+      // validate filter by including only FAILED transactions, should be none
+      p = getCluster().exec(Admin.class, "fate", "--summary", "-j", "-s", 
"FAILED");
+      assertEquals(0, p.getProcess().waitFor());
+      result = p.readStdOut();
+      result = result.substring(result.indexOf("{"), result.lastIndexOf("}") + 
1);
+      report = FateSummaryReport.fromJson(result);
+      assertNotNull(report);
+      assertNotEquals(0, report.getReportTime());
+      assertEquals(Set.of("FAILED"), report.getStatusFilterNames());
+      assertFalse(report.getStatusCounts().isEmpty());
+      assertFalse(report.getStepCounts().isEmpty());
+      assertFalse(report.getCmdCounts().isEmpty());
+      assertEquals(0, report.getFateDetails().size());
+
+    }
+  }
+
+}
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 1ba1ebf1f3..9f5c9944c4 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
@@ -24,7 +24,6 @@ import static 
org.apache.accumulo.harness.AccumuloITBase.MINI_CLUSTER_ONLY;
 import static org.apache.accumulo.harness.AccumuloITBase.SUNNY_DAY;
 import static org.junit.jupiter.api.Assertions.assertEquals;
 import static org.junit.jupiter.api.Assertions.assertFalse;
-import static org.junit.jupiter.api.Assertions.assertNotEquals;
 import static org.junit.jupiter.api.Assertions.assertNotNull;
 import static org.junit.jupiter.api.Assertions.assertThrows;
 import static org.junit.jupiter.api.Assertions.assertTrue;
@@ -45,7 +44,6 @@ import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
-import java.util.Set;
 import java.util.concurrent.TimeUnit;
 import java.util.regex.Pattern;
 
@@ -78,7 +76,6 @@ import org.apache.accumulo.core.util.format.FormatterConfig;
 import org.apache.accumulo.harness.MiniClusterConfigurationCallback;
 import org.apache.accumulo.harness.SharedMiniClusterBase;
 import org.apache.accumulo.miniclusterImpl.MiniAccumuloConfigImpl;
-import org.apache.accumulo.server.util.fateCommand.FateSummaryReport;
 import org.apache.accumulo.test.compaction.TestCompactionStrategy;
 import org.apache.accumulo.test.functional.SlowIterator;
 import org.apache.hadoop.conf.Configuration;
@@ -2140,98 +2137,4 @@ public class ShellServerIT extends SharedMiniClusterBase 
{
     }
   }
 
-  @Test
-  public void testFateSummaryCommandWithSlowCompaction() throws Exception {
-    String namespace = "ns1";
-    final String table = namespace + "." + getUniqueNames(1)[0];
-
-    String orgProps = System.getProperty("accumulo.properties");
-
-    System.setProperty("accumulo.properties",
-        "file://" + 
getCluster().getConfig().getAccumuloPropsFile().getCanonicalPath());
-    // compact
-    ts.exec("createnamespace " + namespace);
-    ts.exec("createtable " + table);
-    ts.exec("addsplits h m r w -t " + table);
-    ts.exec("offline -t " + table);
-    ts.exec("online h m r w -t " + 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");
-
-    // 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");
-
-    // no transactions running
-
-    String cmdOut =
-        ts.exec("fate -summary -np json -t NEW IN_PROGRESS FAILED", true, 
"reportTime", true);
-    // strip command included in shell output
-    String jsonOut = cmdOut.substring(cmdOut.indexOf("{"));
-    FateSummaryReport report = FateSummaryReport.fromJson(jsonOut);
-
-    // validate blank report
-    assertNotNull(report);
-    assertNotEquals(0, report.getReportTime());
-    assertEquals(Set.of("NEW", "IN_PROGRESS", "FAILED"), 
report.getStatusFilterNames());
-    assertEquals(Map.of(), report.getStatusCounts());
-    assertEquals(Map.of(), report.getStepCounts());
-    assertEquals(Map.of(), report.getCmdCounts());
-    assertEquals(Set.of(), report.getFateDetails());
-
-    ts.exec("fate -summary -np", true, "Report Time:", 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.debug("Calling fate summary");
-    ts.exec("fate -summary -np", true, "Report Time:", true);
-
-    cmdOut = ts.exec("fate -summary -np json", true, "reportTime", true);
-    // strip command included in shell output
-    jsonOut = cmdOut.substring(cmdOut.indexOf("{"));
-    log.debug("report to json:\n{}", jsonOut);
-    report = FateSummaryReport.fromJson(jsonOut);
-
-    // validate no filters
-    assertNotNull(report);
-    assertNotEquals(0, report.getReportTime());
-    assertEquals(Set.of(), report.getStatusFilterNames());
-    assertFalse(report.getStatusCounts().isEmpty());
-    assertFalse(report.getStepCounts().isEmpty());
-    assertFalse(report.getCmdCounts().isEmpty());
-    assertFalse(report.getFateDetails().isEmpty());
-
-    // validate filter by excluding all
-    cmdOut = ts.exec("fate -summary -np json -t FAILED", true, "reportTime", 
true);
-    jsonOut = cmdOut.substring(cmdOut.indexOf("{"));
-    report = FateSummaryReport.fromJson(jsonOut);
-
-    // validate blank report
-    assertNotNull(report);
-    assertNotEquals(0, report.getReportTime());
-    assertEquals(Set.of("FAILED"), report.getStatusFilterNames());
-    assertFalse(report.getStatusCounts().isEmpty());
-    assertFalse(report.getStepCounts().isEmpty());
-    assertFalse(report.getCmdCounts().isEmpty());
-    assertEquals(0, report.getFateDetails().size());
-
-    ts.exec("deletetable -f " + table);
-
-    if (orgProps != null) {
-      System.setProperty("accumulo.properties", orgProps);
-    }
-  }
 }

Reply via email to