accumulo-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From ktur...@apache.org
Subject accumulo git commit: ACCUMULO-4152 add fate dump command
Date Thu, 03 Mar 2016 19:40:54 GMT
Repository: accumulo
Updated Branches:
  refs/heads/master e53253066 -> 0e652f161


ACCUMULO-4152 add fate dump command


Project: http://git-wip-us.apache.org/repos/asf/accumulo/repo
Commit: http://git-wip-us.apache.org/repos/asf/accumulo/commit/0e652f16
Tree: http://git-wip-us.apache.org/repos/asf/accumulo/tree/0e652f16
Diff: http://git-wip-us.apache.org/repos/asf/accumulo/diff/0e652f16

Branch: refs/heads/master
Commit: 0e652f16154f540bd8f7ad7a3e5e19facdaa4b06
Parents: e532530
Author: Keith Turner <keith@deenlo.com>
Authored: Thu Mar 3 19:24:24 2016 -0500
Committer: Keith Turner <keith@deenlo.com>
Committed: Thu Mar 3 19:24:24 2016 -0500

----------------------------------------------------------------------
 .../main/asciidoc/chapters/implementation.txt   | 13 +++-
 .../org/apache/accumulo/fate/AgeOffStore.java   |  5 ++
 .../org/apache/accumulo/fate/ReadOnlyStore.java |  5 ++
 .../apache/accumulo/fate/ReadOnlyTStore.java    |  5 ++
 .../java/org/apache/accumulo/fate/ZooStore.java | 42 +++++++++-
 .../org/apache/accumulo/fate/SimpleStore.java   |  5 ++
 shell/pom.xml                                   |  4 +
 .../accumulo/shell/commands/FateCommand.java    | 81 +++++++++++++++++++-
 8 files changed, 157 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/0e652f16/docs/src/main/asciidoc/chapters/implementation.txt
----------------------------------------------------------------------
diff --git a/docs/src/main/asciidoc/chapters/implementation.txt b/docs/src/main/asciidoc/chapters/implementation.txt
index 9ec66ff..520f538 100644
--- a/docs/src/main/asciidoc/chapters/implementation.txt
+++ b/docs/src/main/asciidoc/chapters/implementation.txt
@@ -49,7 +49,7 @@ For example, a command that is not completing could be blocked on the execution
 operation. Accumulo provides an Accumulo shell command to interact with fate.
 
 The +fate+ shell command accepts a number of arguments for different functionality:
-+list+/+print+, +fail+, +delete+.
++list+/+print+, +fail+, +delete+, +dump+.
 
 ==== List/Print
 
@@ -73,3 +73,14 @@ This command requires a transaction ID and will delete any locks that the
transa
 holds. Like the fail command, this command should only be used in extreme circumstances
 by an administrator that understands the implications of the command they are about to
 invoke. It is not normal to invoke this command.
+
+==== Dump
+
+This command accepts zero more transaction IDs.  If given no transaction IDs,
+it will dump all active transactions.  A FATE operations is compromised as a
+sequence of REPOs.  In order to start a FATE transaction, a REPO is pushed onto
+a per transaction REPO stack.  The top of the stack always contains the next
+REPO the FATE transaction should execute.  When a REPO is successful it may
+return another REPO which is pushed on the stack.  The +dump+ command will
+print all of the REPOs on each transactions stack.  The REPOs are serialized to
+JSON in order to make them human readable.

http://git-wip-us.apache.org/repos/asf/accumulo/blob/0e652f16/fate/src/main/java/org/apache/accumulo/fate/AgeOffStore.java
----------------------------------------------------------------------
diff --git a/fate/src/main/java/org/apache/accumulo/fate/AgeOffStore.java b/fate/src/main/java/org/apache/accumulo/fate/AgeOffStore.java
index d023c27..83b726b 100644
--- a/fate/src/main/java/org/apache/accumulo/fate/AgeOffStore.java
+++ b/fate/src/main/java/org/apache/accumulo/fate/AgeOffStore.java
@@ -231,4 +231,9 @@ public class AgeOffStore<T> implements TStore<T> {
   public List<Long> list() {
     return store.list();
   }
+
+  @Override
+  public List<ReadOnlyRepo<T>> getStack(long tid) {
+    return store.getStack(tid);
+  }
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/0e652f16/fate/src/main/java/org/apache/accumulo/fate/ReadOnlyStore.java
----------------------------------------------------------------------
diff --git a/fate/src/main/java/org/apache/accumulo/fate/ReadOnlyStore.java b/fate/src/main/java/org/apache/accumulo/fate/ReadOnlyStore.java
index 0ca59dd..cf2683b 100644
--- a/fate/src/main/java/org/apache/accumulo/fate/ReadOnlyStore.java
+++ b/fate/src/main/java/org/apache/accumulo/fate/ReadOnlyStore.java
@@ -108,4 +108,9 @@ public class ReadOnlyStore<T> implements ReadOnlyTStore<T>
{
   public List<Long> list() {
     return store.list();
   }
+
+  @Override
+  public List<ReadOnlyRepo<T>> getStack(long tid) {
+    return store.getStack(tid);
+  }
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/0e652f16/fate/src/main/java/org/apache/accumulo/fate/ReadOnlyTStore.java
----------------------------------------------------------------------
diff --git a/fate/src/main/java/org/apache/accumulo/fate/ReadOnlyTStore.java b/fate/src/main/java/org/apache/accumulo/fate/ReadOnlyTStore.java
index 5c1344a..9039ad2 100644
--- a/fate/src/main/java/org/apache/accumulo/fate/ReadOnlyTStore.java
+++ b/fate/src/main/java/org/apache/accumulo/fate/ReadOnlyTStore.java
@@ -87,6 +87,11 @@ public interface ReadOnlyTStore<T> {
   ReadOnlyRepo<T> top(long tid);
 
   /**
+   * Get all operations on a transactions stack. Element 0 contains the most recent operation
pushed or the top.
+   */
+  List<ReadOnlyRepo<T>> getStack(long tid);
+
+  /**
    * Get the state of a given transaction.
    *
    * Caller must have already reserved tid.

http://git-wip-us.apache.org/repos/asf/accumulo/blob/0e652f16/fate/src/main/java/org/apache/accumulo/fate/ZooStore.java
----------------------------------------------------------------------
diff --git a/fate/src/main/java/org/apache/accumulo/fate/ZooStore.java b/fate/src/main/java/org/apache/accumulo/fate/ZooStore.java
index 4b4a83f..36a88e3 100644
--- a/fate/src/main/java/org/apache/accumulo/fate/ZooStore.java
+++ b/fate/src/main/java/org/apache/accumulo/fate/ZooStore.java
@@ -16,8 +16,8 @@
  */
 package org.apache.accumulo.fate;
 
-import static java.nio.charset.StandardCharsets.UTF_8;
 import static com.google.common.util.concurrent.Uninterruptibles.sleepUninterruptibly;
+import static java.nio.charset.StandardCharsets.UTF_8;
 
 import java.io.ByteArrayInputStream;
 import java.io.ByteArrayOutputStream;
@@ -458,4 +458,44 @@ public class ZooStore<T> implements TStore<T> {
       throw new RuntimeException(e);
     }
   }
+
+  @Override
+  public List<ReadOnlyRepo<T>> getStack(long tid) {
+    String txpath = getTXPath(tid);
+
+    outer: while (true) {
+      List<String> ops;
+      try {
+        ops = zk.getChildren(txpath);
+      } catch (KeeperException.NoNodeException e) {
+        return Collections.emptyList();
+      } catch (KeeperException | InterruptedException e1) {
+        throw new RuntimeException(e1);
+      }
+
+      ops = new ArrayList<String>(ops);
+      Collections.sort(ops, Collections.reverseOrder());
+
+      ArrayList<ReadOnlyRepo<T>> dops = new ArrayList<>();
+
+      for (String child : ops) {
+        if (child.startsWith("repo_")) {
+          byte[] ser;
+          try {
+            ser = zk.getData(txpath + "/" + child, null);
+            @SuppressWarnings("unchecked")
+            ReadOnlyRepo<T> repo = (ReadOnlyRepo<T>) deserialize(ser);
+            dops.add(repo);
+          } catch (KeeperException.NoNodeException e) {
+            // children changed so start over
+            continue outer;
+          } catch (KeeperException | InterruptedException e) {
+            throw new RuntimeException(e);
+          }
+        }
+      }
+
+      return dops;
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/0e652f16/fate/src/test/java/org/apache/accumulo/fate/SimpleStore.java
----------------------------------------------------------------------
diff --git a/fate/src/test/java/org/apache/accumulo/fate/SimpleStore.java b/fate/src/test/java/org/apache/accumulo/fate/SimpleStore.java
index f0bac88..ab03525 100644
--- a/fate/src/test/java/org/apache/accumulo/fate/SimpleStore.java
+++ b/fate/src/test/java/org/apache/accumulo/fate/SimpleStore.java
@@ -123,4 +123,9 @@ public class SimpleStore<T> implements TStore<T> {
     return new ArrayList<Long>(statuses.keySet());
   }
 
+  @Override
+  public List<ReadOnlyRepo<T>> getStack(long tid) {
+    throw new NotImplementedException();
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/0e652f16/shell/pom.xml
----------------------------------------------------------------------
diff --git a/shell/pom.xml b/shell/pom.xml
index 04bbb88..93fa651 100644
--- a/shell/pom.xml
+++ b/shell/pom.xml
@@ -36,6 +36,10 @@
       <optional>true</optional>
     </dependency>
     <dependency>
+      <groupId>com.google.code.gson</groupId>
+      <artifactId>gson</artifactId>
+    </dependency>
+    <dependency>
       <groupId>com.google.guava</groupId>
       <artifactId>guava</artifactId>
     </dependency>

http://git-wip-us.apache.org/repos/asf/accumulo/blob/0e652f16/shell/src/main/java/org/apache/accumulo/shell/commands/FateCommand.java
----------------------------------------------------------------------
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 cef9a6d..e5d6c59 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
@@ -17,10 +17,14 @@
 package org.apache.accumulo.shell.commands;
 
 import java.io.IOException;
+import java.lang.reflect.Type;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
 import java.util.Collections;
 import java.util.EnumSet;
 import java.util.Formatter;
 import java.util.HashSet;
+import java.util.List;
 import java.util.Set;
 
 import org.apache.accumulo.core.Constants;
@@ -29,9 +33,12 @@ import org.apache.accumulo.core.conf.AccumuloConfiguration;
 import org.apache.accumulo.core.conf.DefaultConfiguration;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.conf.SiteConfiguration;
+import org.apache.accumulo.core.util.Base64;
 import org.apache.accumulo.core.zookeeper.ZooUtil;
 import org.apache.accumulo.fate.AdminUtil;
+import org.apache.accumulo.fate.ReadOnlyRepo;
 import org.apache.accumulo.fate.ReadOnlyTStore.TStatus;
+import org.apache.accumulo.fate.Repo;
 import org.apache.accumulo.fate.ZooStore;
 import org.apache.accumulo.fate.zookeeper.IZooReaderWriter;
 import org.apache.accumulo.fate.zookeeper.ZooReaderWriter;
@@ -43,6 +50,13 @@ import org.apache.commons.cli.Options;
 import org.apache.commons.cli.ParseException;
 import org.apache.zookeeper.KeeperException;
 
+import com.google.gson.Gson;
+import com.google.gson.GsonBuilder;
+import com.google.gson.JsonElement;
+import com.google.gson.JsonObject;
+import com.google.gson.JsonSerializationContext;
+import com.google.gson.JsonSerializer;
+
 /**
  * Manage FATE transactions
  *
@@ -53,6 +67,47 @@ public class FateCommand extends Command {
 
   private static final String USER = "accumulo";
 
+  // this class serializes references to interfaces with the concrete class name
+  private static class InterfaceSerializer<T> implements JsonSerializer<T> {
+    @Override
+    public JsonElement serialize(T link, Type type, JsonSerializationContext context) {
+      JsonElement je = context.serialize(link, link.getClass());
+      JsonObject jo = new JsonObject();
+      jo.add(link.getClass().getName(), je);
+      return jo;
+    }
+  }
+
+  // the purpose of this class is to be serialized as JSon for display
+  public static class ByteArrayContainer {
+    public String asUtf8;
+    public String asBase64;
+
+    ByteArrayContainer(byte[] ba) {
+      asUtf8 = new String(ba, StandardCharsets.UTF_8);
+      asBase64 = Base64.encodeBase64URLSafeString(ba);
+    }
+  }
+
+  // serialize byte arrays in human and machine readable ways
+  private static class ByteArraySerializer implements JsonSerializer<byte[]> {
+    @Override
+    public JsonElement serialize(byte[] link, Type type, JsonSerializationContext context)
{
+      return context.serialize(new ByteArrayContainer(link));
+    }
+  }
+
+  // the purpose of this class is to be serialized as JSon for display
+  public static class FateStack {
+    String txid;
+    List<ReadOnlyRepo<FateCommand>> stack;
+
+    FateStack(Long txid, List<ReadOnlyRepo<FateCommand>> stack) {
+      this.txid = String.format("%016x", txid);
+      this.stack = stack;
+    }
+  }
+
   private Option secretOption;
   private Option statusOption;
   private Option disablePaginationOpt;
@@ -133,6 +188,30 @@ public class FateCommand extends Command {
       Formatter fmt = new Formatter(buf);
       admin.print(zs, zk, ZooUtil.getRoot(instance) + Constants.ZTABLE_LOCKS, fmt, filterTxid,
filterStatus);
       shellState.printLines(Collections.singletonList(buf.toString()).iterator(), !cl.hasOption(disablePaginationOpt.getOpt()));
+    } else if ("dump".equals(cmd)) {
+      List<Long> txids;
+
+      if (args.length == 1) {
+        txids = zs.list();
+      } else {
+        txids = new ArrayList<>();
+        for (int i = 1; i < args.length; i++) {
+          txids.add(Long.parseLong(args[i], 16));
+        }
+      }
+
+      Gson gson = new GsonBuilder().registerTypeAdapter(ReadOnlyRepo.class, new InterfaceSerializer<>())
+          .registerTypeAdapter(Repo.class, new InterfaceSerializer<>()).registerTypeAdapter(byte[].class,
new ByteArraySerializer()).setPrettyPrinting()
+          .create();
+
+      List<FateStack> txStacks = new ArrayList<>();
+
+      for (Long txid : txids) {
+        List<ReadOnlyRepo<FateCommand>> repoStack = zs.getStack(txid);
+        txStacks.add(new FateStack(txid, repoStack));
+      }
+
+      System.out.println(gson.toJson(txStacks));
     } else {
       throw new ParseException("Invalid command option");
     }
@@ -157,7 +236,7 @@ public class FateCommand extends Command {
 
   @Override
   public String usage() {
-    return getName() + " fail <txid>... | delete <txid>... | print [<txid>...]";
+    return getName() + " fail <txid>... | delete <txid>... | print [<txid>...]
| dump [<txid>...]";
   }
 
   @Override


Mime
View raw message