You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@accumulo.apache.org by cs...@apache.org on 2024/01/12 23:03:43 UTC

(accumulo) branch elasticity updated: Add support for multiple Fate instances to the Manager (#4133)

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

cshannon pushed a commit to branch elasticity
in repository https://gitbox.apache.org/repos/asf/accumulo.git


The following commit(s) were added to refs/heads/elasticity by this push:
     new 738cabb6b6 Add support for multiple Fate instances to the Manager (#4133)
738cabb6b6 is described below

commit 738cabb6b686c0705aeb821613cc18952079e958
Author: Christopher L. Shannon <ch...@gmail.com>
AuthorDate: Fri Jan 12 18:03:38 2024 -0500

    Add support for multiple Fate instances to the Manager (#4133)
    
    This commit updates the Manager to create two fate instances, one for
    store FATE operations against tables in the accumulo namespace (meta) in
    the existing zookeeper store and another fate instance to store FATE
    operations against user tables (user). The thrift API has been modified
    so that when acquiring a new transaction, the type is returned of the
    instance used because future operations with the same transaction will
    need to know which instance was used.
    
    The Fate Admin command needs to be updated now that multiple instance
    types are used for Fate operations. This commit fixes the ITs that call
    the admin command operations but there is still more work to be done to
    complete all the changes.
    
    Co-authored-by: Keith Turner <kt...@apache.org>
---
 .../core/clientImpl/TableOperationsImpl.java       |  20 +-
 .../org/apache/accumulo/core/fate/AdminUtil.java   | 110 ++--
 .../accumulo/core/fate/FateInstanceType.java       |  66 +++
 .../accumulo/core/fate/accumulo/AccumuloStore.java |   5 +
 .../apache/accumulo/core/metadata/FateTable.java   |  27 +
 .../accumulo/core/manager/thrift/FateService.java  | 591 +++++++++++++--------
 .../accumulo/core/manager/thrift/TFateId.java      | 522 ++++++++++++++++++
 .../core/manager/thrift/TFateInstanceType.java     |  61 +++
 core/src/main/thrift/manager.thrift                |  21 +-
 .../server/init/FileSystemInitializer.java         |  25 +-
 .../accumulo/server/init/InitialConfiguration.java |   8 +
 .../accumulo/server/init/ZooKeeperInitializer.java |   3 +
 .../org/apache/accumulo/server/util/Admin.java     |  24 +-
 .../accumulo/manager/FateServiceHandler.java       |  74 +--
 .../java/org/apache/accumulo/manager/Manager.java  |  63 ++-
 .../manager/ManagerClientServiceHandler.java       |   3 +-
 .../manager/metrics/fate/FateMetricValues.java     |   3 +-
 .../apache/accumulo/manager/split/SplitTask.java   |   7 +-
 .../java/org/apache/accumulo/test/MetaSplitIT.java |   9 +-
 .../org/apache/accumulo/test/NamespacesIT.java     |   2 +-
 .../test/functional/FateConcurrencyIT.java         |  46 +-
 .../apache/accumulo/test/functional/SplitIT.java   |   2 +
 .../functional/TabletManagementIteratorIT.java     |   8 +-
 .../accumulo/test/functional/WALSunnyDayIT.java    |  13 +-
 24 files changed, 1371 insertions(+), 342 deletions(-)

diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/TableOperationsImpl.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/TableOperationsImpl.java
index 637e0c193b..09e4174dd7 100644
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/TableOperationsImpl.java
+++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/TableOperationsImpl.java
@@ -132,12 +132,15 @@ import org.apache.accumulo.core.dataImpl.thrift.TRowRange;
 import org.apache.accumulo.core.dataImpl.thrift.TSummaries;
 import org.apache.accumulo.core.dataImpl.thrift.TSummarizerConfiguration;
 import org.apache.accumulo.core.dataImpl.thrift.TSummaryRequest;
+import org.apache.accumulo.core.fate.FateInstanceType;
 import org.apache.accumulo.core.iterators.IteratorUtil.IteratorScope;
 import org.apache.accumulo.core.iterators.SortedKeyValueIterator;
 import org.apache.accumulo.core.manager.state.tables.TableState;
 import org.apache.accumulo.core.manager.thrift.FateOperation;
 import org.apache.accumulo.core.manager.thrift.FateService;
 import org.apache.accumulo.core.manager.thrift.ManagerClientService;
+import org.apache.accumulo.core.manager.thrift.TFateId;
+import org.apache.accumulo.core.manager.thrift.TFateInstanceType;
 import org.apache.accumulo.core.metadata.MetadataTable;
 import org.apache.accumulo.core.metadata.RootTable;
 import org.apache.accumulo.core.metadata.TServerInstance;
@@ -277,12 +280,13 @@ public class TableOperationsImpl extends TableOperationsHelper {
     }
   }
 
-  private long beginFateOperation() throws ThriftSecurityException, TException {
+  private TFateId beginFateOperation(TFateInstanceType type)
+      throws ThriftSecurityException, TException {
     while (true) {
       FateService.Client client = null;
       try {
         client = ThriftClientTypes.FATE.getConnectionWithRetry(context);
-        return client.beginFateOperation(TraceUtil.traceInfo(), context.rpcCreds());
+        return client.beginFateOperation(TraceUtil.traceInfo(), context.rpcCreds(), type);
       } catch (TTransportException tte) {
         log.debug("Failed to call beginFateOperation(), retrying ... ", tte);
         sleepUninterruptibly(100, MILLISECONDS);
@@ -298,7 +302,7 @@ public class TableOperationsImpl extends TableOperationsHelper {
 
   // This method is for retrying in the case of network failures;
   // anything else it passes to the caller to deal with
-  private void executeFateOperation(long opid, FateOperation op, List<ByteBuffer> args,
+  private void executeFateOperation(TFateId opid, FateOperation op, List<ByteBuffer> args,
       Map<String,String> opts, boolean autoCleanUp)
       throws ThriftSecurityException, TException, ThriftTableOperationException {
     while (true) {
@@ -321,7 +325,7 @@ public class TableOperationsImpl extends TableOperationsHelper {
     }
   }
 
-  private String waitForFateOperation(long opid)
+  private String waitForFateOperation(TFateId opid)
       throws ThriftSecurityException, TException, ThriftTableOperationException {
     while (true) {
       FateService.Client client = null;
@@ -341,7 +345,7 @@ public class TableOperationsImpl extends TableOperationsHelper {
     }
   }
 
-  private void finishFateOperation(long opid) throws ThriftSecurityException, TException {
+  private void finishFateOperation(TFateId opid) throws ThriftSecurityException, TException {
     while (true) {
       FateService.Client client = null;
       try {
@@ -387,10 +391,12 @@ public class TableOperationsImpl extends TableOperationsHelper {
       String tableOrNamespaceName, boolean wait)
       throws AccumuloSecurityException, TableExistsException, TableNotFoundException,
       AccumuloException, NamespaceExistsException, NamespaceNotFoundException {
-    Long opid = null;
+    TFateId opid = null;
 
     try {
-      opid = beginFateOperation();
+      TFateInstanceType t =
+          FateInstanceType.fromNamespaceOrTableName(tableOrNamespaceName).toThrift();
+      opid = beginFateOperation(t);
       executeFateOperation(opid, op, args, opts, !wait);
       if (!wait) {
         opid = null;
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 590b9fbb12..bbb7f42572 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
@@ -75,6 +75,7 @@ public class AdminUtil<T> {
   public static class TransactionStatus {
 
     private final long txid;
+    private final FateInstanceType instanceType;
     private final TStatus status;
     private final String txName;
     private final List<String> hlocks;
@@ -82,10 +83,11 @@ public class AdminUtil<T> {
     private final String top;
     private final long timeCreated;
 
-    private TransactionStatus(Long tid, TStatus status, String txName, List<String> hlocks,
-        List<String> wlocks, String top, Long timeCreated) {
+    private TransactionStatus(Long tid, FateInstanceType instanceType, TStatus status,
+        String txName, List<String> hlocks, List<String> wlocks, String top, Long timeCreated) {
 
       this.txid = tid;
+      this.instanceType = instanceType;
       this.status = status;
       this.txName = txName;
       this.hlocks = Collections.unmodifiableList(hlocks);
@@ -103,6 +105,10 @@ public class AdminUtil<T> {
       return FastFormat.toHexString(txid);
     }
 
+    public FateInstanceType getInstanceType() {
+      return instanceType;
+    }
+
     public TStatus getStatus() {
       return status;
     }
@@ -216,15 +222,16 @@ public class AdminUtil<T> {
    * method does not process lock information, if lock information is desired, use
    * {@link #getStatus(ReadOnlyFateStore, ZooReader, ServiceLockPath, Set, EnumSet)}
    *
-   * @param zs read-only zoostore
+   * @param fateStores read-only fate stores
    * @param filterTxid filter results to include for provided transaction ids.
    * @param filterStatus filter results to include only provided status types
    * @return list of FATE transactions that match filter criteria
    */
-  public List<TransactionStatus> getTransactionStatus(ReadOnlyFateStore<T> zs, Set<Long> filterTxid,
+  public List<TransactionStatus> getTransactionStatus(
+      Map<FateInstanceType,ReadOnlyFateStore<T>> fateStores, Set<Long> filterTxid,
       EnumSet<TStatus> filterStatus) {
 
-    FateStatus status = getTransactionStatus(zs, filterTxid, filterStatus,
+    FateStatus status = getTransactionStatus(fateStores, filterTxid, filterStatus,
         Collections.<Long,List<String>>emptyMap(), Collections.<Long,List<String>>emptyMap());
 
     return status.getTransactions();
@@ -251,7 +258,26 @@ public class AdminUtil<T> {
 
     findLocks(zk, lockPath, heldLocks, waitingLocks);
 
-    return getTransactionStatus(zs, filterTxid, filterStatus, heldLocks, waitingLocks);
+    return getTransactionStatus(Map.of(FateInstanceType.META, zs), filterTxid, filterStatus,
+        heldLocks, waitingLocks);
+  }
+
+  public FateStatus getStatus(ReadOnlyFateStore<T> as, Set<Long> filterTxid,
+      EnumSet<TStatus> filterStatus) throws KeeperException, InterruptedException {
+
+    return getTransactionStatus(Map.of(FateInstanceType.USER, as), filterTxid, filterStatus,
+        new HashMap<>(), new HashMap<>());
+  }
+
+  public FateStatus getStatus(Map<FateInstanceType,ReadOnlyFateStore<T>> fateStores, 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(fateStores, filterTxid, filterStatus, heldLocks, waitingLocks);
   }
 
   /**
@@ -327,7 +353,7 @@ public class AdminUtil<T> {
   /**
    * Returns fate status, possibly filtered
    *
-   * @param zs read-only access to a populated transaction store.
+   * @param fateStores read-only access to populated transaction stores.
    * @param filterTxid Optional. List of transactions to filter results - if null, all transactions
    *        are returned
    * @param filterStatus Optional. List of status types to filter results - if null, all
@@ -336,49 +362,49 @@ 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(ReadOnlyFateStore<T> zs, Set<Long> filterTxid,
-      EnumSet<TStatus> filterStatus, Map<Long,List<String>> heldLocks,
+  private FateStatus getTransactionStatus(Map<FateInstanceType,ReadOnlyFateStore<T>> fateStores,
+      Set<Long> filterTxid, EnumSet<TStatus> filterStatus, Map<Long,List<String>> heldLocks,
       Map<Long,List<String>> waitingLocks) {
+    final List<TransactionStatus> statuses = new ArrayList<>();
 
-    try (Stream<Long> tids = zs.list()) {
-      List<TransactionStatus> statuses = new ArrayList<>();
+    fateStores.forEach((type, store) -> {
+      try (Stream<Long> tids = store.list()) {
+        tids.forEach(tid -> {
 
-      tids.forEach(tid -> {
+          ReadOnlyFateTxStore<T> txStore = store.read(tid);
 
-        ReadOnlyFateTxStore<T> txStore = zs.read(tid);
+          String txName = (String) txStore.getTransactionInfo(Fate.TxInfo.TX_NAME);
 
-        String txName = (String) txStore.getTransactionInfo(Fate.TxInfo.TX_NAME);
+          List<String> hlocks = heldLocks.remove(tid);
 
-        List<String> hlocks = heldLocks.remove(tid);
-
-        if (hlocks == null) {
-          hlocks = Collections.emptyList();
-        }
-
-        List<String> wlocks = waitingLocks.remove(tid);
+          if (hlocks == null) {
+            hlocks = Collections.emptyList();
+          }
 
-        if (wlocks == null) {
-          wlocks = Collections.emptyList();
-        }
+          List<String> wlocks = waitingLocks.remove(tid);
 
-        String top = null;
-        ReadOnlyRepo<T> repo = txStore.top();
-        if (repo != null) {
-          top = repo.getName();
-        }
+          if (wlocks == null) {
+            wlocks = Collections.emptyList();
+          }
 
-        TStatus status = txStore.getStatus();
+          String top = null;
+          ReadOnlyRepo<T> repo = txStore.top();
+          if (repo != null) {
+            top = repo.getName();
+          }
 
-        long timeCreated = txStore.timeCreated();
+          TStatus status = txStore.getStatus();
 
-        if (includeByStatus(status, filterStatus) && includeByTxid(tid, filterTxid)) {
-          statuses
-              .add(new TransactionStatus(tid, status, txName, hlocks, wlocks, top, timeCreated));
-        }
-      });
+          long timeCreated = txStore.timeCreated();
 
-      return new FateStatus(statuses, heldLocks, waitingLocks);
-    }
+          if (includeByStatus(status, filterStatus) && includeByTxid(tid, filterTxid)) {
+            statuses.add(
+                new TransactionStatus(tid, type, status, txName, hlocks, wlocks, top, timeCreated));
+          }
+        });
+      }
+    });
+    return new FateStatus(statuses, heldLocks, waitingLocks);
   }
 
   private boolean includeByStatus(TStatus status, EnumSet<TStatus> filterStatus) {
@@ -389,15 +415,15 @@ public class AdminUtil<T> {
     return (filterTxid == null) || filterTxid.isEmpty() || filterTxid.contains(tid);
   }
 
-  public void printAll(ReadOnlyFateStore<T> zs, ZooReader zk,
+  public void printAll(Map<FateInstanceType,ReadOnlyFateStore<T>> fateStores, ZooReader zk,
       ServiceLock.ServiceLockPath tableLocksPath) throws KeeperException, InterruptedException {
-    print(zs, zk, tableLocksPath, new Formatter(System.out), null, null);
+    print(fateStores, zk, tableLocksPath, new Formatter(System.out), null, null);
   }
 
-  public void print(ReadOnlyFateStore<T> zs, ZooReader zk,
+  public void print(Map<FateInstanceType,ReadOnlyFateStore<T>> fateStores, ZooReader zk,
       ServiceLock.ServiceLockPath tableLocksPath, Formatter fmt, Set<Long> filterTxid,
       EnumSet<TStatus> filterStatus) throws KeeperException, InterruptedException {
-    FateStatus fateStatus = getStatus(zs, zk, tableLocksPath, filterTxid, filterStatus);
+    FateStatus fateStatus = getStatus(fateStores, zk, tableLocksPath, filterTxid, filterStatus);
 
     for (TransactionStatus txStatus : fateStatus.getTransactions()) {
       fmt.format(
diff --git a/core/src/main/java/org/apache/accumulo/core/fate/FateInstanceType.java b/core/src/main/java/org/apache/accumulo/core/fate/FateInstanceType.java
new file mode 100644
index 0000000000..43e8e35d7a
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/fate/FateInstanceType.java
@@ -0,0 +1,66 @@
+/*
+ * 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.core.fate;
+
+import java.util.Set;
+
+import org.apache.accumulo.core.clientImpl.Namespace;
+import org.apache.accumulo.core.data.TableId;
+import org.apache.accumulo.core.manager.thrift.TFateInstanceType;
+import org.apache.accumulo.core.metadata.FateTable;
+import org.apache.accumulo.core.metadata.MetadataTable;
+import org.apache.accumulo.core.metadata.RootTable;
+
+public enum FateInstanceType {
+  META, USER;
+
+  private static final Set<TableId> META_TABLES =
+      Set.of(RootTable.ID, MetadataTable.ID, FateTable.ID);
+
+  public static FateInstanceType fromNamespaceOrTableName(String tableOrNamespaceName) {
+    return tableOrNamespaceName.startsWith(Namespace.ACCUMULO.name()) ? FateInstanceType.META
+        : FateInstanceType.USER;
+  }
+
+  public TFateInstanceType toThrift() {
+    switch (this) {
+      case USER:
+        return TFateInstanceType.USER;
+      case META:
+        return TFateInstanceType.META;
+      default:
+        throw new IllegalStateException("Unknown FateInstance type " + this);
+    }
+  }
+
+  public static FateInstanceType fromThrift(TFateInstanceType tfit) {
+    switch (tfit) {
+      case USER:
+        return FateInstanceType.USER;
+      case META:
+        return FateInstanceType.META;
+      default:
+        throw new IllegalStateException("Unknown type " + tfit);
+    }
+  }
+
+  public static FateInstanceType fromTableId(TableId tableId) {
+    return META_TABLES.contains(tableId) ? META : USER;
+  }
+}
diff --git a/core/src/main/java/org/apache/accumulo/core/fate/accumulo/AccumuloStore.java b/core/src/main/java/org/apache/accumulo/core/fate/accumulo/AccumuloStore.java
index 26a66632e8..89c1c316c9 100644
--- a/core/src/main/java/org/apache/accumulo/core/fate/accumulo/AccumuloStore.java
+++ b/core/src/main/java/org/apache/accumulo/core/fate/accumulo/AccumuloStore.java
@@ -40,6 +40,7 @@ import org.apache.accumulo.core.fate.StackOverflowException;
 import org.apache.accumulo.core.fate.accumulo.schema.FateSchema.RepoColumnFamily;
 import org.apache.accumulo.core.fate.accumulo.schema.FateSchema.TxColumnFamily;
 import org.apache.accumulo.core.fate.accumulo.schema.FateSchema.TxInfoColumnFamily;
+import org.apache.accumulo.core.metadata.FateTable;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.util.ColumnFQ;
 import org.apache.accumulo.core.util.FastFormat;
@@ -61,6 +62,10 @@ public class AccumuloStore<T> extends AbstractFateStore<T> {
     this.tableName = Objects.requireNonNull(tableName);
   }
 
+  public AccumuloStore(ClientContext context) {
+    this(context, FateTable.NAME);
+  }
+
   @Override
   public long create() {
     long tid = RANDOM.get().nextLong() & 0x7fffffffffffffffL;
diff --git a/core/src/main/java/org/apache/accumulo/core/metadata/FateTable.java b/core/src/main/java/org/apache/accumulo/core/metadata/FateTable.java
new file mode 100644
index 0000000000..0b6d99d453
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/metadata/FateTable.java
@@ -0,0 +1,27 @@
+/*
+ * 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.core.metadata;
+
+import org.apache.accumulo.core.clientImpl.Namespace;
+import org.apache.accumulo.core.data.TableId;
+
+public class FateTable {
+  public static final TableId ID = TableId.of("+fate");
+  public static final String NAME = Namespace.ACCUMULO.name() + ".fate";
+}
diff --git a/core/src/main/thrift-gen-java/org/apache/accumulo/core/manager/thrift/FateService.java b/core/src/main/thrift-gen-java/org/apache/accumulo/core/manager/thrift/FateService.java
index cc16dd4a31..b38fb423bc 100644
--- a/core/src/main/thrift-gen-java/org/apache/accumulo/core/manager/thrift/FateService.java
+++ b/core/src/main/thrift-gen-java/org/apache/accumulo/core/manager/thrift/FateService.java
@@ -29,29 +29,29 @@ public class FateService {
 
   public interface Iface {
 
-    public long beginFateOperation(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.thrift.TException;
+    public TFateId beginFateOperation(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, TFateInstanceType type) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.thrift.TException;
 
-    public void executeFateOperation(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, long opid, FateOperation op, java.util.List<java.nio.ByteBuffer> arguments, java.util.Map<java.lang.String,java.lang.String> options, boolean autoClean) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftTableOperationException, org.apache.accumulo.core.c [...]
+    public void executeFateOperation(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, TFateId opid, FateOperation op, java.util.List<java.nio.ByteBuffer> arguments, java.util.Map<java.lang.String,java.lang.String> options, boolean autoClean) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftTableOperationException, org.apache.accumulo.cor [...]
 
-    public java.lang.String waitForFateOperation(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, long opid) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftTableOperationException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.thrift.TException;
+    public java.lang.String waitForFateOperation(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, TFateId opid) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftTableOperationException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.thrift.TException;
 
-    public void finishFateOperation(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, long opid) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.thrift.TException;
+    public void finishFateOperation(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, TFateId opid) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.thrift.TException;
 
-    public boolean cancelFateOperation(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, long opid) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.thrift.TException;
+    public boolean cancelFateOperation(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, TFateId opid) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.thrift.TException;
 
   }
 
   public interface AsyncIface {
 
-    public void beginFateOperation(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<java.lang.Long> resultHandler) throws org.apache.thrift.TException;
+    public void beginFateOperation(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, TFateInstanceType type, org.apache.thrift.async.AsyncMethodCallback<TFateId> resultHandler) throws org.apache.thrift.TException;
 
-    public void executeFateOperation(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, long opid, FateOperation op, java.util.List<java.nio.ByteBuffer> arguments, java.util.Map<java.lang.String,java.lang.String> options, boolean autoClean, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
+    public void executeFateOperation(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, TFateId opid, FateOperation op, java.util.List<java.nio.ByteBuffer> arguments, java.util.Map<java.lang.String,java.lang.String> options, boolean autoClean, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
 
-    public void waitForFateOperation(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, long opid, org.apache.thrift.async.AsyncMethodCallback<java.lang.String> resultHandler) throws org.apache.thrift.TException;
+    public void waitForFateOperation(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, TFateId opid, org.apache.thrift.async.AsyncMethodCallback<java.lang.String> resultHandler) throws org.apache.thrift.TException;
 
-    public void finishFateOperation(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, long opid, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
+    public void finishFateOperation(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, TFateId opid, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
 
-    public void cancelFateOperation(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, long opid, org.apache.thrift.async.AsyncMethodCallback<java.lang.Boolean> resultHandler) throws org.apache.thrift.TException;
+    public void cancelFateOperation(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, TFateId opid, org.apache.thrift.async.AsyncMethodCallback<java.lang.Boolean> resultHandler) throws org.apache.thrift.TException;
 
   }
 
@@ -78,21 +78,22 @@ public class FateService {
     }
 
     @Override
-    public long beginFateOperation(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.thrift.TException
+    public TFateId beginFateOperation(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, TFateInstanceType type) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.thrift.TException
     {
-      send_beginFateOperation(tinfo, credentials);
+      send_beginFateOperation(tinfo, credentials, type);
       return recv_beginFateOperation();
     }
 
-    public void send_beginFateOperation(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws org.apache.thrift.TException
+    public void send_beginFateOperation(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, TFateInstanceType type) throws org.apache.thrift.TException
     {
       beginFateOperation_args args = new beginFateOperation_args();
       args.setTinfo(tinfo);
       args.setCredentials(credentials);
+      args.setType(type);
       sendBase("beginFateOperation", args);
     }
 
-    public long recv_beginFateOperation() throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.thrift.TException
+    public TFateId recv_beginFateOperation() throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.thrift.TException
     {
       beginFateOperation_result result = new beginFateOperation_result();
       receiveBase(result, "beginFateOperation");
@@ -109,13 +110,13 @@ public class FateService {
     }
 
     @Override
-    public void executeFateOperation(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, long opid, FateOperation op, java.util.List<java.nio.ByteBuffer> arguments, java.util.Map<java.lang.String,java.lang.String> options, boolean autoClean) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftTableOperationException, org.apache.accumulo.core.c [...]
+    public void executeFateOperation(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, TFateId opid, FateOperation op, java.util.List<java.nio.ByteBuffer> arguments, java.util.Map<java.lang.String,java.lang.String> options, boolean autoClean) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftTableOperationException, org.apache.accumulo.cor [...]
     {
       send_executeFateOperation(tinfo, credentials, opid, op, arguments, options, autoClean);
       recv_executeFateOperation();
     }
 
-    public void send_executeFateOperation(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, long opid, FateOperation op, java.util.List<java.nio.ByteBuffer> arguments, java.util.Map<java.lang.String,java.lang.String> options, boolean autoClean) throws org.apache.thrift.TException
+    public void send_executeFateOperation(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, TFateId opid, FateOperation op, java.util.List<java.nio.ByteBuffer> arguments, java.util.Map<java.lang.String,java.lang.String> options, boolean autoClean) throws org.apache.thrift.TException
     {
       executeFateOperation_args args = new executeFateOperation_args();
       args.setTinfo(tinfo);
@@ -145,13 +146,13 @@ public class FateService {
     }
 
     @Override
-    public java.lang.String waitForFateOperation(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, long opid) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftTableOperationException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.thrift.TException
+    public java.lang.String waitForFateOperation(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, TFateId opid) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftTableOperationException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.thrift.TException
     {
       send_waitForFateOperation(tinfo, credentials, opid);
       return recv_waitForFateOperation();
     }
 
-    public void send_waitForFateOperation(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, long opid) throws org.apache.thrift.TException
+    public void send_waitForFateOperation(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, TFateId opid) throws org.apache.thrift.TException
     {
       waitForFateOperation_args args = new waitForFateOperation_args();
       args.setTinfo(tinfo);
@@ -180,13 +181,13 @@ public class FateService {
     }
 
     @Override
-    public void finishFateOperation(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, long opid) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.thrift.TException
+    public void finishFateOperation(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, TFateId opid) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.thrift.TException
     {
       send_finishFateOperation(tinfo, credentials, opid);
       recv_finishFateOperation();
     }
 
-    public void send_finishFateOperation(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, long opid) throws org.apache.thrift.TException
+    public void send_finishFateOperation(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, TFateId opid) throws org.apache.thrift.TException
     {
       finishFateOperation_args args = new finishFateOperation_args();
       args.setTinfo(tinfo);
@@ -209,13 +210,13 @@ public class FateService {
     }
 
     @Override
-    public boolean cancelFateOperation(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, long opid) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.thrift.TException
+    public boolean cancelFateOperation(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, TFateId opid) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.thrift.TException
     {
       send_cancelFateOperation(tinfo, credentials, opid);
       return recv_cancelFateOperation();
     }
 
-    public void send_cancelFateOperation(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, long opid) throws org.apache.thrift.TException
+    public void send_cancelFateOperation(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, TFateId opid) throws org.apache.thrift.TException
     {
       cancelFateOperation_args args = new cancelFateOperation_args();
       args.setTinfo(tinfo);
@@ -260,20 +261,22 @@ public class FateService {
     }
 
     @Override
-    public void beginFateOperation(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<java.lang.Long> resultHandler) throws org.apache.thrift.TException {
+    public void beginFateOperation(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, TFateInstanceType type, org.apache.thrift.async.AsyncMethodCallback<TFateId> resultHandler) throws org.apache.thrift.TException {
       checkReady();
-      beginFateOperation_call method_call = new beginFateOperation_call(tinfo, credentials, resultHandler, this, ___protocolFactory, ___transport);
+      beginFateOperation_call method_call = new beginFateOperation_call(tinfo, credentials, type, resultHandler, this, ___protocolFactory, ___transport);
       this.___currentMethod = method_call;
       ___manager.call(method_call);
     }
 
-    public static class beginFateOperation_call extends org.apache.thrift.async.TAsyncMethodCall<java.lang.Long> {
+    public static class beginFateOperation_call extends org.apache.thrift.async.TAsyncMethodCall<TFateId> {
       private org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo;
       private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials;
-      public beginFateOperation_call(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<java.lang.Long> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+      private TFateInstanceType type;
+      public beginFateOperation_call(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, TFateInstanceType type, org.apache.thrift.async.AsyncMethodCallback<TFateId> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
         super(client, protocolFactory, transport, resultHandler, false);
         this.tinfo = tinfo;
         this.credentials = credentials;
+        this.type = type;
       }
 
       @Override
@@ -282,12 +285,13 @@ public class FateService {
         beginFateOperation_args args = new beginFateOperation_args();
         args.setTinfo(tinfo);
         args.setCredentials(credentials);
+        args.setType(type);
         args.write(prot);
         prot.writeMessageEnd();
       }
 
       @Override
-      public java.lang.Long getResult() throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.thrift.TException {
+      public TFateId getResult() throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.thrift.TException {
         if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
           throw new java.lang.IllegalStateException("Method call not finished!");
         }
@@ -298,7 +302,7 @@ public class FateService {
     }
 
     @Override
-    public void executeFateOperation(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, long opid, FateOperation op, java.util.List<java.nio.ByteBuffer> arguments, java.util.Map<java.lang.String,java.lang.String> options, boolean autoClean, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
+    public void executeFateOperation(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, TFateId opid, FateOperation op, java.util.List<java.nio.ByteBuffer> arguments, java.util.Map<java.lang.String,java.lang.String> options, boolean autoClean, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
       checkReady();
       executeFateOperation_call method_call = new executeFateOperation_call(tinfo, credentials, opid, op, arguments, options, autoClean, resultHandler, this, ___protocolFactory, ___transport);
       this.___currentMethod = method_call;
@@ -308,12 +312,12 @@ public class FateService {
     public static class executeFateOperation_call extends org.apache.thrift.async.TAsyncMethodCall<Void> {
       private org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo;
       private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials;
-      private long opid;
+      private TFateId opid;
       private FateOperation op;
       private java.util.List<java.nio.ByteBuffer> arguments;
       private java.util.Map<java.lang.String,java.lang.String> options;
       private boolean autoClean;
-      public executeFateOperation_call(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, long opid, FateOperation op, java.util.List<java.nio.ByteBuffer> arguments, java.util.Map<java.lang.String,java.lang.String> options, boolean autoClean, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, or [...]
+      public executeFateOperation_call(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, TFateId opid, FateOperation op, java.util.List<java.nio.ByteBuffer> arguments, java.util.Map<java.lang.String,java.lang.String> options, boolean autoClean, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, [...]
         super(client, protocolFactory, transport, resultHandler, false);
         this.tinfo = tinfo;
         this.credentials = credentials;
@@ -352,7 +356,7 @@ public class FateService {
     }
 
     @Override
-    public void waitForFateOperation(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, long opid, org.apache.thrift.async.AsyncMethodCallback<java.lang.String> resultHandler) throws org.apache.thrift.TException {
+    public void waitForFateOperation(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, TFateId opid, org.apache.thrift.async.AsyncMethodCallback<java.lang.String> resultHandler) throws org.apache.thrift.TException {
       checkReady();
       waitForFateOperation_call method_call = new waitForFateOperation_call(tinfo, credentials, opid, resultHandler, this, ___protocolFactory, ___transport);
       this.___currentMethod = method_call;
@@ -362,8 +366,8 @@ public class FateService {
     public static class waitForFateOperation_call extends org.apache.thrift.async.TAsyncMethodCall<java.lang.String> {
       private org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo;
       private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials;
-      private long opid;
-      public waitForFateOperation_call(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, long opid, org.apache.thrift.async.AsyncMethodCallback<java.lang.String> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+      private TFateId opid;
+      public waitForFateOperation_call(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, TFateId opid, org.apache.thrift.async.AsyncMethodCallback<java.lang.String> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
         super(client, protocolFactory, transport, resultHandler, false);
         this.tinfo = tinfo;
         this.credentials = credentials;
@@ -393,7 +397,7 @@ public class FateService {
     }
 
     @Override
-    public void finishFateOperation(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, long opid, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
+    public void finishFateOperation(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, TFateId opid, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
       checkReady();
       finishFateOperation_call method_call = new finishFateOperation_call(tinfo, credentials, opid, resultHandler, this, ___protocolFactory, ___transport);
       this.___currentMethod = method_call;
@@ -403,8 +407,8 @@ public class FateService {
     public static class finishFateOperation_call extends org.apache.thrift.async.TAsyncMethodCall<Void> {
       private org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo;
       private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials;
-      private long opid;
-      public finishFateOperation_call(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, long opid, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+      private TFateId opid;
+      public finishFateOperation_call(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, TFateId opid, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
         super(client, protocolFactory, transport, resultHandler, false);
         this.tinfo = tinfo;
         this.credentials = credentials;
@@ -435,7 +439,7 @@ public class FateService {
     }
 
     @Override
-    public void cancelFateOperation(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, long opid, org.apache.thrift.async.AsyncMethodCallback<java.lang.Boolean> resultHandler) throws org.apache.thrift.TException {
+    public void cancelFateOperation(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, TFateId opid, org.apache.thrift.async.AsyncMethodCallback<java.lang.Boolean> resultHandler) throws org.apache.thrift.TException {
       checkReady();
       cancelFateOperation_call method_call = new cancelFateOperation_call(tinfo, credentials, opid, resultHandler, this, ___protocolFactory, ___transport);
       this.___currentMethod = method_call;
@@ -445,8 +449,8 @@ public class FateService {
     public static class cancelFateOperation_call extends org.apache.thrift.async.TAsyncMethodCall<java.lang.Boolean> {
       private org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo;
       private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials;
-      private long opid;
-      public cancelFateOperation_call(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, long opid, org.apache.thrift.async.AsyncMethodCallback<java.lang.Boolean> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+      private TFateId opid;
+      public cancelFateOperation_call(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, TFateId opid, org.apache.thrift.async.AsyncMethodCallback<java.lang.Boolean> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
         super(client, protocolFactory, transport, resultHandler, false);
         this.tinfo = tinfo;
         this.credentials = credentials;
@@ -520,8 +524,7 @@ public class FateService {
       public beginFateOperation_result getResult(I iface, beginFateOperation_args args) throws org.apache.thrift.TException {
         beginFateOperation_result result = new beginFateOperation_result();
         try {
-          result.success = iface.beginFateOperation(args.tinfo, args.credentials);
-          result.setSuccessIsSet(true);
+          result.success = iface.beginFateOperation(args.tinfo, args.credentials, args.type);
         } catch (org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException sec) {
           result.sec = sec;
         } catch (org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException tnase) {
@@ -693,7 +696,7 @@ public class FateService {
       return processMap;
     }
 
-    public static class beginFateOperation<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, beginFateOperation_args, java.lang.Long> {
+    public static class beginFateOperation<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, beginFateOperation_args, TFateId> {
       public beginFateOperation() {
         super("beginFateOperation");
       }
@@ -704,14 +707,13 @@ public class FateService {
       }
 
       @Override
-      public org.apache.thrift.async.AsyncMethodCallback<java.lang.Long> getResultHandler(final org.apache.thrift.server.AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) {
+      public org.apache.thrift.async.AsyncMethodCallback<TFateId> getResultHandler(final org.apache.thrift.server.AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) {
         final org.apache.thrift.AsyncProcessFunction fcall = this;
-        return new org.apache.thrift.async.AsyncMethodCallback<java.lang.Long>() { 
+        return new org.apache.thrift.async.AsyncMethodCallback<TFateId>() { 
           @Override
-          public void onComplete(java.lang.Long o) {
+          public void onComplete(TFateId o) {
             beginFateOperation_result result = new beginFateOperation_result();
             result.success = o;
-            result.setSuccessIsSet(true);
             try {
               fcall.sendResponse(fb, result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
             } catch (org.apache.thrift.transport.TTransportException e) {
@@ -764,8 +766,8 @@ public class FateService {
       }
 
       @Override
-      public void start(I iface, beginFateOperation_args args, org.apache.thrift.async.AsyncMethodCallback<java.lang.Long> resultHandler) throws org.apache.thrift.TException {
-        iface.beginFateOperation(args.tinfo, args.credentials,resultHandler);
+      public void start(I iface, beginFateOperation_args args, org.apache.thrift.async.AsyncMethodCallback<TFateId> resultHandler) throws org.apache.thrift.TException {
+        iface.beginFateOperation(args.tinfo, args.credentials, args.type,resultHandler);
       }
     }
 
@@ -1084,17 +1086,28 @@ public class FateService {
 
     private static final org.apache.thrift.protocol.TField TINFO_FIELD_DESC = new org.apache.thrift.protocol.TField("tinfo", org.apache.thrift.protocol.TType.STRUCT, (short)1);
     private static final org.apache.thrift.protocol.TField CREDENTIALS_FIELD_DESC = new org.apache.thrift.protocol.TField("credentials", org.apache.thrift.protocol.TType.STRUCT, (short)2);
+    private static final org.apache.thrift.protocol.TField TYPE_FIELD_DESC = new org.apache.thrift.protocol.TField("type", org.apache.thrift.protocol.TType.I32, (short)3);
 
     private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new beginFateOperation_argsStandardSchemeFactory();
     private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new beginFateOperation_argsTupleSchemeFactory();
 
     public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo; // required
     public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials; // required
+    /**
+     * 
+     * @see TFateInstanceType
+     */
+    public @org.apache.thrift.annotation.Nullable TFateInstanceType type; // required
 
     /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
     public enum _Fields implements org.apache.thrift.TFieldIdEnum {
       TINFO((short)1, "tinfo"),
-      CREDENTIALS((short)2, "credentials");
+      CREDENTIALS((short)2, "credentials"),
+      /**
+       * 
+       * @see TFateInstanceType
+       */
+      TYPE((short)3, "type");
 
       private static final java.util.Map<java.lang.String, _Fields> byName = new java.util.HashMap<java.lang.String, _Fields>();
 
@@ -1114,6 +1127,8 @@ public class FateService {
             return TINFO;
           case 2: // CREDENTIALS
             return CREDENTIALS;
+          case 3: // TYPE
+            return TYPE;
           default:
             return null;
         }
@@ -1164,6 +1179,8 @@ public class FateService {
           new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.clientImpl.thrift.TInfo.class)));
       tmpMap.put(_Fields.CREDENTIALS, new org.apache.thrift.meta_data.FieldMetaData("credentials", org.apache.thrift.TFieldRequirementType.DEFAULT, 
           new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.securityImpl.thrift.TCredentials.class)));
+      tmpMap.put(_Fields.TYPE, new org.apache.thrift.meta_data.FieldMetaData("type", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.EnumMetaData(org.apache.thrift.protocol.TType.ENUM, TFateInstanceType.class)));
       metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
       org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(beginFateOperation_args.class, metaDataMap);
     }
@@ -1173,11 +1190,13 @@ public class FateService {
 
     public beginFateOperation_args(
       org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo,
-      org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials)
+      org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials,
+      TFateInstanceType type)
     {
       this();
       this.tinfo = tinfo;
       this.credentials = credentials;
+      this.type = type;
     }
 
     /**
@@ -1190,6 +1209,9 @@ public class FateService {
       if (other.isSetCredentials()) {
         this.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(other.credentials);
       }
+      if (other.isSetType()) {
+        this.type = other.type;
+      }
     }
 
     @Override
@@ -1201,6 +1223,7 @@ public class FateService {
     public void clear() {
       this.tinfo = null;
       this.credentials = null;
+      this.type = null;
     }
 
     @org.apache.thrift.annotation.Nullable
@@ -1253,6 +1276,39 @@ public class FateService {
       }
     }
 
+    /**
+     * 
+     * @see TFateInstanceType
+     */
+    @org.apache.thrift.annotation.Nullable
+    public TFateInstanceType getType() {
+      return this.type;
+    }
+
+    /**
+     * 
+     * @see TFateInstanceType
+     */
+    public beginFateOperation_args setType(@org.apache.thrift.annotation.Nullable TFateInstanceType type) {
+      this.type = type;
+      return this;
+    }
+
+    public void unsetType() {
+      this.type = null;
+    }
+
+    /** Returns true if field type is set (has been assigned a value) and false otherwise */
+    public boolean isSetType() {
+      return this.type != null;
+    }
+
+    public void setTypeIsSet(boolean value) {
+      if (!value) {
+        this.type = null;
+      }
+    }
+
     @Override
     public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable java.lang.Object value) {
       switch (field) {
@@ -1272,6 +1328,14 @@ public class FateService {
         }
         break;
 
+      case TYPE:
+        if (value == null) {
+          unsetType();
+        } else {
+          setType((TFateInstanceType)value);
+        }
+        break;
+
       }
     }
 
@@ -1285,6 +1349,9 @@ public class FateService {
       case CREDENTIALS:
         return getCredentials();
 
+      case TYPE:
+        return getType();
+
       }
       throw new java.lang.IllegalStateException();
     }
@@ -1301,6 +1368,8 @@ public class FateService {
         return isSetTinfo();
       case CREDENTIALS:
         return isSetCredentials();
+      case TYPE:
+        return isSetType();
       }
       throw new java.lang.IllegalStateException();
     }
@@ -1336,6 +1405,15 @@ public class FateService {
           return false;
       }
 
+      boolean this_present_type = true && this.isSetType();
+      boolean that_present_type = true && that.isSetType();
+      if (this_present_type || that_present_type) {
+        if (!(this_present_type && that_present_type))
+          return false;
+        if (!this.type.equals(that.type))
+          return false;
+      }
+
       return true;
     }
 
@@ -1351,6 +1429,10 @@ public class FateService {
       if (isSetCredentials())
         hashCode = hashCode * 8191 + credentials.hashCode();
 
+      hashCode = hashCode * 8191 + ((isSetType()) ? 131071 : 524287);
+      if (isSetType())
+        hashCode = hashCode * 8191 + type.getValue();
+
       return hashCode;
     }
 
@@ -1382,6 +1464,16 @@ public class FateService {
           return lastComparison;
         }
       }
+      lastComparison = java.lang.Boolean.compare(isSetType(), other.isSetType());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetType()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.type, other.type);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
       return 0;
     }
 
@@ -1421,6 +1513,14 @@ public class FateService {
         sb.append(this.credentials);
       }
       first = false;
+      if (!first) sb.append(", ");
+      sb.append("type:");
+      if (this.type == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.type);
+      }
+      first = false;
       sb.append(")");
       return sb.toString();
     }
@@ -1490,6 +1590,14 @@ public class FateService {
                 org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
               }
               break;
+            case 3: // TYPE
+              if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
+                struct.type = org.apache.accumulo.core.manager.thrift.TFateInstanceType.findByValue(iprot.readI32());
+                struct.setTypeIsSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
             default:
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
           }
@@ -1516,6 +1624,11 @@ public class FateService {
           struct.credentials.write(oprot);
           oprot.writeFieldEnd();
         }
+        if (struct.type != null) {
+          oprot.writeFieldBegin(TYPE_FIELD_DESC);
+          oprot.writeI32(struct.type.getValue());
+          oprot.writeFieldEnd();
+        }
         oprot.writeFieldStop();
         oprot.writeStructEnd();
       }
@@ -1541,19 +1654,25 @@ public class FateService {
         if (struct.isSetCredentials()) {
           optionals.set(1);
         }
-        oprot.writeBitSet(optionals, 2);
+        if (struct.isSetType()) {
+          optionals.set(2);
+        }
+        oprot.writeBitSet(optionals, 3);
         if (struct.isSetTinfo()) {
           struct.tinfo.write(oprot);
         }
         if (struct.isSetCredentials()) {
           struct.credentials.write(oprot);
         }
+        if (struct.isSetType()) {
+          oprot.writeI32(struct.type.getValue());
+        }
       }
 
       @Override
       public void read(org.apache.thrift.protocol.TProtocol prot, beginFateOperation_args struct) throws org.apache.thrift.TException {
         org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
-        java.util.BitSet incoming = iprot.readBitSet(2);
+        java.util.BitSet incoming = iprot.readBitSet(3);
         if (incoming.get(0)) {
           struct.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo();
           struct.tinfo.read(iprot);
@@ -1564,6 +1683,10 @@ public class FateService {
           struct.credentials.read(iprot);
           struct.setCredentialsIsSet(true);
         }
+        if (incoming.get(2)) {
+          struct.type = org.apache.accumulo.core.manager.thrift.TFateInstanceType.findByValue(iprot.readI32());
+          struct.setTypeIsSet(true);
+        }
       }
     }
 
@@ -1576,14 +1699,14 @@ public class FateService {
   public static class beginFateOperation_result implements org.apache.thrift.TBase<beginFateOperation_result, beginFateOperation_result._Fields>, java.io.Serializable, Cloneable, Comparable<beginFateOperation_result>   {
     private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("beginFateOperation_result");
 
-    private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.I64, (short)0);
+    private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.STRUCT, (short)0);
     private static final org.apache.thrift.protocol.TField SEC_FIELD_DESC = new org.apache.thrift.protocol.TField("sec", org.apache.thrift.protocol.TType.STRUCT, (short)1);
     private static final org.apache.thrift.protocol.TField TNASE_FIELD_DESC = new org.apache.thrift.protocol.TField("tnase", org.apache.thrift.protocol.TType.STRUCT, (short)2);
 
     private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new beginFateOperation_resultStandardSchemeFactory();
     private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new beginFateOperation_resultTupleSchemeFactory();
 
-    public long success; // required
+    public @org.apache.thrift.annotation.Nullable TFateId success; // required
     public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException sec; // required
     public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException tnase; // required
 
@@ -1656,13 +1779,11 @@ public class FateService {
     }
 
     // isset id assignments
-    private static final int __SUCCESS_ISSET_ID = 0;
-    private byte __isset_bitfield = 0;
     public static final java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
     static {
       java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
       tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, 
-          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64)));
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TFateId.class)));
       tmpMap.put(_Fields.SEC, new org.apache.thrift.meta_data.FieldMetaData("sec", org.apache.thrift.TFieldRequirementType.DEFAULT, 
           new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException.class)));
       tmpMap.put(_Fields.TNASE, new org.apache.thrift.meta_data.FieldMetaData("tnase", org.apache.thrift.TFieldRequirementType.DEFAULT, 
@@ -1675,13 +1796,12 @@ public class FateService {
     }
 
     public beginFateOperation_result(
-      long success,
+      TFateId success,
       org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException sec,
       org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException tnase)
     {
       this();
       this.success = success;
-      setSuccessIsSet(true);
       this.sec = sec;
       this.tnase = tnase;
     }
@@ -1690,8 +1810,9 @@ public class FateService {
      * Performs a deep copy on <i>other</i>.
      */
     public beginFateOperation_result(beginFateOperation_result other) {
-      __isset_bitfield = other.__isset_bitfield;
-      this.success = other.success;
+      if (other.isSetSuccess()) {
+        this.success = new TFateId(other.success);
+      }
       if (other.isSetSec()) {
         this.sec = new org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException(other.sec);
       }
@@ -1707,33 +1828,34 @@ public class FateService {
 
     @Override
     public void clear() {
-      setSuccessIsSet(false);
-      this.success = 0;
+      this.success = null;
       this.sec = null;
       this.tnase = null;
     }
 
-    public long getSuccess() {
+    @org.apache.thrift.annotation.Nullable
+    public TFateId getSuccess() {
       return this.success;
     }
 
-    public beginFateOperation_result setSuccess(long success) {
+    public beginFateOperation_result setSuccess(@org.apache.thrift.annotation.Nullable TFateId success) {
       this.success = success;
-      setSuccessIsSet(true);
       return this;
     }
 
     public void unsetSuccess() {
-      __isset_bitfield = org.apache.thrift.EncodingUtils.clearBit(__isset_bitfield, __SUCCESS_ISSET_ID);
+      this.success = null;
     }
 
     /** Returns true if field success is set (has been assigned a value) and false otherwise */
     public boolean isSetSuccess() {
-      return org.apache.thrift.EncodingUtils.testBit(__isset_bitfield, __SUCCESS_ISSET_ID);
+      return this.success != null;
     }
 
     public void setSuccessIsSet(boolean value) {
-      __isset_bitfield = org.apache.thrift.EncodingUtils.setBit(__isset_bitfield, __SUCCESS_ISSET_ID, value);
+      if (!value) {
+        this.success = null;
+      }
     }
 
     @org.apache.thrift.annotation.Nullable
@@ -1793,7 +1915,7 @@ public class FateService {
         if (value == null) {
           unsetSuccess();
         } else {
-          setSuccess((java.lang.Long)value);
+          setSuccess((TFateId)value);
         }
         break;
 
@@ -1864,12 +1986,12 @@ public class FateService {
       if (this == that)
         return true;
 
-      boolean this_present_success = true;
-      boolean that_present_success = true;
+      boolean this_present_success = true && this.isSetSuccess();
+      boolean that_present_success = true && that.isSetSuccess();
       if (this_present_success || that_present_success) {
         if (!(this_present_success && that_present_success))
           return false;
-        if (this.success != that.success)
+        if (!this.success.equals(that.success))
           return false;
       }
 
@@ -1898,7 +2020,9 @@ public class FateService {
     public int hashCode() {
       int hashCode = 1;
 
-      hashCode = hashCode * 8191 + org.apache.thrift.TBaseHelper.hashCode(success);
+      hashCode = hashCode * 8191 + ((isSetSuccess()) ? 131071 : 524287);
+      if (isSetSuccess())
+        hashCode = hashCode * 8191 + success.hashCode();
 
       hashCode = hashCode * 8191 + ((isSetSec()) ? 131071 : 524287);
       if (isSetSec())
@@ -1973,7 +2097,11 @@ public class FateService {
       boolean first = true;
 
       sb.append("success:");
-      sb.append(this.success);
+      if (this.success == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.success);
+      }
       first = false;
       if (!first) sb.append(", ");
       sb.append("sec:");
@@ -1998,6 +2126,9 @@ public class FateService {
     public void validate() throws org.apache.thrift.TException {
       // check for required fields
       // check for sub-struct validity
+      if (success != null) {
+        success.validate();
+      }
     }
 
     private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
@@ -2010,8 +2141,6 @@ public class FateService {
 
     private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, java.lang.ClassNotFoundException {
       try {
-        // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor.
-        __isset_bitfield = 0;
         read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
       } catch (org.apache.thrift.TException te) {
         throw new java.io.IOException(te);
@@ -2039,8 +2168,9 @@ public class FateService {
           }
           switch (schemeField.id) {
             case 0: // SUCCESS
-              if (schemeField.type == org.apache.thrift.protocol.TType.I64) {
-                struct.success = iprot.readI64();
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+                struct.success = new TFateId();
+                struct.success.read(iprot);
                 struct.setSuccessIsSet(true);
               } else { 
                 org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
@@ -2080,9 +2210,9 @@ public class FateService {
         struct.validate();
 
         oprot.writeStructBegin(STRUCT_DESC);
-        if (struct.isSetSuccess()) {
+        if (struct.success != null) {
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
-          oprot.writeI64(struct.success);
+          struct.success.write(oprot);
           oprot.writeFieldEnd();
         }
         if (struct.sec != null) {
@@ -2125,7 +2255,7 @@ public class FateService {
         }
         oprot.writeBitSet(optionals, 3);
         if (struct.isSetSuccess()) {
-          oprot.writeI64(struct.success);
+          struct.success.write(oprot);
         }
         if (struct.isSetSec()) {
           struct.sec.write(oprot);
@@ -2140,7 +2270,8 @@ public class FateService {
         org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
         java.util.BitSet incoming = iprot.readBitSet(3);
         if (incoming.get(0)) {
-          struct.success = iprot.readI64();
+          struct.success = new TFateId();
+          struct.success.read(iprot);
           struct.setSuccessIsSet(true);
         }
         if (incoming.get(1)) {
@@ -2167,7 +2298,7 @@ public class FateService {
 
     private static final org.apache.thrift.protocol.TField TINFO_FIELD_DESC = new org.apache.thrift.protocol.TField("tinfo", org.apache.thrift.protocol.TType.STRUCT, (short)1);
     private static final org.apache.thrift.protocol.TField CREDENTIALS_FIELD_DESC = new org.apache.thrift.protocol.TField("credentials", org.apache.thrift.protocol.TType.STRUCT, (short)2);
-    private static final org.apache.thrift.protocol.TField OPID_FIELD_DESC = new org.apache.thrift.protocol.TField("opid", org.apache.thrift.protocol.TType.I64, (short)3);
+    private static final org.apache.thrift.protocol.TField OPID_FIELD_DESC = new org.apache.thrift.protocol.TField("opid", org.apache.thrift.protocol.TType.STRUCT, (short)3);
     private static final org.apache.thrift.protocol.TField OP_FIELD_DESC = new org.apache.thrift.protocol.TField("op", org.apache.thrift.protocol.TType.I32, (short)4);
     private static final org.apache.thrift.protocol.TField ARGUMENTS_FIELD_DESC = new org.apache.thrift.protocol.TField("arguments", org.apache.thrift.protocol.TType.LIST, (short)5);
     private static final org.apache.thrift.protocol.TField OPTIONS_FIELD_DESC = new org.apache.thrift.protocol.TField("options", org.apache.thrift.protocol.TType.MAP, (short)6);
@@ -2178,7 +2309,7 @@ public class FateService {
 
     public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo; // required
     public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials; // required
-    public long opid; // required
+    public @org.apache.thrift.annotation.Nullable TFateId opid; // required
     /**
      * 
      * @see FateOperation
@@ -2273,8 +2404,7 @@ public class FateService {
     }
 
     // isset id assignments
-    private static final int __OPID_ISSET_ID = 0;
-    private static final int __AUTOCLEAN_ISSET_ID = 1;
+    private static final int __AUTOCLEAN_ISSET_ID = 0;
     private byte __isset_bitfield = 0;
     public static final java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
     static {
@@ -2284,7 +2414,7 @@ public class FateService {
       tmpMap.put(_Fields.CREDENTIALS, new org.apache.thrift.meta_data.FieldMetaData("credentials", org.apache.thrift.TFieldRequirementType.DEFAULT, 
           new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.securityImpl.thrift.TCredentials.class)));
       tmpMap.put(_Fields.OPID, new org.apache.thrift.meta_data.FieldMetaData("opid", org.apache.thrift.TFieldRequirementType.DEFAULT, 
-          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64)));
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TFateId.class)));
       tmpMap.put(_Fields.OP, new org.apache.thrift.meta_data.FieldMetaData("op", org.apache.thrift.TFieldRequirementType.DEFAULT, 
           new org.apache.thrift.meta_data.EnumMetaData(org.apache.thrift.protocol.TType.ENUM, FateOperation.class)));
       tmpMap.put(_Fields.ARGUMENTS, new org.apache.thrift.meta_data.FieldMetaData("arguments", org.apache.thrift.TFieldRequirementType.DEFAULT, 
@@ -2306,7 +2436,7 @@ public class FateService {
     public executeFateOperation_args(
       org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo,
       org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials,
-      long opid,
+      TFateId opid,
       FateOperation op,
       java.util.List<java.nio.ByteBuffer> arguments,
       java.util.Map<java.lang.String,java.lang.String> options,
@@ -2316,7 +2446,6 @@ public class FateService {
       this.tinfo = tinfo;
       this.credentials = credentials;
       this.opid = opid;
-      setOpidIsSet(true);
       this.op = op;
       this.arguments = arguments;
       this.options = options;
@@ -2335,7 +2464,9 @@ public class FateService {
       if (other.isSetCredentials()) {
         this.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(other.credentials);
       }
-      this.opid = other.opid;
+      if (other.isSetOpid()) {
+        this.opid = new TFateId(other.opid);
+      }
       if (other.isSetOp()) {
         this.op = other.op;
       }
@@ -2359,8 +2490,7 @@ public class FateService {
     public void clear() {
       this.tinfo = null;
       this.credentials = null;
-      setOpidIsSet(false);
-      this.opid = 0;
+      this.opid = null;
       this.op = null;
       this.arguments = null;
       this.options = null;
@@ -2418,27 +2548,29 @@ public class FateService {
       }
     }
 
-    public long getOpid() {
+    @org.apache.thrift.annotation.Nullable
+    public TFateId getOpid() {
       return this.opid;
     }
 
-    public executeFateOperation_args setOpid(long opid) {
+    public executeFateOperation_args setOpid(@org.apache.thrift.annotation.Nullable TFateId opid) {
       this.opid = opid;
-      setOpidIsSet(true);
       return this;
     }
 
     public void unsetOpid() {
-      __isset_bitfield = org.apache.thrift.EncodingUtils.clearBit(__isset_bitfield, __OPID_ISSET_ID);
+      this.opid = null;
     }
 
     /** Returns true if field opid is set (has been assigned a value) and false otherwise */
     public boolean isSetOpid() {
-      return org.apache.thrift.EncodingUtils.testBit(__isset_bitfield, __OPID_ISSET_ID);
+      return this.opid != null;
     }
 
     public void setOpidIsSet(boolean value) {
-      __isset_bitfield = org.apache.thrift.EncodingUtils.setBit(__isset_bitfield, __OPID_ISSET_ID, value);
+      if (!value) {
+        this.opid = null;
+      }
     }
 
     /**
@@ -2597,7 +2729,7 @@ public class FateService {
         if (value == null) {
           unsetOpid();
         } else {
-          setOpid((java.lang.Long)value);
+          setOpid((TFateId)value);
         }
         break;
 
@@ -2722,12 +2854,12 @@ public class FateService {
           return false;
       }
 
-      boolean this_present_opid = true;
-      boolean that_present_opid = true;
+      boolean this_present_opid = true && this.isSetOpid();
+      boolean that_present_opid = true && that.isSetOpid();
       if (this_present_opid || that_present_opid) {
         if (!(this_present_opid && that_present_opid))
           return false;
-        if (this.opid != that.opid)
+        if (!this.opid.equals(that.opid))
           return false;
       }
 
@@ -2782,7 +2914,9 @@ public class FateService {
       if (isSetCredentials())
         hashCode = hashCode * 8191 + credentials.hashCode();
 
-      hashCode = hashCode * 8191 + org.apache.thrift.TBaseHelper.hashCode(opid);
+      hashCode = hashCode * 8191 + ((isSetOpid()) ? 131071 : 524287);
+      if (isSetOpid())
+        hashCode = hashCode * 8191 + opid.hashCode();
 
       hashCode = hashCode * 8191 + ((isSetOp()) ? 131071 : 524287);
       if (isSetOp())
@@ -2920,7 +3054,11 @@ public class FateService {
       first = false;
       if (!first) sb.append(", ");
       sb.append("opid:");
-      sb.append(this.opid);
+      if (this.opid == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.opid);
+      }
       first = false;
       if (!first) sb.append(", ");
       sb.append("op:");
@@ -2963,6 +3101,9 @@ public class FateService {
       if (credentials != null) {
         credentials.validate();
       }
+      if (opid != null) {
+        opid.validate();
+      }
     }
 
     private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
@@ -3022,8 +3163,9 @@ public class FateService {
               }
               break;
             case 3: // OPID
-              if (schemeField.type == org.apache.thrift.protocol.TType.I64) {
-                struct.opid = iprot.readI64();
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+                struct.opid = new TFateId();
+                struct.opid.read(iprot);
                 struct.setOpidIsSet(true);
               } else { 
                 org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
@@ -3109,9 +3251,11 @@ public class FateService {
           struct.credentials.write(oprot);
           oprot.writeFieldEnd();
         }
-        oprot.writeFieldBegin(OPID_FIELD_DESC);
-        oprot.writeI64(struct.opid);
-        oprot.writeFieldEnd();
+        if (struct.opid != null) {
+          oprot.writeFieldBegin(OPID_FIELD_DESC);
+          struct.opid.write(oprot);
+          oprot.writeFieldEnd();
+        }
         if (struct.op != null) {
           oprot.writeFieldBegin(OP_FIELD_DESC);
           oprot.writeI32(struct.op.getValue());
@@ -3193,7 +3337,7 @@ public class FateService {
           struct.credentials.write(oprot);
         }
         if (struct.isSetOpid()) {
-          oprot.writeI64(struct.opid);
+          struct.opid.write(oprot);
         }
         if (struct.isSetOp()) {
           oprot.writeI32(struct.op.getValue());
@@ -3237,7 +3381,8 @@ public class FateService {
           struct.setCredentialsIsSet(true);
         }
         if (incoming.get(2)) {
-          struct.opid = iprot.readI64();
+          struct.opid = new TFateId();
+          struct.opid.read(iprot);
           struct.setOpidIsSet(true);
         }
         if (incoming.get(3)) {
@@ -3884,14 +4029,14 @@ public class FateService {
 
     private static final org.apache.thrift.protocol.TField TINFO_FIELD_DESC = new org.apache.thrift.protocol.TField("tinfo", org.apache.thrift.protocol.TType.STRUCT, (short)1);
     private static final org.apache.thrift.protocol.TField CREDENTIALS_FIELD_DESC = new org.apache.thrift.protocol.TField("credentials", org.apache.thrift.protocol.TType.STRUCT, (short)2);
-    private static final org.apache.thrift.protocol.TField OPID_FIELD_DESC = new org.apache.thrift.protocol.TField("opid", org.apache.thrift.protocol.TType.I64, (short)3);
+    private static final org.apache.thrift.protocol.TField OPID_FIELD_DESC = new org.apache.thrift.protocol.TField("opid", org.apache.thrift.protocol.TType.STRUCT, (short)3);
 
     private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new waitForFateOperation_argsStandardSchemeFactory();
     private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new waitForFateOperation_argsTupleSchemeFactory();
 
     public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo; // required
     public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials; // required
-    public long opid; // required
+    public @org.apache.thrift.annotation.Nullable TFateId opid; // required
 
     /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
     public enum _Fields implements org.apache.thrift.TFieldIdEnum {
@@ -3962,8 +4107,6 @@ public class FateService {
     }
 
     // isset id assignments
-    private static final int __OPID_ISSET_ID = 0;
-    private byte __isset_bitfield = 0;
     public static final java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
     static {
       java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
@@ -3972,7 +4115,7 @@ public class FateService {
       tmpMap.put(_Fields.CREDENTIALS, new org.apache.thrift.meta_data.FieldMetaData("credentials", org.apache.thrift.TFieldRequirementType.DEFAULT, 
           new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.securityImpl.thrift.TCredentials.class)));
       tmpMap.put(_Fields.OPID, new org.apache.thrift.meta_data.FieldMetaData("opid", org.apache.thrift.TFieldRequirementType.DEFAULT, 
-          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64)));
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TFateId.class)));
       metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
       org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(waitForFateOperation_args.class, metaDataMap);
     }
@@ -3983,27 +4126,27 @@ public class FateService {
     public waitForFateOperation_args(
       org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo,
       org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials,
-      long opid)
+      TFateId opid)
     {
       this();
       this.tinfo = tinfo;
       this.credentials = credentials;
       this.opid = opid;
-      setOpidIsSet(true);
     }
 
     /**
      * Performs a deep copy on <i>other</i>.
      */
     public waitForFateOperation_args(waitForFateOperation_args other) {
-      __isset_bitfield = other.__isset_bitfield;
       if (other.isSetTinfo()) {
         this.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo(other.tinfo);
       }
       if (other.isSetCredentials()) {
         this.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(other.credentials);
       }
-      this.opid = other.opid;
+      if (other.isSetOpid()) {
+        this.opid = new TFateId(other.opid);
+      }
     }
 
     @Override
@@ -4015,8 +4158,7 @@ public class FateService {
     public void clear() {
       this.tinfo = null;
       this.credentials = null;
-      setOpidIsSet(false);
-      this.opid = 0;
+      this.opid = null;
     }
 
     @org.apache.thrift.annotation.Nullable
@@ -4069,27 +4211,29 @@ public class FateService {
       }
     }
 
-    public long getOpid() {
+    @org.apache.thrift.annotation.Nullable
+    public TFateId getOpid() {
       return this.opid;
     }
 
-    public waitForFateOperation_args setOpid(long opid) {
+    public waitForFateOperation_args setOpid(@org.apache.thrift.annotation.Nullable TFateId opid) {
       this.opid = opid;
-      setOpidIsSet(true);
       return this;
     }
 
     public void unsetOpid() {
-      __isset_bitfield = org.apache.thrift.EncodingUtils.clearBit(__isset_bitfield, __OPID_ISSET_ID);
+      this.opid = null;
     }
 
     /** Returns true if field opid is set (has been assigned a value) and false otherwise */
     public boolean isSetOpid() {
-      return org.apache.thrift.EncodingUtils.testBit(__isset_bitfield, __OPID_ISSET_ID);
+      return this.opid != null;
     }
 
     public void setOpidIsSet(boolean value) {
-      __isset_bitfield = org.apache.thrift.EncodingUtils.setBit(__isset_bitfield, __OPID_ISSET_ID, value);
+      if (!value) {
+        this.opid = null;
+      }
     }
 
     @Override
@@ -4115,7 +4259,7 @@ public class FateService {
         if (value == null) {
           unsetOpid();
         } else {
-          setOpid((java.lang.Long)value);
+          setOpid((TFateId)value);
         }
         break;
 
@@ -4188,12 +4332,12 @@ public class FateService {
           return false;
       }
 
-      boolean this_present_opid = true;
-      boolean that_present_opid = true;
+      boolean this_present_opid = true && this.isSetOpid();
+      boolean that_present_opid = true && that.isSetOpid();
       if (this_present_opid || that_present_opid) {
         if (!(this_present_opid && that_present_opid))
           return false;
-        if (this.opid != that.opid)
+        if (!this.opid.equals(that.opid))
           return false;
       }
 
@@ -4212,7 +4356,9 @@ public class FateService {
       if (isSetCredentials())
         hashCode = hashCode * 8191 + credentials.hashCode();
 
-      hashCode = hashCode * 8191 + org.apache.thrift.TBaseHelper.hashCode(opid);
+      hashCode = hashCode * 8191 + ((isSetOpid()) ? 131071 : 524287);
+      if (isSetOpid())
+        hashCode = hashCode * 8191 + opid.hashCode();
 
       return hashCode;
     }
@@ -4296,7 +4442,11 @@ public class FateService {
       first = false;
       if (!first) sb.append(", ");
       sb.append("opid:");
-      sb.append(this.opid);
+      if (this.opid == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.opid);
+      }
       first = false;
       sb.append(")");
       return sb.toString();
@@ -4311,6 +4461,9 @@ public class FateService {
       if (credentials != null) {
         credentials.validate();
       }
+      if (opid != null) {
+        opid.validate();
+      }
     }
 
     private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
@@ -4323,8 +4476,6 @@ public class FateService {
 
     private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, java.lang.ClassNotFoundException {
       try {
-        // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor.
-        __isset_bitfield = 0;
         read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
       } catch (org.apache.thrift.TException te) {
         throw new java.io.IOException(te);
@@ -4370,8 +4521,9 @@ public class FateService {
               }
               break;
             case 3: // OPID
-              if (schemeField.type == org.apache.thrift.protocol.TType.I64) {
-                struct.opid = iprot.readI64();
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+                struct.opid = new TFateId();
+                struct.opid.read(iprot);
                 struct.setOpidIsSet(true);
               } else { 
                 org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
@@ -4403,9 +4555,11 @@ public class FateService {
           struct.credentials.write(oprot);
           oprot.writeFieldEnd();
         }
-        oprot.writeFieldBegin(OPID_FIELD_DESC);
-        oprot.writeI64(struct.opid);
-        oprot.writeFieldEnd();
+        if (struct.opid != null) {
+          oprot.writeFieldBegin(OPID_FIELD_DESC);
+          struct.opid.write(oprot);
+          oprot.writeFieldEnd();
+        }
         oprot.writeFieldStop();
         oprot.writeStructEnd();
       }
@@ -4442,7 +4596,7 @@ public class FateService {
           struct.credentials.write(oprot);
         }
         if (struct.isSetOpid()) {
-          oprot.writeI64(struct.opid);
+          struct.opid.write(oprot);
         }
       }
 
@@ -4461,7 +4615,8 @@ public class FateService {
           struct.setCredentialsIsSet(true);
         }
         if (incoming.get(2)) {
-          struct.opid = iprot.readI64();
+          struct.opid = new TFateId();
+          struct.opid.read(iprot);
           struct.setOpidIsSet(true);
         }
       }
@@ -5177,14 +5332,14 @@ public class FateService {
 
     private static final org.apache.thrift.protocol.TField TINFO_FIELD_DESC = new org.apache.thrift.protocol.TField("tinfo", org.apache.thrift.protocol.TType.STRUCT, (short)1);
     private static final org.apache.thrift.protocol.TField CREDENTIALS_FIELD_DESC = new org.apache.thrift.protocol.TField("credentials", org.apache.thrift.protocol.TType.STRUCT, (short)2);
-    private static final org.apache.thrift.protocol.TField OPID_FIELD_DESC = new org.apache.thrift.protocol.TField("opid", org.apache.thrift.protocol.TType.I64, (short)3);
+    private static final org.apache.thrift.protocol.TField OPID_FIELD_DESC = new org.apache.thrift.protocol.TField("opid", org.apache.thrift.protocol.TType.STRUCT, (short)3);
 
     private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new finishFateOperation_argsStandardSchemeFactory();
     private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new finishFateOperation_argsTupleSchemeFactory();
 
     public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo; // required
     public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials; // required
-    public long opid; // required
+    public @org.apache.thrift.annotation.Nullable TFateId opid; // required
 
     /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
     public enum _Fields implements org.apache.thrift.TFieldIdEnum {
@@ -5255,8 +5410,6 @@ public class FateService {
     }
 
     // isset id assignments
-    private static final int __OPID_ISSET_ID = 0;
-    private byte __isset_bitfield = 0;
     public static final java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
     static {
       java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
@@ -5265,7 +5418,7 @@ public class FateService {
       tmpMap.put(_Fields.CREDENTIALS, new org.apache.thrift.meta_data.FieldMetaData("credentials", org.apache.thrift.TFieldRequirementType.DEFAULT, 
           new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.securityImpl.thrift.TCredentials.class)));
       tmpMap.put(_Fields.OPID, new org.apache.thrift.meta_data.FieldMetaData("opid", org.apache.thrift.TFieldRequirementType.DEFAULT, 
-          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64)));
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TFateId.class)));
       metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
       org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(finishFateOperation_args.class, metaDataMap);
     }
@@ -5276,27 +5429,27 @@ public class FateService {
     public finishFateOperation_args(
       org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo,
       org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials,
-      long opid)
+      TFateId opid)
     {
       this();
       this.tinfo = tinfo;
       this.credentials = credentials;
       this.opid = opid;
-      setOpidIsSet(true);
     }
 
     /**
      * Performs a deep copy on <i>other</i>.
      */
     public finishFateOperation_args(finishFateOperation_args other) {
-      __isset_bitfield = other.__isset_bitfield;
       if (other.isSetTinfo()) {
         this.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo(other.tinfo);
       }
       if (other.isSetCredentials()) {
         this.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(other.credentials);
       }
-      this.opid = other.opid;
+      if (other.isSetOpid()) {
+        this.opid = new TFateId(other.opid);
+      }
     }
 
     @Override
@@ -5308,8 +5461,7 @@ public class FateService {
     public void clear() {
       this.tinfo = null;
       this.credentials = null;
-      setOpidIsSet(false);
-      this.opid = 0;
+      this.opid = null;
     }
 
     @org.apache.thrift.annotation.Nullable
@@ -5362,27 +5514,29 @@ public class FateService {
       }
     }
 
-    public long getOpid() {
+    @org.apache.thrift.annotation.Nullable
+    public TFateId getOpid() {
       return this.opid;
     }
 
-    public finishFateOperation_args setOpid(long opid) {
+    public finishFateOperation_args setOpid(@org.apache.thrift.annotation.Nullable TFateId opid) {
       this.opid = opid;
-      setOpidIsSet(true);
       return this;
     }
 
     public void unsetOpid() {
-      __isset_bitfield = org.apache.thrift.EncodingUtils.clearBit(__isset_bitfield, __OPID_ISSET_ID);
+      this.opid = null;
     }
 
     /** Returns true if field opid is set (has been assigned a value) and false otherwise */
     public boolean isSetOpid() {
-      return org.apache.thrift.EncodingUtils.testBit(__isset_bitfield, __OPID_ISSET_ID);
+      return this.opid != null;
     }
 
     public void setOpidIsSet(boolean value) {
-      __isset_bitfield = org.apache.thrift.EncodingUtils.setBit(__isset_bitfield, __OPID_ISSET_ID, value);
+      if (!value) {
+        this.opid = null;
+      }
     }
 
     @Override
@@ -5408,7 +5562,7 @@ public class FateService {
         if (value == null) {
           unsetOpid();
         } else {
-          setOpid((java.lang.Long)value);
+          setOpid((TFateId)value);
         }
         break;
 
@@ -5481,12 +5635,12 @@ public class FateService {
           return false;
       }
 
-      boolean this_present_opid = true;
-      boolean that_present_opid = true;
+      boolean this_present_opid = true && this.isSetOpid();
+      boolean that_present_opid = true && that.isSetOpid();
       if (this_present_opid || that_present_opid) {
         if (!(this_present_opid && that_present_opid))
           return false;
-        if (this.opid != that.opid)
+        if (!this.opid.equals(that.opid))
           return false;
       }
 
@@ -5505,7 +5659,9 @@ public class FateService {
       if (isSetCredentials())
         hashCode = hashCode * 8191 + credentials.hashCode();
 
-      hashCode = hashCode * 8191 + org.apache.thrift.TBaseHelper.hashCode(opid);
+      hashCode = hashCode * 8191 + ((isSetOpid()) ? 131071 : 524287);
+      if (isSetOpid())
+        hashCode = hashCode * 8191 + opid.hashCode();
 
       return hashCode;
     }
@@ -5589,7 +5745,11 @@ public class FateService {
       first = false;
       if (!first) sb.append(", ");
       sb.append("opid:");
-      sb.append(this.opid);
+      if (this.opid == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.opid);
+      }
       first = false;
       sb.append(")");
       return sb.toString();
@@ -5604,6 +5764,9 @@ public class FateService {
       if (credentials != null) {
         credentials.validate();
       }
+      if (opid != null) {
+        opid.validate();
+      }
     }
 
     private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
@@ -5616,8 +5779,6 @@ public class FateService {
 
     private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, java.lang.ClassNotFoundException {
       try {
-        // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor.
-        __isset_bitfield = 0;
         read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
       } catch (org.apache.thrift.TException te) {
         throw new java.io.IOException(te);
@@ -5663,8 +5824,9 @@ public class FateService {
               }
               break;
             case 3: // OPID
-              if (schemeField.type == org.apache.thrift.protocol.TType.I64) {
-                struct.opid = iprot.readI64();
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+                struct.opid = new TFateId();
+                struct.opid.read(iprot);
                 struct.setOpidIsSet(true);
               } else { 
                 org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
@@ -5696,9 +5858,11 @@ public class FateService {
           struct.credentials.write(oprot);
           oprot.writeFieldEnd();
         }
-        oprot.writeFieldBegin(OPID_FIELD_DESC);
-        oprot.writeI64(struct.opid);
-        oprot.writeFieldEnd();
+        if (struct.opid != null) {
+          oprot.writeFieldBegin(OPID_FIELD_DESC);
+          struct.opid.write(oprot);
+          oprot.writeFieldEnd();
+        }
         oprot.writeFieldStop();
         oprot.writeStructEnd();
       }
@@ -5735,7 +5899,7 @@ public class FateService {
           struct.credentials.write(oprot);
         }
         if (struct.isSetOpid()) {
-          oprot.writeI64(struct.opid);
+          struct.opid.write(oprot);
         }
       }
 
@@ -5754,7 +5918,8 @@ public class FateService {
           struct.setCredentialsIsSet(true);
         }
         if (incoming.get(2)) {
-          struct.opid = iprot.readI64();
+          struct.opid = new TFateId();
+          struct.opid.read(iprot);
           struct.setOpidIsSet(true);
         }
       }
@@ -6258,14 +6423,14 @@ public class FateService {
 
     private static final org.apache.thrift.protocol.TField TINFO_FIELD_DESC = new org.apache.thrift.protocol.TField("tinfo", org.apache.thrift.protocol.TType.STRUCT, (short)1);
     private static final org.apache.thrift.protocol.TField CREDENTIALS_FIELD_DESC = new org.apache.thrift.protocol.TField("credentials", org.apache.thrift.protocol.TType.STRUCT, (short)2);
-    private static final org.apache.thrift.protocol.TField OPID_FIELD_DESC = new org.apache.thrift.protocol.TField("opid", org.apache.thrift.protocol.TType.I64, (short)3);
+    private static final org.apache.thrift.protocol.TField OPID_FIELD_DESC = new org.apache.thrift.protocol.TField("opid", org.apache.thrift.protocol.TType.STRUCT, (short)3);
 
     private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new cancelFateOperation_argsStandardSchemeFactory();
     private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new cancelFateOperation_argsTupleSchemeFactory();
 
     public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo; // required
     public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials; // required
-    public long opid; // required
+    public @org.apache.thrift.annotation.Nullable TFateId opid; // required
 
     /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
     public enum _Fields implements org.apache.thrift.TFieldIdEnum {
@@ -6336,8 +6501,6 @@ public class FateService {
     }
 
     // isset id assignments
-    private static final int __OPID_ISSET_ID = 0;
-    private byte __isset_bitfield = 0;
     public static final java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
     static {
       java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
@@ -6346,7 +6509,7 @@ public class FateService {
       tmpMap.put(_Fields.CREDENTIALS, new org.apache.thrift.meta_data.FieldMetaData("credentials", org.apache.thrift.TFieldRequirementType.DEFAULT, 
           new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.securityImpl.thrift.TCredentials.class)));
       tmpMap.put(_Fields.OPID, new org.apache.thrift.meta_data.FieldMetaData("opid", org.apache.thrift.TFieldRequirementType.DEFAULT, 
-          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64)));
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TFateId.class)));
       metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
       org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(cancelFateOperation_args.class, metaDataMap);
     }
@@ -6357,27 +6520,27 @@ public class FateService {
     public cancelFateOperation_args(
       org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo,
       org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials,
-      long opid)
+      TFateId opid)
     {
       this();
       this.tinfo = tinfo;
       this.credentials = credentials;
       this.opid = opid;
-      setOpidIsSet(true);
     }
 
     /**
      * Performs a deep copy on <i>other</i>.
      */
     public cancelFateOperation_args(cancelFateOperation_args other) {
-      __isset_bitfield = other.__isset_bitfield;
       if (other.isSetTinfo()) {
         this.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo(other.tinfo);
       }
       if (other.isSetCredentials()) {
         this.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(other.credentials);
       }
-      this.opid = other.opid;
+      if (other.isSetOpid()) {
+        this.opid = new TFateId(other.opid);
+      }
     }
 
     @Override
@@ -6389,8 +6552,7 @@ public class FateService {
     public void clear() {
       this.tinfo = null;
       this.credentials = null;
-      setOpidIsSet(false);
-      this.opid = 0;
+      this.opid = null;
     }
 
     @org.apache.thrift.annotation.Nullable
@@ -6443,27 +6605,29 @@ public class FateService {
       }
     }
 
-    public long getOpid() {
+    @org.apache.thrift.annotation.Nullable
+    public TFateId getOpid() {
       return this.opid;
     }
 
-    public cancelFateOperation_args setOpid(long opid) {
+    public cancelFateOperation_args setOpid(@org.apache.thrift.annotation.Nullable TFateId opid) {
       this.opid = opid;
-      setOpidIsSet(true);
       return this;
     }
 
     public void unsetOpid() {
-      __isset_bitfield = org.apache.thrift.EncodingUtils.clearBit(__isset_bitfield, __OPID_ISSET_ID);
+      this.opid = null;
     }
 
     /** Returns true if field opid is set (has been assigned a value) and false otherwise */
     public boolean isSetOpid() {
-      return org.apache.thrift.EncodingUtils.testBit(__isset_bitfield, __OPID_ISSET_ID);
+      return this.opid != null;
     }
 
     public void setOpidIsSet(boolean value) {
-      __isset_bitfield = org.apache.thrift.EncodingUtils.setBit(__isset_bitfield, __OPID_ISSET_ID, value);
+      if (!value) {
+        this.opid = null;
+      }
     }
 
     @Override
@@ -6489,7 +6653,7 @@ public class FateService {
         if (value == null) {
           unsetOpid();
         } else {
-          setOpid((java.lang.Long)value);
+          setOpid((TFateId)value);
         }
         break;
 
@@ -6562,12 +6726,12 @@ public class FateService {
           return false;
       }
 
-      boolean this_present_opid = true;
-      boolean that_present_opid = true;
+      boolean this_present_opid = true && this.isSetOpid();
+      boolean that_present_opid = true && that.isSetOpid();
       if (this_present_opid || that_present_opid) {
         if (!(this_present_opid && that_present_opid))
           return false;
-        if (this.opid != that.opid)
+        if (!this.opid.equals(that.opid))
           return false;
       }
 
@@ -6586,7 +6750,9 @@ public class FateService {
       if (isSetCredentials())
         hashCode = hashCode * 8191 + credentials.hashCode();
 
-      hashCode = hashCode * 8191 + org.apache.thrift.TBaseHelper.hashCode(opid);
+      hashCode = hashCode * 8191 + ((isSetOpid()) ? 131071 : 524287);
+      if (isSetOpid())
+        hashCode = hashCode * 8191 + opid.hashCode();
 
       return hashCode;
     }
@@ -6670,7 +6836,11 @@ public class FateService {
       first = false;
       if (!first) sb.append(", ");
       sb.append("opid:");
-      sb.append(this.opid);
+      if (this.opid == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.opid);
+      }
       first = false;
       sb.append(")");
       return sb.toString();
@@ -6685,6 +6855,9 @@ public class FateService {
       if (credentials != null) {
         credentials.validate();
       }
+      if (opid != null) {
+        opid.validate();
+      }
     }
 
     private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
@@ -6697,8 +6870,6 @@ public class FateService {
 
     private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, java.lang.ClassNotFoundException {
       try {
-        // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor.
-        __isset_bitfield = 0;
         read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
       } catch (org.apache.thrift.TException te) {
         throw new java.io.IOException(te);
@@ -6744,8 +6915,9 @@ public class FateService {
               }
               break;
             case 3: // OPID
-              if (schemeField.type == org.apache.thrift.protocol.TType.I64) {
-                struct.opid = iprot.readI64();
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+                struct.opid = new TFateId();
+                struct.opid.read(iprot);
                 struct.setOpidIsSet(true);
               } else { 
                 org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
@@ -6777,9 +6949,11 @@ public class FateService {
           struct.credentials.write(oprot);
           oprot.writeFieldEnd();
         }
-        oprot.writeFieldBegin(OPID_FIELD_DESC);
-        oprot.writeI64(struct.opid);
-        oprot.writeFieldEnd();
+        if (struct.opid != null) {
+          oprot.writeFieldBegin(OPID_FIELD_DESC);
+          struct.opid.write(oprot);
+          oprot.writeFieldEnd();
+        }
         oprot.writeFieldStop();
         oprot.writeStructEnd();
       }
@@ -6816,7 +6990,7 @@ public class FateService {
           struct.credentials.write(oprot);
         }
         if (struct.isSetOpid()) {
-          oprot.writeI64(struct.opid);
+          struct.opid.write(oprot);
         }
       }
 
@@ -6835,7 +7009,8 @@ public class FateService {
           struct.setCredentialsIsSet(true);
         }
         if (incoming.get(2)) {
-          struct.opid = iprot.readI64();
+          struct.opid = new TFateId();
+          struct.opid.read(iprot);
           struct.setOpidIsSet(true);
         }
       }
diff --git a/core/src/main/thrift-gen-java/org/apache/accumulo/core/manager/thrift/TFateId.java b/core/src/main/thrift-gen-java/org/apache/accumulo/core/manager/thrift/TFateId.java
new file mode 100644
index 0000000000..d781407397
--- /dev/null
+++ b/core/src/main/thrift-gen-java/org/apache/accumulo/core/manager/thrift/TFateId.java
@@ -0,0 +1,522 @@
+/*
+ * 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.
+ */
+/**
+ * Autogenerated by Thrift Compiler (0.17.0)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package org.apache.accumulo.core.manager.thrift;
+
+@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"})
+public class TFateId implements org.apache.thrift.TBase<TFateId, TFateId._Fields>, java.io.Serializable, Cloneable, Comparable<TFateId> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TFateId");
+
+  private static final org.apache.thrift.protocol.TField TYPE_FIELD_DESC = new org.apache.thrift.protocol.TField("type", org.apache.thrift.protocol.TType.I32, (short)1);
+  private static final org.apache.thrift.protocol.TField TID_FIELD_DESC = new org.apache.thrift.protocol.TField("tid", org.apache.thrift.protocol.TType.I64, (short)2);
+
+  private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new TFateIdStandardSchemeFactory();
+  private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new TFateIdTupleSchemeFactory();
+
+  /**
+   * 
+   * @see TFateInstanceType
+   */
+  public @org.apache.thrift.annotation.Nullable TFateInstanceType type; // required
+  public long tid; // required
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+    /**
+     * 
+     * @see TFateInstanceType
+     */
+    TYPE((short)1, "type"),
+    TID((short)2, "tid");
+
+    private static final java.util.Map<java.lang.String, _Fields> byName = new java.util.HashMap<java.lang.String, _Fields>();
+
+    static {
+      for (_Fields field : java.util.EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    @org.apache.thrift.annotation.Nullable
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // TYPE
+          return TYPE;
+        case 2: // TID
+          return TID;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new java.lang.IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    @org.apache.thrift.annotation.Nullable
+    public static _Fields findByName(java.lang.String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final java.lang.String _fieldName;
+
+    _Fields(short thriftId, java.lang.String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    @Override
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    @Override
+    public java.lang.String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  // isset id assignments
+  private static final int __TID_ISSET_ID = 0;
+  private byte __isset_bitfield = 0;
+  public static final java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+  static {
+    java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.TYPE, new org.apache.thrift.meta_data.FieldMetaData("type", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+        new org.apache.thrift.meta_data.EnumMetaData(org.apache.thrift.protocol.TType.ENUM, TFateInstanceType.class)));
+    tmpMap.put(_Fields.TID, new org.apache.thrift.meta_data.FieldMetaData("tid", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64)));
+    metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TFateId.class, metaDataMap);
+  }
+
+  public TFateId() {
+  }
+
+  public TFateId(
+    TFateInstanceType type,
+    long tid)
+  {
+    this();
+    this.type = type;
+    this.tid = tid;
+    setTidIsSet(true);
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public TFateId(TFateId other) {
+    __isset_bitfield = other.__isset_bitfield;
+    if (other.isSetType()) {
+      this.type = other.type;
+    }
+    this.tid = other.tid;
+  }
+
+  @Override
+  public TFateId deepCopy() {
+    return new TFateId(this);
+  }
+
+  @Override
+  public void clear() {
+    this.type = null;
+    setTidIsSet(false);
+    this.tid = 0;
+  }
+
+  /**
+   * 
+   * @see TFateInstanceType
+   */
+  @org.apache.thrift.annotation.Nullable
+  public TFateInstanceType getType() {
+    return this.type;
+  }
+
+  /**
+   * 
+   * @see TFateInstanceType
+   */
+  public TFateId setType(@org.apache.thrift.annotation.Nullable TFateInstanceType type) {
+    this.type = type;
+    return this;
+  }
+
+  public void unsetType() {
+    this.type = null;
+  }
+
+  /** Returns true if field type is set (has been assigned a value) and false otherwise */
+  public boolean isSetType() {
+    return this.type != null;
+  }
+
+  public void setTypeIsSet(boolean value) {
+    if (!value) {
+      this.type = null;
+    }
+  }
+
+  public long getTid() {
+    return this.tid;
+  }
+
+  public TFateId setTid(long tid) {
+    this.tid = tid;
+    setTidIsSet(true);
+    return this;
+  }
+
+  public void unsetTid() {
+    __isset_bitfield = org.apache.thrift.EncodingUtils.clearBit(__isset_bitfield, __TID_ISSET_ID);
+  }
+
+  /** Returns true if field tid is set (has been assigned a value) and false otherwise */
+  public boolean isSetTid() {
+    return org.apache.thrift.EncodingUtils.testBit(__isset_bitfield, __TID_ISSET_ID);
+  }
+
+  public void setTidIsSet(boolean value) {
+    __isset_bitfield = org.apache.thrift.EncodingUtils.setBit(__isset_bitfield, __TID_ISSET_ID, value);
+  }
+
+  @Override
+  public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable java.lang.Object value) {
+    switch (field) {
+    case TYPE:
+      if (value == null) {
+        unsetType();
+      } else {
+        setType((TFateInstanceType)value);
+      }
+      break;
+
+    case TID:
+      if (value == null) {
+        unsetTid();
+      } else {
+        setTid((java.lang.Long)value);
+      }
+      break;
+
+    }
+  }
+
+  @org.apache.thrift.annotation.Nullable
+  @Override
+  public java.lang.Object getFieldValue(_Fields field) {
+    switch (field) {
+    case TYPE:
+      return getType();
+
+    case TID:
+      return getTid();
+
+    }
+    throw new java.lang.IllegalStateException();
+  }
+
+  /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+  @Override
+  public boolean isSet(_Fields field) {
+    if (field == null) {
+      throw new java.lang.IllegalArgumentException();
+    }
+
+    switch (field) {
+    case TYPE:
+      return isSetType();
+    case TID:
+      return isSetTid();
+    }
+    throw new java.lang.IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(java.lang.Object that) {
+    if (that instanceof TFateId)
+      return this.equals((TFateId)that);
+    return false;
+  }
+
+  public boolean equals(TFateId that) {
+    if (that == null)
+      return false;
+    if (this == that)
+      return true;
+
+    boolean this_present_type = true && this.isSetType();
+    boolean that_present_type = true && that.isSetType();
+    if (this_present_type || that_present_type) {
+      if (!(this_present_type && that_present_type))
+        return false;
+      if (!this.type.equals(that.type))
+        return false;
+    }
+
+    boolean this_present_tid = true;
+    boolean that_present_tid = true;
+    if (this_present_tid || that_present_tid) {
+      if (!(this_present_tid && that_present_tid))
+        return false;
+      if (this.tid != that.tid)
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    int hashCode = 1;
+
+    hashCode = hashCode * 8191 + ((isSetType()) ? 131071 : 524287);
+    if (isSetType())
+      hashCode = hashCode * 8191 + type.getValue();
+
+    hashCode = hashCode * 8191 + org.apache.thrift.TBaseHelper.hashCode(tid);
+
+    return hashCode;
+  }
+
+  @Override
+  public int compareTo(TFateId other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+
+    lastComparison = java.lang.Boolean.compare(isSetType(), other.isSetType());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetType()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.type, other.type);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = java.lang.Boolean.compare(isSetTid(), other.isSetTid());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetTid()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tid, other.tid);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    return 0;
+  }
+
+  @org.apache.thrift.annotation.Nullable
+  @Override
+  public _Fields fieldForId(int fieldId) {
+    return _Fields.findByThriftId(fieldId);
+  }
+
+  @Override
+  public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+    scheme(iprot).read(iprot, this);
+  }
+
+  @Override
+  public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    scheme(oprot).write(oprot, this);
+  }
+
+  @Override
+  public java.lang.String toString() {
+    java.lang.StringBuilder sb = new java.lang.StringBuilder("TFateId(");
+    boolean first = true;
+
+    sb.append("type:");
+    if (this.type == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.type);
+    }
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("tid:");
+    sb.append(this.tid);
+    first = false;
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    // check for sub-struct validity
+  }
+
+  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+    try {
+      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, java.lang.ClassNotFoundException {
+    try {
+      // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor.
+      __isset_bitfield = 0;
+      read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private static class TFateIdStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+    @Override
+    public TFateIdStandardScheme getScheme() {
+      return new TFateIdStandardScheme();
+    }
+  }
+
+  private static class TFateIdStandardScheme extends org.apache.thrift.scheme.StandardScheme<TFateId> {
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol iprot, TFateId struct) throws org.apache.thrift.TException {
+      org.apache.thrift.protocol.TField schemeField;
+      iprot.readStructBegin();
+      while (true)
+      {
+        schemeField = iprot.readFieldBegin();
+        if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+          break;
+        }
+        switch (schemeField.id) {
+          case 1: // TYPE
+            if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
+              struct.type = org.apache.accumulo.core.manager.thrift.TFateInstanceType.findByValue(iprot.readI32());
+              struct.setTypeIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 2: // TID
+            if (schemeField.type == org.apache.thrift.protocol.TType.I64) {
+              struct.tid = iprot.readI64();
+              struct.setTidIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          default:
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+
+      // check for required fields of primitive type, which can't be checked in the validate method
+      struct.validate();
+    }
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol oprot, TFateId struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      if (struct.type != null) {
+        oprot.writeFieldBegin(TYPE_FIELD_DESC);
+        oprot.writeI32(struct.type.getValue());
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldBegin(TID_FIELD_DESC);
+      oprot.writeI64(struct.tid);
+      oprot.writeFieldEnd();
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class TFateIdTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+    @Override
+    public TFateIdTupleScheme getScheme() {
+      return new TFateIdTupleScheme();
+    }
+  }
+
+  private static class TFateIdTupleScheme extends org.apache.thrift.scheme.TupleScheme<TFateId> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, TFateId struct) throws org.apache.thrift.TException {
+      org.apache.thrift.protocol.TTupleProtocol oprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
+      java.util.BitSet optionals = new java.util.BitSet();
+      if (struct.isSetType()) {
+        optionals.set(0);
+      }
+      if (struct.isSetTid()) {
+        optionals.set(1);
+      }
+      oprot.writeBitSet(optionals, 2);
+      if (struct.isSetType()) {
+        oprot.writeI32(struct.type.getValue());
+      }
+      if (struct.isSetTid()) {
+        oprot.writeI64(struct.tid);
+      }
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, TFateId struct) throws org.apache.thrift.TException {
+      org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
+      java.util.BitSet incoming = iprot.readBitSet(2);
+      if (incoming.get(0)) {
+        struct.type = org.apache.accumulo.core.manager.thrift.TFateInstanceType.findByValue(iprot.readI32());
+        struct.setTypeIsSet(true);
+      }
+      if (incoming.get(1)) {
+        struct.tid = iprot.readI64();
+        struct.setTidIsSet(true);
+      }
+    }
+  }
+
+  private static <S extends org.apache.thrift.scheme.IScheme> S scheme(org.apache.thrift.protocol.TProtocol proto) {
+    return (org.apache.thrift.scheme.StandardScheme.class.equals(proto.getScheme()) ? STANDARD_SCHEME_FACTORY : TUPLE_SCHEME_FACTORY).getScheme();
+  }
+  private static void unusedMethod() {}
+}
+
diff --git a/core/src/main/thrift-gen-java/org/apache/accumulo/core/manager/thrift/TFateInstanceType.java b/core/src/main/thrift-gen-java/org/apache/accumulo/core/manager/thrift/TFateInstanceType.java
new file mode 100644
index 0000000000..488e082906
--- /dev/null
+++ b/core/src/main/thrift-gen-java/org/apache/accumulo/core/manager/thrift/TFateInstanceType.java
@@ -0,0 +1,61 @@
+/*
+ * 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.
+ */
+/**
+ * Autogenerated by Thrift Compiler (0.17.0)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package org.apache.accumulo.core.manager.thrift;
+
+
+public enum TFateInstanceType implements org.apache.thrift.TEnum {
+  META(0),
+  USER(1);
+
+  private final int value;
+
+  private TFateInstanceType(int value) {
+    this.value = value;
+  }
+
+  /**
+   * Get the integer value of this enum value, as defined in the Thrift IDL.
+   */
+  @Override
+  public int getValue() {
+    return value;
+  }
+
+  /**
+   * Find a the enum type by its integer value, as defined in the Thrift IDL.
+   * @return null if the value is not found.
+   */
+  @org.apache.thrift.annotation.Nullable
+  public static TFateInstanceType findByValue(int value) { 
+    switch (value) {
+      case 0:
+        return META;
+      case 1:
+        return USER;
+      default:
+        return null;
+    }
+  }
+}
diff --git a/core/src/main/thrift/manager.thrift b/core/src/main/thrift/manager.thrift
index dc9ddbb34a..8a8b413ce6 100644
--- a/core/src/main/thrift/manager.thrift
+++ b/core/src/main/thrift/manager.thrift
@@ -164,12 +164,23 @@ struct ManagerMonitorInfo {
   9:list<BulkImportStatus> bulkImports
 }
 
+enum TFateInstanceType {
+  META
+  USER
+}
+
+struct TFateId {
+  1:TFateInstanceType type
+  2:i64 tid
+}
+
 service FateService {
 
   // register a fate operation by reserving an opid
-  i64 beginFateOperation(
+  TFateId beginFateOperation(
     1:client.TInfo tinfo
     2:security.TCredentials credentials
+    3:TFateInstanceType type
   ) throws (
     1:client.ThriftSecurityException sec
     2:client.ThriftNotActiveServiceException tnase
@@ -179,7 +190,7 @@ service FateService {
   void executeFateOperation(
     1:client.TInfo tinfo
     2:security.TCredentials credentials
-    3:i64 opid
+    3:TFateId opid
     4:FateOperation op
     5:list<binary> arguments
     6:map<string, string> options
@@ -194,7 +205,7 @@ service FateService {
   string waitForFateOperation(
     1:client.TInfo tinfo
     2:security.TCredentials credentials
-    3:i64 opid
+    3:TFateId opid
   ) throws (
     1:client.ThriftSecurityException sec
     2:client.ThriftTableOperationException tope
@@ -205,7 +216,7 @@ service FateService {
   void finishFateOperation(
     1:client.TInfo tinfo
     2:security.TCredentials credentials
-    3:i64 opid
+    3:TFateId opid
   ) throws (
     1:client.ThriftSecurityException sec
     2:client.ThriftNotActiveServiceException tnase
@@ -215,7 +226,7 @@ service FateService {
   bool cancelFateOperation(
     1:client.TInfo tinfo
     2:security.TCredentials credentials
-    3:i64 opid
+    3:TFateId opid
   ) throws (
     1:client.ThriftSecurityException sec
     2:client.ThriftNotActiveServiceException tnase
diff --git a/server/base/src/main/java/org/apache/accumulo/server/init/FileSystemInitializer.java b/server/base/src/main/java/org/apache/accumulo/server/init/FileSystemInitializer.java
index 21c38ff8a0..12cfd0ccbf 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/init/FileSystemInitializer.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/init/FileSystemInitializer.java
@@ -34,6 +34,7 @@ import org.apache.accumulo.core.client.admin.TabletHostingGoal;
 import org.apache.accumulo.core.client.admin.TimeType;
 import org.apache.accumulo.core.clientImpl.TabletHostingGoalUtil;
 import org.apache.accumulo.core.conf.AccumuloConfiguration;
+import org.apache.accumulo.core.conf.DefaultConfiguration;
 import org.apache.accumulo.core.conf.SiteConfiguration;
 import org.apache.accumulo.core.crypto.CryptoFactoryLoader;
 import org.apache.accumulo.core.data.InstanceId;
@@ -43,9 +44,11 @@ import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.fate.zookeeper.ZooReaderWriter;
 import org.apache.accumulo.core.file.FileOperations;
 import org.apache.accumulo.core.file.FileSKVWriter;
+import org.apache.accumulo.core.metadata.FateTable;
 import org.apache.accumulo.core.metadata.MetadataTable;
 import org.apache.accumulo.core.metadata.ReferencedTabletFile;
 import org.apache.accumulo.core.metadata.RootTable;
+import org.apache.accumulo.core.metadata.StoredTabletFile;
 import org.apache.accumulo.core.metadata.schema.DataFileValue;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema;
 import org.apache.accumulo.core.metadata.schema.MetadataTime;
@@ -104,6 +107,13 @@ class FileSystemInitializer {
     String tableMetadataTabletDirUri =
         fs.choose(chooserEnv, context.getBaseUris()) + Constants.HDFS_TABLES_DIR + Path.SEPARATOR
             + MetadataTable.ID + Path.SEPARATOR + tableMetadataTabletDirName;
+    chooserEnv = new VolumeChooserEnvironmentImpl(VolumeChooserEnvironment.Scope.INIT, FateTable.ID,
+        null, context);
+    String fateTableDefaultTabletDirName =
+        MetadataSchema.TabletsSection.ServerColumnFamily.DEFAULT_TABLET_DIR_NAME;
+    String fateTableDefaultTabletDirUri =
+        fs.choose(chooserEnv, context.getBaseUris()) + Constants.HDFS_TABLES_DIR + Path.SEPARATOR
+            + FateTable.ID + Path.SEPARATOR + fateTableDefaultTabletDirName;
     chooserEnv = new VolumeChooserEnvironmentImpl(VolumeChooserEnvironment.Scope.INIT,
         MetadataTable.ID, null, context);
     String defaultMetadataTabletDirName =
@@ -113,11 +123,19 @@ class FileSystemInitializer {
             + MetadataTable.ID + Path.SEPARATOR + defaultMetadataTabletDirName;
 
     // create table and default tablets directories
-    createDirectories(fs, rootTabletDirUri, tableMetadataTabletDirUri, defaultMetadataTabletDirUri);
+    createDirectories(fs, rootTabletDirUri, tableMetadataTabletDirUri, defaultMetadataTabletDirUri,
+        fateTableDefaultTabletDirUri);
+
+    String ext = FileOperations.getNewFileExtension(DefaultConfiguration.getInstance());
+
+    // populate the metadata tablet with info about the fate tablet
+    String metadataFileName = tableMetadataTabletDirUri + Path.SEPARATOR + "0_1." + ext;
+    Tablet fateTablet = new Tablet(FateTable.ID, fateTableDefaultTabletDirName, null, null);
+    createMetadataFile(fs, metadataFileName, siteConfig, fateTablet);
 
     // populate the root tablet with info about the metadata table's two initial tablets
-    Tablet tablesTablet =
-        new Tablet(MetadataTable.ID, tableMetadataTabletDirName, null, splitPoint);
+    Tablet tablesTablet = new Tablet(MetadataTable.ID, tableMetadataTabletDirName, null, splitPoint,
+        StoredTabletFile.of(new Path(metadataFileName)).getMetadata());
     Tablet defaultTablet =
         new Tablet(MetadataTable.ID, defaultMetadataTabletDirName, splitPoint, null);
     createMetadataFile(fs, rootTabletFileUri, siteConfig, tablesTablet, defaultTablet);
@@ -148,6 +166,7 @@ class FileSystemInitializer {
     setTableProperties(context, RootTable.ID, initConfig.getRootMetaConf());
     setTableProperties(context, MetadataTable.ID, initConfig.getRootMetaConf());
     setTableProperties(context, MetadataTable.ID, initConfig.getMetaTableConf());
+    setTableProperties(context, FateTable.ID, initConfig.getFateTableConf());
   }
 
   private void setTableProperties(final ServerContext context, TableId tableId,
diff --git a/server/base/src/main/java/org/apache/accumulo/server/init/InitialConfiguration.java b/server/base/src/main/java/org/apache/accumulo/server/init/InitialConfiguration.java
index bd083b00c2..720140356e 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/init/InitialConfiguration.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/init/InitialConfiguration.java
@@ -41,6 +41,8 @@ class InitialConfiguration {
   private final HashMap<String,String> initialRootMetaConf = new HashMap<>();
   // config for only metadata table
   private final HashMap<String,String> initialMetaConf = new HashMap<>();
+  // config for only fate table
+  private final HashMap<String,String> initialFateTableConf = new HashMap<>();
   private final Configuration hadoopConf;
   private final SiteConfiguration siteConf;
 
@@ -88,6 +90,8 @@ class InitialConfiguration {
         SimpleCompactionDispatcher.class.getName());
     initialMetaConf.put(Property.TABLE_COMPACTION_DISPATCHER_OPTS.getKey() + "service", "meta");
 
+    // TODO configure initial fate table config.. probably needs compaction config??
+
     int max = hadoopConf.getInt("dfs.replication.max", 512);
     // Hadoop 0.23 switched the min value configuration name
     int min = Math.max(hadoopConf.getInt("dfs.replication.min", 1),
@@ -126,6 +130,10 @@ class InitialConfiguration {
     return initialMetaConf;
   }
 
+  HashMap<String,String> getFateTableConf() {
+    return initialFateTableConf;
+  }
+
   Configuration getHadoopConf() {
     return hadoopConf;
   }
diff --git a/server/base/src/main/java/org/apache/accumulo/server/init/ZooKeeperInitializer.java b/server/base/src/main/java/org/apache/accumulo/server/init/ZooKeeperInitializer.java
index 96e4a541ec..cbf59bc52c 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/init/ZooKeeperInitializer.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/init/ZooKeeperInitializer.java
@@ -34,6 +34,7 @@ import org.apache.accumulo.core.fate.zookeeper.ZooReaderWriter;
 import org.apache.accumulo.core.fate.zookeeper.ZooUtil;
 import org.apache.accumulo.core.manager.state.tables.TableState;
 import org.apache.accumulo.core.manager.thrift.ManagerGoalState;
+import org.apache.accumulo.core.metadata.FateTable;
 import org.apache.accumulo.core.metadata.MetadataTable;
 import org.apache.accumulo.core.metadata.RootTable;
 import org.apache.accumulo.core.metadata.StoredTabletFile;
@@ -125,6 +126,8 @@ public class ZooKeeperInitializer {
         RootTable.NAME, TableState.ONLINE, ZooUtil.NodeExistsPolicy.FAIL);
     TableManager.prepareNewTableState(context, MetadataTable.ID, Namespace.ACCUMULO.id(),
         MetadataTable.NAME, TableState.ONLINE, ZooUtil.NodeExistsPolicy.FAIL);
+    TableManager.prepareNewTableState(context, FateTable.ID, Namespace.ACCUMULO.id(),
+        FateTable.NAME, TableState.ONLINE, ZooUtil.NodeExistsPolicy.FAIL);
     zoo.putPersistentData(zkInstanceRoot + Constants.ZTSERVERS, EMPTY_BYTE_ARRAY,
         ZooUtil.NodeExistsPolicy.FAIL);
     zoo.putPersistentData(zkInstanceRoot + Constants.ZPROBLEMS, EMPTY_BYTE_ARRAY,
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 144013dc1e..a26ed3f205 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
@@ -54,13 +54,17 @@ import org.apache.accumulo.core.conf.DefaultConfiguration;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.fate.AdminUtil;
+import org.apache.accumulo.core.fate.FateInstanceType;
 import org.apache.accumulo.core.fate.FateTxId;
 import org.apache.accumulo.core.fate.ReadOnlyFateStore;
 import org.apache.accumulo.core.fate.ZooStore;
+import org.apache.accumulo.core.fate.accumulo.AccumuloStore;
 import org.apache.accumulo.core.fate.zookeeper.ZooCache;
 import org.apache.accumulo.core.fate.zookeeper.ZooReaderWriter;
 import org.apache.accumulo.core.lock.ServiceLock;
 import org.apache.accumulo.core.manager.thrift.FateService;
+import org.apache.accumulo.core.manager.thrift.TFateId;
+import org.apache.accumulo.core.manager.thrift.TFateInstanceType;
 import org.apache.accumulo.core.metadata.MetadataTable;
 import org.apache.accumulo.core.rpc.ThriftUtil;
 import org.apache.accumulo.core.rpc.clients.ThriftClientTypes;
@@ -762,6 +766,9 @@ public class Admin implements KeywordExecutable {
     String fateZkPath = zkRoot + Constants.ZFATE;
     ZooReaderWriter zk = context.getZooReaderWriter();
     ZooStore<Admin> zs = new ZooStore<>(fateZkPath, zk);
+    AccumuloStore<Admin> as = new AccumuloStore<>(context);
+    Map<FateInstanceType,ReadOnlyFateStore<Admin>> fateStores =
+        Map.of(FateInstanceType.META, zs, FateInstanceType.USER, as);
 
     if (fateOpsCommand.cancel) {
       cancelSubmittedFateTxs(context, fateOpsCommand.txList);
@@ -785,13 +792,14 @@ public class Admin implements KeywordExecutable {
       fateOpsCommand.txList.forEach(s -> sortedTxs.add(parseTidFromUserInput(s)));
       EnumSet<ReadOnlyFateStore.TStatus> statusFilter =
           getCmdLineStatusFilters(fateOpsCommand.states);
-      admin.print(zs, zk, zTableLocksPath, new Formatter(System.out), sortedTxs, statusFilter);
+      admin.print(fateStores, zk, zTableLocksPath, new Formatter(System.out), sortedTxs,
+          statusFilter);
       // print line break at the end
       System.out.println();
     }
 
     if (fateOpsCommand.summarize) {
-      summarizeFateTx(context, fateOpsCommand, admin, zs, zTableLocksPath);
+      summarizeFateTx(context, fateOpsCommand, admin, fateStores, zTableLocksPath);
     }
   }
 
@@ -809,8 +817,10 @@ public class Admin implements KeywordExecutable {
   private void cancelSubmittedFateTxs(ServerContext context, List<String> txList)
       throws AccumuloException {
     for (String txStr : txList) {
+      // TODO: We need to pass and then parse the instance type to create TFateId,
+      // maybe something like <type>:txid
       long txid = Long.parseLong(txStr, 16);
-      boolean cancelled = cancelFateOperation(context, txid);
+      boolean cancelled = cancelFateOperation(context, new TFateId(TFateInstanceType.META, txid));
       if (cancelled) {
         System.out.println("FaTE transaction " + FateTxId.formatTid(txid)
             + " was cancelled or already completed.");
@@ -821,7 +831,8 @@ public class Admin implements KeywordExecutable {
     }
   }
 
-  private boolean cancelFateOperation(ClientContext context, long txid) throws AccumuloException {
+  private boolean cancelFateOperation(ClientContext context, TFateId txid)
+      throws AccumuloException {
     FateService.Client client = null;
     try {
       client = ThriftClientTypes.FATE.getConnectionWithRetry(context);
@@ -836,11 +847,12 @@ public class Admin implements KeywordExecutable {
   }
 
   private void summarizeFateTx(ServerContext context, FateOpsCommand cmd, AdminUtil<Admin> admin,
-      ReadOnlyFateStore<Admin> zs, ServiceLock.ServiceLockPath tableLocksPath)
+      Map<FateInstanceType,ReadOnlyFateStore<Admin>> fateStores,
+      ServiceLock.ServiceLockPath tableLocksPath)
       throws InterruptedException, AccumuloException, AccumuloSecurityException, KeeperException {
 
     ZooReaderWriter zk = context.getZooReaderWriter();
-    var transactions = admin.getStatus(zs, zk, tableLocksPath, null, null);
+    var transactions = admin.getStatus(fateStores, zk, tableLocksPath, null, null);
 
     // build id map - relies on unique ids for tables and namespaces
     // used to look up the names of either table or namespace by id.
diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/FateServiceHandler.java b/server/manager/src/main/java/org/apache/accumulo/manager/FateServiceHandler.java
index 08ee56db30..d8567ca2a2 100644
--- a/server/manager/src/main/java/org/apache/accumulo/manager/FateServiceHandler.java
+++ b/server/manager/src/main/java/org/apache/accumulo/manager/FateServiceHandler.java
@@ -68,10 +68,13 @@ import org.apache.accumulo.core.data.NamespaceId;
 import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.dataImpl.KeyExtent;
 import org.apache.accumulo.core.dataImpl.thrift.TRange;
+import org.apache.accumulo.core.fate.FateInstanceType;
 import org.apache.accumulo.core.fate.ReadOnlyFateStore.TStatus;
 import org.apache.accumulo.core.manager.thrift.BulkImportState;
 import org.apache.accumulo.core.manager.thrift.FateOperation;
 import org.apache.accumulo.core.manager.thrift.FateService;
+import org.apache.accumulo.core.manager.thrift.TFateId;
+import org.apache.accumulo.core.manager.thrift.TFateInstanceType;
 import org.apache.accumulo.core.manager.thrift.ThriftPropertyException;
 import org.apache.accumulo.core.securityImpl.thrift.TCredentials;
 import org.apache.accumulo.core.util.ByteBufferUtil;
@@ -118,18 +121,20 @@ class FateServiceHandler implements FateService.Iface {
   }
 
   @Override
-  public long beginFateOperation(TInfo tinfo, TCredentials credentials)
+  public TFateId beginFateOperation(TInfo tinfo, TCredentials credentials, TFateInstanceType type)
       throws ThriftSecurityException {
     authenticate(credentials);
-    return manager.fate().startTransaction();
+    return new TFateId(type, manager.fate(FateInstanceType.fromThrift(type)).startTransaction());
   }
 
   @Override
-  public void executeFateOperation(TInfo tinfo, TCredentials c, long opid, FateOperation op,
+  public void executeFateOperation(TInfo tinfo, TCredentials c, TFateId opid, FateOperation op,
       List<ByteBuffer> arguments, Map<String,String> options, boolean autoCleanup)
       throws ThriftSecurityException, ThriftTableOperationException, ThriftPropertyException {
     authenticate(c);
     String goalMessage = op.toString() + " ";
+    long tid = opid.getTid();
+    FateInstanceType type = FateInstanceType.fromThrift(opid.getType());
 
     switch (op) {
       case NAMESPACE_CREATE: {
@@ -142,7 +147,7 @@ class FateServiceHandler implements FateService.Iface {
         }
 
         goalMessage += "Create " + namespace + " namespace.";
-        manager.fate().seedTransaction(op.toString(), opid,
+        manager.fate(type).seedTransaction(op.toString(), tid,
             new TraceRepo<>(new CreateNamespace(c.getPrincipal(), namespace, options)), autoCleanup,
             goalMessage);
         break;
@@ -161,7 +166,7 @@ class FateServiceHandler implements FateService.Iface {
         }
 
         goalMessage += "Rename " + oldName + " namespace to " + newName;
-        manager.fate().seedTransaction(op.toString(), opid,
+        manager.fate(type).seedTransaction(op.toString(), tid,
             new TraceRepo<>(new RenameNamespace(namespaceId, oldName, newName)), autoCleanup,
             goalMessage);
         break;
@@ -179,7 +184,7 @@ class FateServiceHandler implements FateService.Iface {
         }
 
         goalMessage += "Delete namespace Id: " + namespaceId;
-        manager.fate().seedTransaction(op.toString(), opid,
+        manager.fate(type).seedTransaction(op.toString(), tid,
             new TraceRepo<>(new DeleteNamespace(namespaceId)), autoCleanup, goalMessage);
         break;
       }
@@ -242,7 +247,7 @@ class FateServiceHandler implements FateService.Iface {
         goalMessage += "Create table " + tableName + " " + initialTableState + " with " + splitCount
             + " splits and initial hosting goal of " + initialHostingGoal;
 
-        manager.fate().seedTransaction(op.toString(), opid,
+        manager.fate(type).seedTransaction(op.toString(), tid,
             new TraceRepo<>(new CreateTable(c.getPrincipal(), tableName, timeType, options,
                 splitsPath, splitCount, splitsDirsPath, initialTableState, initialHostingGoal,
                 namespaceId)),
@@ -278,7 +283,7 @@ class FateServiceHandler implements FateService.Iface {
         goalMessage += "Rename table " + oldTableName + "(" + tableId + ") to " + oldTableName;
 
         try {
-          manager.fate().seedTransaction(op.toString(), opid,
+          manager.fate(type).seedTransaction(op.toString(), tid,
               new TraceRepo<>(new RenameTable(namespaceId, tableId, oldTableName, newTableName)),
               autoCleanup, goalMessage);
         } catch (NamespaceNotFoundException e) {
@@ -349,8 +354,8 @@ class FateServiceHandler implements FateService.Iface {
           goalMessage += " and keep offline.";
         }
 
-        manager.fate().seedTransaction(
-            op.toString(), opid, new TraceRepo<>(new CloneTable(c.getPrincipal(), namespaceId,
+        manager.fate(type).seedTransaction(
+            op.toString(), tid, new TraceRepo<>(new CloneTable(c.getPrincipal(), namespaceId,
                 srcTableId, tableName, propertiesToSet, propertiesToExclude, keepOffline)),
             autoCleanup, goalMessage);
 
@@ -379,7 +384,7 @@ class FateServiceHandler implements FateService.Iface {
         }
 
         goalMessage += "Delete table " + tableName + "(" + tableId + ")";
-        manager.fate().seedTransaction(op.toString(), opid,
+        manager.fate(type).seedTransaction(op.toString(), tid,
             new TraceRepo<>(new PreDeleteTable(namespaceId, tableId)), autoCleanup, goalMessage);
         break;
       }
@@ -402,7 +407,7 @@ class FateServiceHandler implements FateService.Iface {
         }
 
         goalMessage += "Online table " + tableId;
-        manager.fate().seedTransaction(op.toString(), opid,
+        manager.fate(type).seedTransaction(op.toString(), tid,
             new TraceRepo<>(new ChangeTableState(namespaceId, tableId, tableOp)), autoCleanup,
             goalMessage);
         break;
@@ -427,7 +432,7 @@ class FateServiceHandler implements FateService.Iface {
         }
 
         goalMessage += "Offline table " + tableId;
-        manager.fate().seedTransaction(op.toString(), opid,
+        manager.fate(type).seedTransaction(op.toString(), tid,
             new TraceRepo<>(new ChangeTableState(namespaceId, tableId, tableOp)), autoCleanup,
             goalMessage);
         break;
@@ -462,7 +467,7 @@ class FateServiceHandler implements FateService.Iface {
             startRowStr, endRowStr);
         goalMessage += "Merge table " + tableName + "(" + tableId + ") splits from " + startRowStr
             + " to " + endRowStr;
-        manager.fate().seedTransaction(op.toString(), opid, new TraceRepo<>(
+        manager.fate(type).seedTransaction(op.toString(), tid, new TraceRepo<>(
             new TableRangeOp(MergeInfo.Operation.MERGE, namespaceId, tableId, startRow, endRow)),
             autoCleanup, goalMessage);
         break;
@@ -494,7 +499,7 @@ class FateServiceHandler implements FateService.Iface {
 
         goalMessage +=
             "Delete table " + tableName + "(" + tableId + ") range " + startRow + " to " + endRow;
-        manager.fate().seedTransaction(op.toString(), opid, new TraceRepo<>(
+        manager.fate(type).seedTransaction(op.toString(), tid, new TraceRepo<>(
             new TableRangeOp(MergeInfo.Operation.DELETE, namespaceId, tableId, startRow, endRow)),
             autoCleanup, goalMessage);
         break;
@@ -520,7 +525,7 @@ class FateServiceHandler implements FateService.Iface {
         }
 
         goalMessage += "Compact table (" + tableId + ") with config " + compactionConfig;
-        manager.fate().seedTransaction(op.toString(), opid,
+        manager.fate(type).seedTransaction(op.toString(), tid,
             new TraceRepo<>(new CompactRange(namespaceId, tableId, compactionConfig)), autoCleanup,
             goalMessage);
         break;
@@ -544,7 +549,7 @@ class FateServiceHandler implements FateService.Iface {
         }
 
         goalMessage += "Cancel compaction of table (" + tableId + ")";
-        manager.fate().seedTransaction(op.toString(), opid,
+        manager.fate(type).seedTransaction(op.toString(), tid,
             new TraceRepo<>(new CancelCompactions(namespaceId, tableId)), autoCleanup, goalMessage);
         break;
       }
@@ -585,8 +590,8 @@ class FateServiceHandler implements FateService.Iface {
         }
 
         goalMessage += "Import table with new name: " + tableName + " from " + exportDirs;
-        manager.fate()
-            .seedTransaction(op.toString(), opid, new TraceRepo<>(new ImportTable(c.getPrincipal(),
+        manager.fate(type)
+            .seedTransaction(op.toString(), tid, new TraceRepo<>(new ImportTable(c.getPrincipal(),
                 tableName, exportDirs, namespaceId, keepMappings, keepOffline)), autoCleanup,
                 goalMessage);
         break;
@@ -615,7 +620,7 @@ class FateServiceHandler implements FateService.Iface {
         }
 
         goalMessage += "Export table " + tableName + "(" + tableId + ") to " + exportDir;
-        manager.fate().seedTransaction(op.toString(), opid,
+        manager.fate(type).seedTransaction(op.toString(), tid,
             new TraceRepo<>(new ExportTable(namespaceId, tableName, tableId, exportDir)),
             autoCleanup, goalMessage);
         break;
@@ -652,7 +657,7 @@ class FateServiceHandler implements FateService.Iface {
         manager.updateBulkImportStatus(dir, BulkImportState.INITIAL);
 
         goalMessage += "Bulk import (v2)  " + dir + " to " + tableName + "(" + tableId + ")";
-        manager.fate().seedTransaction(op.toString(), opid,
+        manager.fate(type).seedTransaction(op.toString(), tid,
             new TraceRepo<>(new PrepBulkImport(tableId, dir, setTime)), autoCleanup, goalMessage);
         break;
       }
@@ -694,7 +699,7 @@ class FateServiceHandler implements FateService.Iface {
 
         goalMessage += "Set Hosting Goal for table: " + tableName + "(" + tableId + ") range: "
             + tRange + " to: " + goal.name();
-        manager.fate().seedTransaction(op.toString(), opid,
+        manager.fate(type).seedTransaction(op.toString(), tid,
             new TraceRepo<>(new SetHostingGoal(tableId, namespaceId, tRange, goal)), autoCleanup,
             goalMessage);
         break;
@@ -771,7 +776,7 @@ class FateServiceHandler implements FateService.Iface {
         }
 
         goalMessage = "Splitting " + extent + " for user into " + (splits.size() + 1) + " tablets";
-        manager.fate().seedTransaction(op.toString(), opid, new PreSplit(extent, splits),
+        manager.fate(type).seedTransaction(op.toString(), tid, new PreSplit(extent, splits),
             autoCleanup, goalMessage);
         break;
       }
@@ -818,13 +823,14 @@ class FateServiceHandler implements FateService.Iface {
   }
 
   @Override
-  public String waitForFateOperation(TInfo tinfo, TCredentials credentials, long opid)
+  public String waitForFateOperation(TInfo tinfo, TCredentials credentials, TFateId opid)
       throws ThriftSecurityException, ThriftTableOperationException {
     authenticate(credentials);
 
-    TStatus status = manager.fate().waitForCompletion(opid);
+    FateInstanceType type = FateInstanceType.fromThrift(opid.getType());
+    TStatus status = manager.fate(type).waitForCompletion(opid.getTid());
     if (status == TStatus.FAILED) {
-      Exception e = manager.fate().getException(opid);
+      Exception e = manager.fate(type).getException(opid.getTid());
       if (e instanceof ThriftTableOperationException) {
         throw (ThriftTableOperationException) e;
       } else if (e instanceof ThriftSecurityException) {
@@ -836,7 +842,7 @@ class FateServiceHandler implements FateService.Iface {
       }
     }
 
-    String ret = manager.fate().getReturn(opid);
+    String ret = manager.fate(type).getReturn(opid.getTid());
     if (ret == null) {
       ret = ""; // thrift does not like returning null
     }
@@ -844,10 +850,10 @@ class FateServiceHandler implements FateService.Iface {
   }
 
   @Override
-  public void finishFateOperation(TInfo tinfo, TCredentials credentials, long opid)
+  public void finishFateOperation(TInfo tinfo, TCredentials credentials, TFateId opid)
       throws ThriftSecurityException {
     authenticate(credentials);
-    manager.fate().delete(opid);
+    manager.fate(FateInstanceType.fromThrift(opid.getType())).delete(opid.getTid());
   }
 
   protected void authenticate(TCredentials credentials) throws ThriftSecurityException {
@@ -926,9 +932,10 @@ class FateServiceHandler implements FateService.Iface {
    *
    * @return the path of the created directory
    */
-  public Path mkTempDir(long opid) throws IOException {
+  public Path mkTempDir(TFateId opid) throws IOException {
     Volume vol = manager.getVolumeManager().getFirst();
-    Path p = vol.prefixChild("/tmp/fate-" + FastFormat.toHexString(opid));
+    Path p = vol
+        .prefixChild("/tmp/fate-" + opid.getType() + "-" + FastFormat.toHexString(opid.getTid()));
     FileSystem fs = vol.getFileSystem();
     if (fs.exists(p)) {
       fs.delete(p, true);
@@ -938,7 +945,7 @@ class FateServiceHandler implements FateService.Iface {
   }
 
   @Override
-  public boolean cancelFateOperation(TInfo tinfo, TCredentials credentials, long opid)
+  public boolean cancelFateOperation(TInfo tinfo, TCredentials credentials, TFateId opid)
       throws ThriftSecurityException, ThriftNotActiveServiceException {
 
     if (!manager.security.canPerformSystemActions(credentials)) {
@@ -946,7 +953,6 @@ class FateServiceHandler implements FateService.Iface {
           SecurityErrorCode.PERMISSION_DENIED);
     }
 
-    return manager.fate().cancel(opid);
+    return manager.fate(FateInstanceType.fromThrift(opid.getType())).cancel(opid.getTid());
   }
-
 }
diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/Manager.java b/server/manager/src/main/java/org/apache/accumulo/manager/Manager.java
index 2418dba1fe..cfb311f383 100644
--- a/server/manager/src/main/java/org/apache/accumulo/manager/Manager.java
+++ b/server/manager/src/main/java/org/apache/accumulo/manager/Manager.java
@@ -73,6 +73,10 @@ import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.dataImpl.KeyExtent;
 import org.apache.accumulo.core.fate.AgeOffStore;
 import org.apache.accumulo.core.fate.Fate;
+import org.apache.accumulo.core.fate.FateInstanceType;
+import org.apache.accumulo.core.fate.FateStore;
+import org.apache.accumulo.core.fate.ZooStore;
+import org.apache.accumulo.core.fate.accumulo.AccumuloStore;
 import org.apache.accumulo.core.fate.zookeeper.ZooReaderWriter;
 import org.apache.accumulo.core.fate.zookeeper.ZooUtil.NodeExistsPolicy;
 import org.apache.accumulo.core.lock.ServiceLock;
@@ -94,6 +98,7 @@ import org.apache.accumulo.core.manager.thrift.ManagerMonitorInfo;
 import org.apache.accumulo.core.manager.thrift.ManagerState;
 import org.apache.accumulo.core.manager.thrift.TableInfo;
 import org.apache.accumulo.core.manager.thrift.TabletServerStatus;
+import org.apache.accumulo.core.metadata.FateTable;
 import org.apache.accumulo.core.metadata.MetadataTable;
 import org.apache.accumulo.core.metadata.RootTable;
 import org.apache.accumulo.core.metadata.TServerInstance;
@@ -155,6 +160,7 @@ import org.apache.zookeeper.Watcher;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import com.google.common.base.Preconditions;
 import com.google.common.collect.ImmutableSortedMap;
 import com.google.common.collect.Maps;
 import com.google.common.util.concurrent.RateLimiter;
@@ -210,11 +216,12 @@ public class Manager extends AbstractServer
 
   private ManagerState state = ManagerState.INITIAL;
 
-  // fateReadyLatch and fateRef go together; when this latch is ready, then the fate reference
-  // should already have been set; still need to use atomic reference or volatile for fateRef, so no
-  // thread's cached view shows that fateRef is still null after the latch is ready
+  // fateReadyLatch and fateRefs go together; when this latch is ready, then the fate references
+  // should already have been set; ConcurrentHashMap will guarantee that all threads will see
+  // the initialized fate references after the latch is ready
   private final CountDownLatch fateReadyLatch = new CountDownLatch(1);
-  private final AtomicReference<Fate<Manager>> fateRef = new AtomicReference<>(null);
+  private final AtomicReference<Map<FateInstanceType,Fate<Manager>>> fateRefs =
+      new AtomicReference<>();
 
   volatile SortedMap<TServerInstance,TabletServerStatus> tserverStatus = emptySortedMap();
   volatile SortedMap<TabletServerId,TServerStatus> tserverStatusForBalancer = emptySortedMap();
@@ -265,7 +272,7 @@ public class Manager extends AbstractServer
    *
    * @return the Fate object, only after the fate components are running and ready
    */
-  public Fate<Manager> fate() {
+  public Fate<Manager> fate(FateInstanceType type) {
     try {
       // block up to 30 seconds until it's ready; if it's still not ready, introduce some logging
       if (!fateReadyLatch.await(30, SECONDS)) {
@@ -286,7 +293,7 @@ public class Manager extends AbstractServer
       Thread.currentThread().interrupt();
       throw new IllegalStateException("Thread was interrupted; cannot proceed");
     }
-    return fateRef.get();
+    return getFateRefs().get(type);
   }
 
   static final boolean X = true;
@@ -329,7 +336,7 @@ public class Manager extends AbstractServer
     }
 
     if (oldState != newState && (newState == ManagerState.NORMAL)) {
-      if (fateRef.get() != null) {
+      if (!getFateRefs().isEmpty()) {
         throw new IllegalStateException("Access to Fate should not have been"
             + " initialized prior to the Manager finishing upgrades. Please save"
             + " all logs and file a bug.");
@@ -1068,17 +1075,17 @@ public class Manager extends AbstractServer
     }
 
     try {
-      final AgeOffStore<Manager> store = new AgeOffStore<>(
-          new org.apache.accumulo.core.fate.ZooStore<>(getZooKeeperRoot() + Constants.ZFATE,
-              context.getZooReaderWriter()),
-          HOURS.toMillis(8), System::currentTimeMillis);
-
-      Fate<Manager> f = new Fate<>(this, store, TraceRepo::toLogString, getConfiguration());
-      fateRef.set(f);
+      var metaInstance = initializeFateInstance(context, FateInstanceType.META,
+          new ZooStore<>(getZooKeeperRoot() + Constants.ZFATE, context.getZooReaderWriter()));
+      var userInstance = initializeFateInstance(context, FateInstanceType.USER,
+          new AccumuloStore<>(context, FateTable.NAME));
+
+      if (!fateRefs.compareAndSet(null,
+          Map.of(FateInstanceType.META, metaInstance, FateInstanceType.USER, userInstance))) {
+        throw new IllegalStateException(
+            "Unexpected previous fate reference map already initialized");
+      }
       fateReadyLatch.countDown();
-
-      ThreadPools.watchCriticalScheduledTask(context.getScheduledExecutor()
-          .scheduleWithFixedDelay(store::ageOff, 63000, 63000, MILLISECONDS));
     } catch (KeeperException | InterruptedException e) {
       throw new IllegalStateException("Exception setting up FaTE cleanup thread", e);
     }
@@ -1139,7 +1146,7 @@ public class Manager extends AbstractServer
       sleepUninterruptibly(500, MILLISECONDS);
     }
     log.info("Shutting down fate.");
-    fate().shutdown();
+    getFateRefs().keySet().forEach(type -> fate(type).shutdown());
 
     splitter.stop();
 
@@ -1178,6 +1185,20 @@ public class Manager extends AbstractServer
     log.info("exiting");
   }
 
+  private Fate<Manager> initializeFateInstance(ServerContext context, FateInstanceType type,
+      FateStore<Manager> store) {
+    final AgeOffStore<Manager> ageOffStore =
+        new AgeOffStore<>(store, HOURS.toMillis(8), System::currentTimeMillis);
+
+    final Fate<Manager> fateInstance =
+        new Fate<>(this, ageOffStore, TraceRepo::toLogString, getConfiguration());
+
+    ThreadPools.watchCriticalScheduledTask(context.getScheduledExecutor()
+        .scheduleWithFixedDelay(ageOffStore::ageOff, 63000, 63000, MILLISECONDS));
+
+    return fateInstance;
+  }
+
   /**
    * Allows property configuration to block manager start-up waiting for a minimum number of
    * tservers to register in zookeeper. It also accepts a maximum time to wait - if the time
@@ -1656,4 +1677,10 @@ public class Manager extends AbstractServer
     super.registerMetrics(registry);
     compactionCoordinator.registerMetrics(registry);
   }
+
+  private Map<FateInstanceType,Fate<Manager>> getFateRefs() {
+    var fateRefs = this.fateRefs.get();
+    Preconditions.checkState(fateRefs != null, "Unexpected null fate references map");
+    return fateRefs;
+  }
 }
diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/ManagerClientServiceHandler.java b/server/manager/src/main/java/org/apache/accumulo/manager/ManagerClientServiceHandler.java
index 97b9a07682..4d3980a9ed 100644
--- a/server/manager/src/main/java/org/apache/accumulo/manager/ManagerClientServiceHandler.java
+++ b/server/manager/src/main/java/org/apache/accumulo/manager/ManagerClientServiceHandler.java
@@ -57,6 +57,7 @@ import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.dataImpl.KeyExtent;
 import org.apache.accumulo.core.dataImpl.thrift.TKeyExtent;
 import org.apache.accumulo.core.fate.Fate;
+import org.apache.accumulo.core.fate.FateInstanceType;
 import org.apache.accumulo.core.fate.zookeeper.ZooReaderWriter;
 import org.apache.accumulo.core.manager.thrift.ManagerClientService;
 import org.apache.accumulo.core.manager.thrift.ManagerGoalState;
@@ -329,7 +330,7 @@ public class ManagerClientServiceHandler implements ManagerClientService.Iface {
       }
     }
 
-    Fate<Manager> fate = manager.fate();
+    Fate<Manager> fate = manager.fate(FateInstanceType.META);
     long tid = fate.startTransaction();
 
     String msg = "Shutdown tserver " + tabletServer;
diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/metrics/fate/FateMetricValues.java b/server/manager/src/main/java/org/apache/accumulo/manager/metrics/fate/FateMetricValues.java
index e31ff65398..4f99d01586 100644
--- a/server/manager/src/main/java/org/apache/accumulo/manager/metrics/fate/FateMetricValues.java
+++ b/server/manager/src/main/java/org/apache/accumulo/manager/metrics/fate/FateMetricValues.java
@@ -24,6 +24,7 @@ import java.util.Map;
 import java.util.TreeMap;
 
 import org.apache.accumulo.core.fate.AdminUtil;
+import org.apache.accumulo.core.fate.FateInstanceType;
 import org.apache.accumulo.core.fate.ReadOnlyFateStore;
 import org.apache.accumulo.server.ServerContext;
 import org.apache.zookeeper.KeeperException;
@@ -110,7 +111,7 @@ class FateMetricValues {
     try {
 
       List<AdminUtil.TransactionStatus> currFates =
-          admin.getTransactionStatus(zooStore, null, null);
+          admin.getTransactionStatus(Map.of(FateInstanceType.META, zooStore), null, null);
 
       builder.withCurrentFateOps(currFates.size());
 
diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/split/SplitTask.java b/server/manager/src/main/java/org/apache/accumulo/manager/split/SplitTask.java
index 0c60e9347f..18958c92be 100644
--- a/server/manager/src/main/java/org/apache/accumulo/manager/split/SplitTask.java
+++ b/server/manager/src/main/java/org/apache/accumulo/manager/split/SplitTask.java
@@ -21,6 +21,7 @@ package org.apache.accumulo.manager.split;
 import java.time.Duration;
 import java.util.SortedSet;
 
+import org.apache.accumulo.core.fate.FateInstanceType;
 import org.apache.accumulo.core.metadata.schema.TabletMetadata;
 import org.apache.accumulo.manager.Manager;
 import org.apache.accumulo.manager.tableOps.split.PreSplit;
@@ -83,9 +84,11 @@ public class SplitTask implements Runnable {
         return;
       }
 
-      long fateTxId = manager.fate().startTransaction();
+      var fateInstanceType = FateInstanceType.fromTableId((tablet.getTableId()));
+      long fateTxId = manager.fate(fateInstanceType).startTransaction();
 
-      manager.fate().seedTransaction("SYSTEM_SPLIT", fateTxId, new PreSplit(extent, splits), true,
+      manager.fate(fateInstanceType).seedTransaction("SYSTEM_SPLIT", fateTxId,
+          new PreSplit(extent, splits), true,
           "System initiated split of tablet " + extent + " into " + splits.size() + " splits");
     } catch (Exception e) {
       log.error("Failed to split {}", tablet.getExtent(), e);
diff --git a/test/src/main/java/org/apache/accumulo/test/MetaSplitIT.java b/test/src/main/java/org/apache/accumulo/test/MetaSplitIT.java
index 013a1e7dbd..dad2782288 100644
--- a/test/src/main/java/org/apache/accumulo/test/MetaSplitIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/MetaSplitIT.java
@@ -188,10 +188,11 @@ public class MetaSplitIT extends AccumuloClusterHarness {
   private void verifyMetadataTableScan(AccumuloClient client) throws Exception {
     var tables = client.tableOperations().tableIdMap();
     var expectedExtents = tables.entrySet().stream()
-        .filter(e -> !e.getKey().startsWith("accumulo.")).map(Map.Entry::getValue).map(TableId::of)
-        .map(tid -> new KeyExtent(tid, null, null)).collect(Collectors.toSet());
-    // Verify we have 10 tablets for metadata
-    assertEquals(10, expectedExtents.size());
+        .filter(e -> !e.getKey().equals(RootTable.NAME) && !e.getKey().equals(MetadataTable.NAME))
+        .map(Map.Entry::getValue).map(TableId::of).map(tid -> new KeyExtent(tid, null, null))
+        .collect(Collectors.toSet());
+    // Verify we have 11 tablets for metadata (Includes FateTable)
+    assertEquals(11, expectedExtents.size());
 
     // Scan each tablet to verify data exists
     var ample = ((ClientContext) client).getAmple();
diff --git a/test/src/main/java/org/apache/accumulo/test/NamespacesIT.java b/test/src/main/java/org/apache/accumulo/test/NamespacesIT.java
index 63b3240e58..6df062d2fb 100644
--- a/test/src/main/java/org/apache/accumulo/test/NamespacesIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/NamespacesIT.java
@@ -135,7 +135,7 @@ public class NamespacesIT extends SharedMiniClusterBase {
         c.tableOperations().delete(t);
       }
     }
-    assertEquals(2, c.tableOperations().list().size());
+    assertEquals(3, c.tableOperations().list().size());
     for (String n : c.namespaceOperations().list()) {
       if (!n.equals(Namespace.ACCUMULO.name()) && !n.equals(Namespace.DEFAULT.name())) {
         c.namespaceOperations().delete(n);
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/FateConcurrencyIT.java b/test/src/main/java/org/apache/accumulo/test/functional/FateConcurrencyIT.java
index a9e0a437a8..68c24ff3cb 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/FateConcurrencyIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/FateConcurrencyIT.java
@@ -30,6 +30,7 @@ import java.io.IOException;
 import java.time.Duration;
 import java.util.Arrays;
 import java.util.List;
+import java.util.Map;
 import java.util.concurrent.Callable;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
@@ -43,11 +44,15 @@ import org.apache.accumulo.core.client.AccumuloException;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.TableNotFoundException;
 import org.apache.accumulo.core.clientImpl.ClientContext;
+import org.apache.accumulo.core.clientImpl.Namespace;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.data.InstanceId;
 import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.fate.AdminUtil;
+import org.apache.accumulo.core.fate.FateInstanceType;
+import org.apache.accumulo.core.fate.ReadOnlyFateStore;
 import org.apache.accumulo.core.fate.ZooStore;
+import org.apache.accumulo.core.fate.accumulo.AccumuloStore;
 import org.apache.accumulo.core.fate.zookeeper.ZooReaderWriter;
 import org.apache.accumulo.core.fate.zookeeper.ZooUtil;
 import org.apache.accumulo.core.lock.ServiceLock;
@@ -192,11 +197,14 @@ public class FateConcurrencyIT extends AccumuloClusterHarness {
   }
 
   private boolean findFate(String aTableName) {
+    boolean isMeta = aTableName.startsWith(Namespace.ACCUMULO.name());
     log.debug("Look for fate {}", aTableName);
     for (int retry = 0; retry < 5; retry++) {
       try {
-        boolean found = lookupFateInZookeeper(aTableName);
-        log.trace("Try {}: Fate in zk for table {} : {}", retry, aTableName, found);
+        boolean found =
+            isMeta ? lookupFateInZookeeper(aTableName) : lookupFateInAccumulo(aTableName);
+        log.trace("Try {}: Fate in {} for table {} : {}", retry, isMeta ? "zk" : "accumulo",
+            aTableName, found);
         if (found) {
           log.debug("Found fate {}", aTableName);
           return true;
@@ -257,11 +265,14 @@ public class FateConcurrencyIT extends AccumuloClusterHarness {
         ZooStore<String> zs = new ZooStore<>(ZooUtil.getRoot(instanceId) + Constants.ZFATE, zk);
         var lockPath =
             ServiceLock.path(ZooUtil.getRoot(instanceId) + Constants.ZTABLE_LOCKS + "/" + tableId);
+        AccumuloStore<String> as = new AccumuloStore<>(context);
+        Map<FateInstanceType,ReadOnlyFateStore<String>> fateStores =
+            Map.of(FateInstanceType.META, zs, FateInstanceType.USER, as);
 
-        withLocks = admin.getStatus(zs, zk, lockPath, null, null);
+        withLocks = admin.getStatus(fateStores, zk, lockPath, null, null);
 
         // call method that does not use locks.
-        noLocks = admin.getTransactionStatus(zs, null, null);
+        noLocks = admin.getTransactionStatus(fateStores, null, null);
 
         // no zk exception, no need to retry
         break;
@@ -366,6 +377,33 @@ public class FateConcurrencyIT extends AccumuloClusterHarness {
     return Boolean.FALSE;
   }
 
+  private boolean lookupFateInAccumulo(final String tableName) throws KeeperException {
+    AdminUtil<String> admin = new AdminUtil<>(false);
+
+    try {
+      TableId tableId = context.getTableId(tableName);
+
+      log.trace("tid: {}", tableId);
+
+      AccumuloStore<String> as = new AccumuloStore<>(context);
+      AdminUtil.FateStatus fateStatus = admin.getStatus(as, null, null);
+
+      log.trace("current fates: {}", fateStatus.getTransactions().size());
+
+      for (AdminUtil.TransactionStatus tx : fateStatus.getTransactions()) {
+        if (isCompaction(tx)) {
+          return true;
+        }
+      }
+
+    } catch (TableNotFoundException | InterruptedException ex) {
+      throw new IllegalStateException(ex);
+    }
+
+    // did not find appropriate fate transaction for compaction.
+    return Boolean.FALSE;
+  }
+
   /**
    * Test that the transaction top contains "CompactionDriver" and the debug message contains
    * "CompactRange"
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/SplitIT.java b/test/src/main/java/org/apache/accumulo/test/functional/SplitIT.java
index 2d7a71aff3..c2c5b7d0cf 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/SplitIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/SplitIT.java
@@ -62,6 +62,7 @@ import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.Ta
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.util.Pair;
 import org.apache.accumulo.harness.AccumuloClusterHarness;
+import org.apache.accumulo.minicluster.MemoryUnit;
 import org.apache.accumulo.minicluster.ServerType;
 import org.apache.accumulo.miniclusterImpl.MiniAccumuloConfigImpl;
 import org.apache.accumulo.server.util.CheckForMetadataProblems;
@@ -92,6 +93,7 @@ public class SplitIT extends AccumuloClusterHarness {
   @Override
   public void configureMiniCluster(MiniAccumuloConfigImpl cfg, Configuration hadoopCoreSite) {
     cfg.setProperty(Property.TSERV_MAXMEM, "5K");
+    cfg.setMemory(ServerType.TABLET_SERVER, 384, MemoryUnit.MEGABYTE);
   }
 
   private String tservMaxMem, tservMajcDelay;
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/TabletManagementIteratorIT.java b/test/src/main/java/org/apache/accumulo/test/functional/TabletManagementIteratorIT.java
index 06c3bec142..0213a7f48d 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/TabletManagementIteratorIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/TabletManagementIteratorIT.java
@@ -61,6 +61,7 @@ import org.apache.accumulo.core.lock.ServiceLock;
 import org.apache.accumulo.core.manager.state.TabletManagement;
 import org.apache.accumulo.core.manager.state.tables.TableState;
 import org.apache.accumulo.core.manager.thrift.ManagerState;
+import org.apache.accumulo.core.metadata.FateTable;
 import org.apache.accumulo.core.metadata.MetadataTable;
 import org.apache.accumulo.core.metadata.StoredTabletFile;
 import org.apache.accumulo.core.metadata.TServerInstance;
@@ -400,10 +401,10 @@ public class TabletManagementIteratorIT extends AccumuloClusterHarness {
         while (row.hasNext()) {
           Entry<Key,Value> entry = row.next();
           Key k = entry.getKey();
+
           if (m == null) {
             m = new Mutation(k.getRow());
           }
-
           m.put(k.getColumnFamily(), k.getColumnQualifier(), k.getColumnVisibilityParsed(),
               k.getTimestamp(), entry.getValue());
         }
@@ -412,9 +413,10 @@ public class TabletManagementIteratorIT extends AccumuloClusterHarness {
       }
     }
 
-    // metadata should be stable with only 6 rows (2 for each table)
+    // metadata should be stable with only 9 rows (2 for each table) + 1 for the FateTable
     log.debug("Gathered {} rows to create copy {}", mutations.size(), copy);
-    assertEquals(8, mutations.size(), "Metadata should have 8 rows (2 for each table)");
+    assertEquals(9, mutations.size(),
+        "Metadata should have 8 rows (2 for each table) + one row for " + FateTable.ID.canonical());
     client.tableOperations().create(copy);
 
     try (BatchWriter writer = client.createBatchWriter(copy)) {
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/WALSunnyDayIT.java b/test/src/main/java/org/apache/accumulo/test/functional/WALSunnyDayIT.java
index 8114b1b950..4a93d5acc1 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/WALSunnyDayIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/WALSunnyDayIT.java
@@ -49,6 +49,7 @@ import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Mutation;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.dataImpl.KeyExtent;
+import org.apache.accumulo.core.metadata.FateTable;
 import org.apache.accumulo.core.metadata.MetadataTable;
 import org.apache.accumulo.core.metadata.RootTable;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection;
@@ -122,7 +123,8 @@ public class WALSunnyDayIT extends ConfigurableMacBase {
       assertEquals(3, countInUse(walsAfterRoll.values()), "all WALs should be in use");
 
       // flush the tables
-      for (String table : new String[] {tableName, MetadataTable.NAME, RootTable.NAME}) {
+      for (String table : new String[] {tableName, MetadataTable.NAME, RootTable.NAME,
+          FateTable.NAME}) {
         c.tableOperations().flush(table, null, null, true);
       }
       Thread.sleep(SECONDS.toMillis(1));
@@ -148,9 +150,14 @@ public class WALSunnyDayIT extends ConfigurableMacBase {
       Thread.sleep(SECONDS.toMillis(5));
       Map<KeyExtent,List<String>> markers = getRecoveryMarkers(c);
       // log.debug("markers " + markers);
-      assertEquals(1, markers.size(), "one tablet should have markers");
-      assertEquals("1", markers.keySet().iterator().next().tableId().canonical(),
+      // There should be markers for the created table and also the FateTable
+      assertEquals(2, markers.size(), "two tablets should have markers");
+      assertTrue(
+          markers.keySet().stream().anyMatch(extent -> extent.tableId().canonical().equals("1")),
           "tableId of the keyExtent should be 1");
+      assertTrue(
+          markers.keySet().stream().anyMatch(extent -> extent.tableId().equals(FateTable.ID)),
+          "tableId of the FateTable can't be found");
 
       // put some data in the WAL
       assertEquals(0, cluster.exec(SetGoalState.class, "NORMAL").getProcess().waitFor());