You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@accumulo.apache.org by dl...@apache.org on 2021/06/01 14:11:59 UTC

[accumulo] branch main updated: External Compactions feature (#1451)

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

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


The following commit(s) were added to refs/heads/main by this push:
     new 8a636a3  External Compactions feature (#1451)
8a636a3 is described below

commit 8a636a3ba91f5dae1d8b09b095178889a7d79c1d
Author: Dave Marion <dl...@apache.org>
AuthorDate: Tue Jun 1 10:11:50 2021 -0400

    External Compactions feature (#1451)
    
    Added a feature to perform major compactions outside of the TabletServer. This
    feature is optional and disabled by default. External compactions can be
    accomplished by using the type 'external' in the compaction service configuration
    for a table and starting two new server processes, the CompactionCoordinator and
    the Compactor. External compactions:
    
      * Allow compactions to outlive a TabletServer
      * Allow compactions to occur concurrent to a Tablet being re-hosted
      * Reduce the load on the TabletServer, giving it more cycles to insert mutations and respond to scans
      * Allow compactions to scale up and down independent of tablet server scaling and tablet locations. Tablet location does not constrain where compactions can run.  This allows tablets on a few tservers with lots of compaction work to spread the work out in ways not possible before.
    
    
    Co-authored-by: Keith Turner <kt...@apache.org>
---
 assemble/conf/accumulo-env.sh                      |    2 +-
 assemble/pom.xml                                   |   10 +
 .../java/org/apache/accumulo/core/Constants.java   |    5 +
 .../core/client/admin/ActiveCompaction.java        |   22 +
 .../core/client/admin/InstanceOperations.java      |   13 +-
 .../core/clientImpl/ActiveCompactionImpl.java      |   29 +-
 .../core/clientImpl/InstanceOperationsImpl.java    |   56 +-
 .../org/apache/accumulo/core/conf/Property.java    |   49 +-
 .../core/iterators/user/LargeRowFilter.java        |   12 +-
 .../core/iterators/user/RowDeletingIterator.java   |   12 +-
 .../iteratorsImpl/system/DeletingIterator.java     |   14 +-
 .../accumulo/core/metadata/schema/Ample.java       |   20 +
 .../accumulo/core/metadata/schema/AmpleImpl.java   |    4 +
 .../schema/ExternalCompactionFinalState.java       |  139 +
 .../core/metadata/schema/ExternalCompactionId.java |   59 +
 .../schema/ExternalCompactionMetadata.java         |  165 +
 .../core/metadata/schema/MetadataSchema.java       |   17 +
 .../core/metadata/schema/TabletMetadata.java       |   17 +-
 .../core/metadata/schema/TabletsMetadata.java      |    4 +
 .../core/spi/compaction/CompactionExecutorId.java  |    6 +-
 .../spi/compaction/DefaultCompactionPlanner.java   |   96 +-
 .../core/spi/compaction/ExecutorManager.java       |    5 +
 .../apache/accumulo/core/util/ServerServices.java  |    2 +-
 .../compaction/CompactionExecutorIdImpl.java}      |   39 +-
 .../core/util/compaction/CompactionJobImpl.java    |   21 +-
 .../core/util/compaction/CompactionPlanImpl.java   |    5 +-
 .../util/compaction/ExternalCompactionUtil.java    |  270 +
 .../accumulo/fate/zookeeper/ServiceLock.java       |    4 +-
 core/src/main/scripts/generate-thrift.sh           |    2 +-
 core/src/main/spotbugs/exclude-filter.xml          |    1 +
 .../thrift/CompactionCoordinatorService.java       | 5477 ++++++++++++++++++
 .../core/compaction/thrift/CompactorService.java   | 4643 ++++++++++++++++
 .../accumulo/core/compaction/thrift/Status.java    |  826 +++
 .../thrift/TCompactionState.java}                  |   31 +-
 .../thrift/UnknownCompactionIdException.java       |  282 +
 .../core/tabletserver/thrift/ActiveCompaction.java |   48 +-
 .../core/tabletserver/thrift/InputFile.java        |  687 +++
 ...{CompactionReason.java => TCompactionKind.java} |   25 +-
 .../thrift/TCompactionQueueSummary.java            |  495 ++
 ...ompactionReason.java => TCompactionReason.java} |    6 +-
 .../core/tabletserver/thrift/TCompactionStats.java |  582 ++
 .../{CompactionType.java => TCompactionType.java}  |    6 +-
 .../thrift/TExternalCompactionJob.java             | 1338 +++++
 .../tabletserver/thrift/TabletClientService.java   | 5793 +++++++++++++++++---
 core/src/main/thrift/compaction-coordinator.thrift |  140 +
 core/src/main/thrift/data.thrift                   |    1 +
 core/src/main/thrift/tabletserver.thrift           |   79 +-
 .../compaction/DefaultCompactionPlannerTest.java   |   37 +-
 .../util/compaction/CompactionPrioritizerTest.java |    4 +-
 .../miniclusterImpl/MiniAccumuloClusterImpl.java   |    2 +-
 pom.xml                                            |   12 +
 .../server/compaction}/CompactionInfo.java         |   53 +-
 .../server/compaction}/CompactionStats.java        |    4 +-
 .../server/compaction}/CountingIterator.java       |    2 +-
 .../accumulo/server/compaction/FileCompactor.java} |   74 +-
 .../server/compaction/RetryableThriftCall.java     |  128 +
 .../compaction/RetryableThriftFunction.java}       |   17 +-
 .../server/constraints/MetadataConstraints.java    |    4 +-
 .../apache/accumulo/server/fs}/FileManager.java    |    2 +-
 .../accumulo/server/fs}/TooManyFilesException.java |    2 +-
 .../apache/accumulo/server/init/Initialize.java    |    7 +
 .../iterators/SystemIteratorEnvironment.java       |    5 +
 .../iterators}/TabletIteratorEnvironment.java      |    6 +-
 .../accumulo/server/manager/LiveTServerSet.java    |    4 +
 .../accumulo/server/metadata/ServerAmpleImpl.java  |   52 +
 .../server/metadata/TabletMutatorBase.java         |   17 +
 .../apache/accumulo/server/rpc/TServerUtils.java   |    9 +-
 .../accumulo/server/util/ManagerMetadataUtil.java  |    7 +-
 .../accumulo/server/util/MetadataTableUtil.java    |    7 +-
 .../accumulo/server/util/TServerUtilsTest.java     |    3 +-
 server/compaction-coordinator/.gitignore           |   28 +
 server/compaction-coordinator/pom.xml              |  101 +
 .../coordinator/CompactionCoordinator.java         |  658 +++
 .../accumulo/coordinator/CompactionFinalizer.java  |  219 +
 .../accumulo/coordinator/CompactionUpdate.java}    |   30 +-
 .../coordinator/CoordinatorExecutable.java         |   38 +-
 .../coordinator/CoordinatorLockWatcher.java        |   94 +
 .../coordinator/DeadCompactionDetector.java        |  121 +
 .../coordinator/ExternalCompactionMetrics.java     |   82 +
 .../accumulo/coordinator/QueueAndPriority.java     |   89 +
 .../accumulo/coordinator/QueueSummaries.java       |  220 +
 .../accumulo/coordinator/RunningCompaction.java    |   55 +
 .../coordinator/CompactionCoordinatorTest.java     |  610 +++
 .../accumulo/coordinator/QueueSummariesTest.java   |  194 +
 .../src/test/resources/log4j2-test.properties      |   35 +
 server/compactor/.gitignore                        |   28 +
 server/compactor/pom.xml                           |  104 +
 .../accumulo/compactor/CompactionEnvironment.java  |  114 +
 .../accumulo/compactor/CompactionJobHolder.java    |   78 +
 .../org/apache/accumulo/compactor/Compactor.java   |  897 +++
 .../accumulo/compactor/CompactorExecutable.java}   |   30 +-
 .../apache/accumulo/compactor/CompactorTest.java   |  466 ++
 .../src/test/resources/log4j2-test.properties      |   35 +
 .../accumulo/manager/TabletGroupWatcher.java       |   16 +-
 .../accumulo/manager/upgrade/Upgrader9to10.java    |   17 +
 .../accumulo/tserver/ConditionCheckerContext.java  |    1 +
 .../org/apache/accumulo/tserver/TabletServer.java  |    9 +-
 .../tserver/TabletServerResourceManager.java       |    3 +-
 .../accumulo/tserver/ThriftClientHandler.java      |   76 +-
 .../accumulo/tserver/compactions/Compactable.java  |   26 +
 .../tserver/compactions/CompactionExecutor.java    |  193 +-
 .../tserver/compactions/CompactionManager.java     |  159 +-
 .../tserver/compactions/CompactionService.java     |   74 +-
 .../compactions/ExternalCompactionExecutor.java    |  224 +
 .../tserver/compactions/ExternalCompactionJob.java |   96 +
 ...ecutor.java => InternalCompactionExecutor.java} |   43 +-
 .../metrics/CompactionExecutorsMetrics.java        |   60 +-
 .../accumulo/tserver/scan/NextBatchTask.java       |    2 +-
 .../accumulo/tserver/tablet/CompactableImpl.java   |  637 ++-
 .../accumulo/tserver/tablet/CompactableUtils.java  |  139 +-
 .../accumulo/tserver/tablet/CompactionWatcher.java |    4 +-
 .../accumulo/tserver/tablet/DatafileManager.java   |   18 +-
 .../accumulo/tserver/tablet/MinorCompactor.java    |   42 +-
 .../accumulo/tserver/tablet/ScanDataSource.java    |    4 +-
 .../org/apache/accumulo/tserver/tablet/Tablet.java |   56 +-
 .../apache/accumulo/tserver/tablet/TabletData.java |   10 +
 .../accumulo/tserver/CountingIteratorTest.java     |    2 +-
 .../apache/accumulo/tserver/tablet/TabletTest.java |   26 +-
 .../shell/commands/ActiveCompactionHelper.java     |   84 +-
 .../shell/commands/ListCompactionsCommand.java     |   14 +-
 test/pom.xml                                       |   26 +
 .../apache/accumulo/test/CompactionExecutorIT.java |    2 +-
 .../apache/accumulo/test/ExternalCompactionIT.java | 1085 ++++
 .../accumulo/test/ExternalCompactionTServer.java   |   34 +-
 .../accumulo/test/ExternalDoNothingCompactor.java  |   91 +
 ...ttingExternalCompactionThriftClientHandler.java |   50 +
 .../accumulo/test/TestCompactionCoordinator.java   |  169 +
 .../TestCompactionCoordinatorForOfflineTable.java  |   79 +
 .../accumulo/test/functional/SplitRecoveryIT.java  |    3 +-
 .../accumulo/test/performance/NullTserver.java     |   27 +
 .../resources/hadoop-metrics2-accumulo.properties  |    7 +-
 test/src/main/resources/log4j2-test.properties     |    3 +
 132 files changed, 28578 insertions(+), 1376 deletions(-)

diff --git a/assemble/conf/accumulo-env.sh b/assemble/conf/accumulo-env.sh
index 3161b8b..d5b6012 100644
--- a/assemble/conf/accumulo-env.sh
+++ b/assemble/conf/accumulo-env.sh
@@ -95,7 +95,7 @@ JAVA_OPTS=("${JAVA_OPTS[@]}"
 )
 
 case "$cmd" in
-  monitor|gc|manager|master|tserver|tracer)
+  monitor|gc|manager|master|tserver|tracer|compaction-coordinator|compactor)
     JAVA_OPTS=("${JAVA_OPTS[@]}" "-Dlog4j.configurationFile=log4j2-service.properties")
     ;;
   *)
diff --git a/assemble/pom.xml b/assemble/pom.xml
index ee4fcda..3b6ea0d 100644
--- a/assemble/pom.xml
+++ b/assemble/pom.xml
@@ -171,6 +171,16 @@
     </dependency>
     <dependency>
       <groupId>org.apache.accumulo</groupId>
+      <artifactId>accumulo-compaction-coordinator</artifactId>
+      <optional>true</optional>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.accumulo</groupId>
+      <artifactId>accumulo-compactor</artifactId>
+      <optional>true</optional>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.accumulo</groupId>
       <artifactId>accumulo-core</artifactId>
       <optional>true</optional>
     </dependency>
diff --git a/core/src/main/java/org/apache/accumulo/core/Constants.java b/core/src/main/java/org/apache/accumulo/core/Constants.java
index 96b8b43..e2fac29 100644
--- a/core/src/main/java/org/apache/accumulo/core/Constants.java
+++ b/core/src/main/java/org/apache/accumulo/core/Constants.java
@@ -59,6 +59,11 @@ public class Constants {
 
   public static final String ZTSERVERS = "/tservers";
 
+  public static final String ZCOMPACTORS = "/compactors";
+
+  public static final String ZCOORDINATOR = "/coordinators";
+  public static final String ZCOORDINATOR_LOCK = ZCOORDINATOR + "/lock";
+
   public static final String ZDEAD = "/dead";
   public static final String ZDEADTSERVERS = ZDEAD + "/tservers";
 
diff --git a/core/src/main/java/org/apache/accumulo/core/client/admin/ActiveCompaction.java b/core/src/main/java/org/apache/accumulo/core/client/admin/ActiveCompaction.java
index 5a404e1..4506692 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/admin/ActiveCompaction.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/admin/ActiveCompaction.java
@@ -128,4 +128,26 @@ public abstract class ActiveCompaction {
    * @return the per compaction iterators configured
    */
   public abstract List<IteratorSetting> getIterators();
+
+  /**
+   * @since 2.1.0
+   */
+  public interface CompactionHost {
+    enum Type {
+      TSERVER, COMPACTOR
+    }
+
+    Type getType();
+
+    String getAddress();
+
+    int getPort();
+  }
+
+  /**
+   * Return the host where the compaction is running.
+   *
+   * @since 2.1.0
+   */
+  public abstract CompactionHost getHost();
 }
diff --git a/core/src/main/java/org/apache/accumulo/core/client/admin/InstanceOperations.java b/core/src/main/java/org/apache/accumulo/core/client/admin/InstanceOperations.java
index 45e7ae4..0346eee 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/admin/InstanceOperations.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/admin/InstanceOperations.java
@@ -102,7 +102,10 @@ public interface InstanceOperations {
       throws AccumuloException, AccumuloSecurityException;
 
   /**
-   * List the active compaction running on a tablet server
+   * List the active compaction running on a tablet server. Using this method with
+   * {@link #getTabletServers()} will only show compactions running on tservers, leaving out any
+   * external compactions running on compactors. Use {@link #getActiveCompactions()} to get a list
+   * of all compactions running on tservers and compactors.
    *
    * @param tserver
    *          The tablet server address should be of the form {@code <ip address>:<port>}
@@ -113,6 +116,14 @@ public interface InstanceOperations {
       throws AccumuloException, AccumuloSecurityException;
 
   /**
+   * List all internal and external compactions running in Accumulo.
+   *
+   * @return the list of active compactions
+   * @since 2.1.0
+   */
+  List<ActiveCompaction> getActiveCompactions() throws AccumuloException, AccumuloSecurityException;
+
+  /**
    * Throws an exception if a tablet server can not be contacted.
    *
    * @param tserver
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/ActiveCompactionImpl.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/ActiveCompactionImpl.java
index 380e91b..729034e 100644
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/ActiveCompactionImpl.java
+++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/ActiveCompactionImpl.java
@@ -25,10 +25,12 @@ import java.util.Map;
 import org.apache.accumulo.core.client.IteratorSetting;
 import org.apache.accumulo.core.client.TableNotFoundException;
 import org.apache.accumulo.core.client.admin.ActiveCompaction;
+import org.apache.accumulo.core.client.admin.ActiveCompaction.CompactionHost.Type;
 import org.apache.accumulo.core.data.TabletId;
 import org.apache.accumulo.core.dataImpl.KeyExtent;
 import org.apache.accumulo.core.dataImpl.TabletIdImpl;
 import org.apache.accumulo.core.dataImpl.thrift.IterInfo;
+import org.apache.accumulo.core.util.HostAndPort;
 
 /**
  * @since 1.6.0
@@ -37,11 +39,16 @@ public class ActiveCompactionImpl extends ActiveCompaction {
 
   private org.apache.accumulo.core.tabletserver.thrift.ActiveCompaction tac;
   private ClientContext context;
+  private HostAndPort hostport;
+  private Type type;
 
   ActiveCompactionImpl(ClientContext context,
-      org.apache.accumulo.core.tabletserver.thrift.ActiveCompaction tac) {
+      org.apache.accumulo.core.tabletserver.thrift.ActiveCompaction tac, HostAndPort hostport,
+      CompactionHost.Type type) {
     this.tac = tac;
     this.context = context;
+    this.hostport = hostport;
+    this.type = type;
   }
 
   @Override
@@ -109,4 +116,24 @@ public class ActiveCompactionImpl extends ActiveCompaction {
 
     return ret;
   }
+
+  @Override
+  public CompactionHost getHost() {
+    return new CompactionHost() {
+      @Override
+      public Type getType() {
+        return type;
+      }
+
+      @Override
+      public String getAddress() {
+        return hostport.getHost();
+      }
+
+      @Override
+      public int getPort() {
+        return hostport.getPort();
+      }
+    };
+  }
 }
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/InstanceOperationsImpl.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/InstanceOperationsImpl.java
index 1ec0005..ad81117 100644
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/InstanceOperationsImpl.java
+++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/InstanceOperationsImpl.java
@@ -20,6 +20,7 @@ package org.apache.accumulo.core.clientImpl;
 
 import static com.google.common.base.Preconditions.checkArgument;
 import static java.nio.charset.StandardCharsets.UTF_8;
+import static java.util.stream.Collectors.toList;
 import static org.apache.accumulo.core.rpc.ThriftUtil.createClient;
 import static org.apache.accumulo.core.rpc.ThriftUtil.createTransport;
 import static org.apache.accumulo.core.rpc.ThriftUtil.getTServerClient;
@@ -30,12 +31,16 @@ import java.util.Collections;
 import java.util.List;
 import java.util.Map;
 import java.util.UUID;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Future;
 
 import org.apache.accumulo.core.Constants;
 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.client.admin.ActiveCompaction;
+import org.apache.accumulo.core.client.admin.ActiveCompaction.CompactionHost;
 import org.apache.accumulo.core.client.admin.ActiveScan;
 import org.apache.accumulo.core.client.admin.InstanceOperations;
 import org.apache.accumulo.core.clientImpl.thrift.ConfigurationType;
@@ -47,6 +52,8 @@ import org.apache.accumulo.core.util.AddressUtil;
 import org.apache.accumulo.core.util.HostAndPort;
 import org.apache.accumulo.core.util.LocalityGroupUtil;
 import org.apache.accumulo.core.util.LocalityGroupUtil.LocalityGroupConfigurationError;
+import org.apache.accumulo.core.util.compaction.ExternalCompactionUtil;
+import org.apache.accumulo.core.util.threads.ThreadPools;
 import org.apache.accumulo.fate.zookeeper.ZooCache;
 import org.apache.thrift.TException;
 import org.apache.thrift.transport.TTransport;
@@ -191,7 +198,7 @@ public class InstanceOperationsImpl implements InstanceOperations {
 
       List<ActiveCompaction> as = new ArrayList<>();
       for (var tac : client.getActiveCompactions(TraceUtil.traceInfo(), context.rpcCreds())) {
-        as.add(new ActiveCompactionImpl(context, tac));
+        as.add(new ActiveCompactionImpl(context, tac, parsedTserver, CompactionHost.Type.TSERVER));
       }
       return as;
     } catch (ThriftSecurityException e) {
@@ -205,6 +212,53 @@ public class InstanceOperationsImpl implements InstanceOperations {
   }
 
   @Override
+  public List<ActiveCompaction> getActiveCompactions()
+      throws AccumuloException, AccumuloSecurityException {
+
+    List<HostAndPort> compactors = ExternalCompactionUtil.getCompactorAddrs(context);
+    List<String> tservers = getTabletServers();
+
+    int numThreads = Math.max(4, Math.min((tservers.size() + compactors.size()) / 10, 256));
+    var executorService =
+        ThreadPools.createFixedThreadPool(numThreads, "getactivecompactions", false);
+    try {
+      List<Future<List<ActiveCompaction>>> futures = new ArrayList<>();
+
+      for (String tserver : tservers) {
+        futures.add(executorService.submit(() -> getActiveCompactions(tserver)));
+      }
+
+      for (HostAndPort compactorAddr : compactors) {
+        Callable<List<ActiveCompaction>> task =
+            () -> ExternalCompactionUtil.getActiveCompaction(compactorAddr, context).stream()
+                .map(tac -> new ActiveCompactionImpl(context, tac, compactorAddr,
+                    CompactionHost.Type.COMPACTOR))
+                .collect(toList());
+
+        futures.add(executorService.submit(task));
+      }
+
+      List<ActiveCompaction> ret = new ArrayList<>();
+      for (Future<List<ActiveCompaction>> future : futures) {
+        try {
+          ret.addAll(future.get());
+        } catch (InterruptedException | ExecutionException e) {
+          if (e.getCause() instanceof ThriftSecurityException) {
+            ThriftSecurityException tse = (ThriftSecurityException) e.getCause();
+            throw new AccumuloSecurityException(tse.user, tse.code, e);
+          }
+          throw new AccumuloException(e);
+        }
+      }
+
+      return ret;
+
+    } finally {
+      executorService.shutdown();
+    }
+  }
+
+  @Override
   public void ping(String tserver) throws AccumuloException {
     try (
         TTransport transport = createTransport(AddressUtil.parseAddress(tserver, false), context)) {
diff --git a/core/src/main/java/org/apache/accumulo/core/conf/Property.java b/core/src/main/java/org/apache/accumulo/core/conf/Property.java
index 1b14a8c..8688608 100644
--- a/core/src/main/java/org/apache/accumulo/core/conf/Property.java
+++ b/core/src/main/java/org/apache/accumulo/core/conf/Property.java
@@ -442,7 +442,7 @@ public enum Property {
       "The maximum number of files a compaction will open"),
   TSERV_COMPACTION_SERVICE_ROOT_EXECUTORS(
       "tserver.compaction.major.service.root.planner.opts.executors",
-      ("[{'name':'small','maxSize':'32M','numThreads':1},{'name':'huge','numThreads':1}]")
+      ("[{'name':'small','type':'internal','maxSize':'32M','numThreads':1},{'name':'huge','type':'internal','numThreads':1}]")
           .replaceAll("'", "\""),
       PropertyType.STRING,
       "See {% jlink -f org.apache.accumulo.core.spi.compaction.DefaultCompactionPlanner %} "),
@@ -458,7 +458,7 @@ public enum Property {
       "The maximum number of files a compaction will open"),
   TSERV_COMPACTION_SERVICE_META_EXECUTORS(
       "tserver.compaction.major.service.meta.planner.opts.executors",
-      ("[{'name':'small','maxSize':'32M','numThreads':2},{'name':'huge','numThreads':2}]")
+      ("[{'name':'small','type':'internal','maxSize':'32M','numThreads':2},{'name':'huge','type':'internal','numThreads':2}]")
           .replaceAll("'", "\""),
       PropertyType.STRING,
       "See {% jlink -f org.apache.accumulo.core.spi.compaction.DefaultCompactionPlanner %} "),
@@ -474,7 +474,7 @@ public enum Property {
       "The maximum number of files a compaction will open"),
   TSERV_COMPACTION_SERVICE_DEFAULT_EXECUTORS(
       "tserver.compaction.major.service.default.planner.opts.executors",
-      ("[{'name':'small','maxSize':'32M','numThreads':2},{'name':'medium','maxSize':'128M','numThreads':2},{'name':'large','numThreads':2}]")
+      ("[{'name':'small','type':'internal','maxSize':'32M','numThreads':2},{'name':'medium','type':'internal','maxSize':'128M','numThreads':2},{'name':'large','type':'internal','numThreads':2}]")
           .replaceAll("'", "\""),
       PropertyType.STRING,
       "See {% jlink -f org.apache.accumulo.core.spi.compaction.DefaultCompactionPlanner %} "),
@@ -1021,6 +1021,49 @@ public enum Property {
   REPLICATION_RPC_TIMEOUT("replication.rpc.timeout", "2m", PropertyType.TIMEDURATION,
       "Amount of time for a single replication RPC call to last before failing"
           + " the attempt. See replication.work.attempts."),
+  // Compactor properties
+  COMPACTOR_PREFIX("compactor.", null, PropertyType.PREFIX,
+      "Properties in this category affect the behavior of the accumulo compactor server."),
+  COMPACTOR_PORTSEARCH("compactor.port.search", "true", PropertyType.BOOLEAN,
+      "If the compactor.port.client is in use, search higher ports until one is available"),
+  COMPACTOR_CLIENTPORT("compactor.port.client", "9101", PropertyType.PORT,
+      "The port used for handling client connections on the compactor servers"),
+  COMPACTOR_MINTHREADS("compactor.server.threads.minimum", "1", PropertyType.COUNT,
+      "The minimum number of threads to use to handle incoming requests."),
+  COMPACTOR_MINTHREADS_TIMEOUT("compactor.server.threads.timeout", "0s", PropertyType.TIMEDURATION,
+      "The time after which incoming request threads terminate with no work available.  Zero (0) will keep the threads alive indefinitely."),
+  COMPACTOR_THREADCHECK("compactor.server.threadcheck.time", "1s", PropertyType.TIMEDURATION,
+      "The time between adjustments of the server thread pool."),
+  COMPACTOR_MAX_MESSAGE_SIZE("compactor.server.message.size.max", "10M", PropertyType.BYTES,
+      "The maximum size of a message that can be sent to a tablet server."),
+  // CompactionCoordinator properties
+  COORDINATOR_PREFIX("coordinator.", null, PropertyType.PREFIX,
+      "Properties in this category affect the behavior of the accumulo compaction coordinator server."),
+  COORDINATOR_THRIFTCLIENT_PORTSEARCH("coordinator.thrift.port.search", "false",
+      PropertyType.BOOLEAN,
+      "If the ports above are in use, search higher ports until one is available"),
+  COORDINATOR_CLIENTPORT("coordinator.port.client", "9100", PropertyType.PORT,
+      "The port used for handling Thrift client connections on the compaction coordinator server"),
+  COORDINATOR_MINTHREADS("coordinator.server.threads.minimum", "1", PropertyType.COUNT,
+      "The minimum number of threads to use to handle incoming requests."),
+  COORDINATOR_MINTHREADS_TIMEOUT("coordinator.server.threads.timeout", "0s",
+      PropertyType.TIMEDURATION,
+      "The time after which incoming request threads terminate with no work available.  Zero (0) will keep the threads alive indefinitely."),
+  COORDINATOR_THREADCHECK("coordinator.server.threadcheck.time", "1s", PropertyType.TIMEDURATION,
+      "The time between adjustments of the server thread pool."),
+  COORDINATOR_MAX_MESSAGE_SIZE("coordinator.server.message.size.max", "10M", PropertyType.BYTES,
+      "The maximum size of a message that can be sent to a tablet server."),
+  COORDINATOR_DEAD_COMPACTOR_CHECK_INTERVAL("coordinator.server.compactor.dead.check.interval",
+      "5m", PropertyType.TIMEDURATION, "The interval at which to check for dead compactors."),
+  COORDINATOR_FINALIZER_TSERVER_NOTIFIER_MAXTHREADS("coordinator.server.finalizer.threads.maximum",
+      "5", PropertyType.COUNT,
+      "The maximum number of threads to use for notifying tablet servers that an external compaction has completed."),
+  COORDINATOR_FINALIZER_COMPLETION_CHECK_INTERVAL("coordinator.server.finalizer.check.interval",
+      "60s", PropertyType.TIMEDURATION,
+      "The interval at which to check for external compaction final state markers in the metadata table."),
+  COORDINATOR_TSERVER_COMPACTION_CHECK_INTERVAL(
+      "coordinator.server.tserver.compaction.check.interval", "1m", PropertyType.TIMEDURATION,
+      "The interval at which to check the tservers for external compactions."),
   // deprecated properties grouped at the end to reference property that replaces them
   @Deprecated(since = "1.6.0")
   @ReplacedBy(property = INSTANCE_VOLUMES)
diff --git a/core/src/main/java/org/apache/accumulo/core/iterators/user/LargeRowFilter.java b/core/src/main/java/org/apache/accumulo/core/iterators/user/LargeRowFilter.java
index 644b1cf..34f2baf 100644
--- a/core/src/main/java/org/apache/accumulo/core/iterators/user/LargeRowFilter.java
+++ b/core/src/main/java/org/apache/accumulo/core/iterators/user/LargeRowFilter.java
@@ -65,7 +65,7 @@ public class LargeRowFilter implements SortedKeyValueIterator<Key,Value>, Option
 
   private int maxColumns;
 
-  private boolean propogateSuppression = false;
+  private boolean propagateSuppression = false;
 
   private Range range;
   private Collection<ByteSequence> columnFamilies;
@@ -154,16 +154,16 @@ public class LargeRowFilter implements SortedKeyValueIterator<Key,Value>, Option
 
     bufferNextRow();
 
-    while (!propogateSuppression && currentPosition < keys.size()
+    while (!propagateSuppression && currentPosition < keys.size()
         && isSuppressionMarker(keys.get(0), values.get(0))) {
       bufferNextRow();
     }
   }
 
-  private LargeRowFilter(SortedKeyValueIterator<Key,Value> source, boolean propogateSuppression,
+  private LargeRowFilter(SortedKeyValueIterator<Key,Value> source, boolean propagateSuppression,
       int maxColumns) {
     this.source = source;
-    this.propogateSuppression = propogateSuppression;
+    this.propagateSuppression = propagateSuppression;
     this.maxColumns = maxColumns;
   }
 
@@ -174,7 +174,7 @@ public class LargeRowFilter implements SortedKeyValueIterator<Key,Value>, Option
       IteratorEnvironment env) throws IOException {
     this.source = source;
     this.maxColumns = Integer.parseInt(options.get(MAX_COLUMNS));
-    this.propogateSuppression = env.getIteratorScope() != IteratorScope.scan;
+    this.propagateSuppression = env.getIteratorScope() != IteratorScope.scan;
   }
 
   @Override
@@ -251,7 +251,7 @@ public class LargeRowFilter implements SortedKeyValueIterator<Key,Value>, Option
 
   @Override
   public SortedKeyValueIterator<Key,Value> deepCopy(IteratorEnvironment env) {
-    return new LargeRowFilter(source.deepCopy(env), propogateSuppression, maxColumns);
+    return new LargeRowFilter(source.deepCopy(env), propagateSuppression, maxColumns);
   }
 
   @Override
diff --git a/core/src/main/java/org/apache/accumulo/core/iterators/user/RowDeletingIterator.java b/core/src/main/java/org/apache/accumulo/core/iterators/user/RowDeletingIterator.java
index f45a482..c69f805 100644
--- a/core/src/main/java/org/apache/accumulo/core/iterators/user/RowDeletingIterator.java
+++ b/core/src/main/java/org/apache/accumulo/core/iterators/user/RowDeletingIterator.java
@@ -54,7 +54,7 @@ public class RowDeletingIterator implements SortedKeyValueIterator<Key,Value> {
 
   public static final Value DELETE_ROW_VALUE = new Value("DEL_ROW");
   private SortedKeyValueIterator<Key,Value> source;
-  private boolean propogateDeletes;
+  private boolean propagateDeletes;
   private ByteSequence currentRow;
   private boolean currentRowDeleted;
   private long deleteTS;
@@ -63,16 +63,16 @@ public class RowDeletingIterator implements SortedKeyValueIterator<Key,Value> {
 
   private static final ByteSequence EMPTY = new ArrayByteSequence(new byte[] {});
 
-  private RowDeletingIterator(SortedKeyValueIterator<Key,Value> source, boolean propogateDeletes2) {
+  private RowDeletingIterator(SortedKeyValueIterator<Key,Value> source, boolean propagateDeletes2) {
     this.source = source;
-    this.propogateDeletes = propogateDeletes2;
+    this.propagateDeletes = propagateDeletes2;
   }
 
   public RowDeletingIterator() {}
 
   @Override
   public SortedKeyValueIterator<Key,Value> deepCopy(IteratorEnvironment env) {
-    return new RowDeletingIterator(source.deepCopy(env), propogateDeletes);
+    return new RowDeletingIterator(source.deepCopy(env), propagateDeletes);
   }
 
   @Override
@@ -94,7 +94,7 @@ public class RowDeletingIterator implements SortedKeyValueIterator<Key,Value> {
   public void init(SortedKeyValueIterator<Key,Value> source, Map<String,String> options,
       IteratorEnvironment env) throws IOException {
     this.source = source;
-    this.propogateDeletes =
+    this.propagateDeletes =
         (env.getIteratorScope() == IteratorScope.majc && !env.isFullMajorCompaction())
             || env.getIteratorScope() == IteratorScope.minc;
   }
@@ -139,7 +139,7 @@ public class RowDeletingIterator implements SortedKeyValueIterator<Key,Value> {
         currentRowDeleted = true;
         deleteTS = source.getTopKey().getTimestamp();
 
-        if (propogateDeletes)
+        if (propagateDeletes)
           break;
       } else {
         break;
diff --git a/core/src/main/java/org/apache/accumulo/core/iteratorsImpl/system/DeletingIterator.java b/core/src/main/java/org/apache/accumulo/core/iteratorsImpl/system/DeletingIterator.java
index b744c5f..f29c061 100644
--- a/core/src/main/java/org/apache/accumulo/core/iteratorsImpl/system/DeletingIterator.java
+++ b/core/src/main/java/org/apache/accumulo/core/iteratorsImpl/system/DeletingIterator.java
@@ -35,7 +35,7 @@ import org.apache.accumulo.core.iterators.ServerWrappingIterator;
 import org.apache.accumulo.core.iterators.SortedKeyValueIterator;
 
 public class DeletingIterator extends ServerWrappingIterator {
-  private boolean propogateDeletes;
+  private boolean propagateDeletes;
   private Key workKey = new Key();
 
   public enum Behavior {
@@ -49,12 +49,12 @@ public class DeletingIterator extends ServerWrappingIterator {
 
   private DeletingIterator(DeletingIterator other, IteratorEnvironment env) {
     super(other.source.deepCopy(env));
-    propogateDeletes = other.propogateDeletes;
+    propagateDeletes = other.propagateDeletes;
   }
 
-  private DeletingIterator(SortedKeyValueIterator<Key,Value> iterator, boolean propogateDeletes) {
+  private DeletingIterator(SortedKeyValueIterator<Key,Value> iterator, boolean propagateDeletes) {
     super(iterator);
-    this.propogateDeletes = propogateDeletes;
+    this.propagateDeletes = propagateDeletes;
   }
 
   @Override
@@ -88,7 +88,7 @@ public class DeletingIterator extends ServerWrappingIterator {
   }
 
   private void findTop() throws IOException {
-    if (!propogateDeletes) {
+    if (!propagateDeletes) {
       while (source.hasTop() && source.getTopKey().isDeleted()) {
         skipRowColumn();
       }
@@ -114,10 +114,10 @@ public class DeletingIterator extends ServerWrappingIterator {
   }
 
   public static SortedKeyValueIterator<Key,Value> wrap(SortedKeyValueIterator<Key,Value> source,
-      boolean propogateDeletes, Behavior behavior) {
+      boolean propagateDeletes, Behavior behavior) {
     switch (behavior) {
       case PROCESS:
-        return new DeletingIterator(source, propogateDeletes);
+        return new DeletingIterator(source, propagateDeletes);
       case FAIL:
         return new ServerWrappingIterator(source) {
           @Override
diff --git a/core/src/main/java/org/apache/accumulo/core/metadata/schema/Ample.java b/core/src/main/java/org/apache/accumulo/core/metadata/schema/Ample.java
index 28c39a6..fd5634e 100644
--- a/core/src/main/java/org/apache/accumulo/core/metadata/schema/Ample.java
+++ b/core/src/main/java/org/apache/accumulo/core/metadata/schema/Ample.java
@@ -20,6 +20,7 @@ package org.apache.accumulo.core.metadata.schema;
 
 import java.util.Collection;
 import java.util.Iterator;
+import java.util.stream.Stream;
 
 import org.apache.accumulo.core.data.Mutation;
 import org.apache.accumulo.core.data.TableId;
@@ -201,6 +202,20 @@ public interface Ample {
     throw new UnsupportedOperationException();
   }
 
+  default void
+      putExternalCompactionFinalStates(Collection<ExternalCompactionFinalState> finalStates) {
+    throw new UnsupportedOperationException();
+  }
+
+  default Stream<ExternalCompactionFinalState> getExternalCompactionFinalStates() {
+    throw new UnsupportedOperationException();
+  }
+
+  default void
+      deleteExternalCompactionFinalStates(Collection<ExternalCompactionId> statusesToDelete) {
+    throw new UnsupportedOperationException();
+  }
+
   /**
    * Return an encoded delete marker Mutation to delete the specified TabletFile path. A String is
    * used for the parameter because the Garbage Collector is optimized to store a directory for
@@ -270,6 +285,11 @@ public interface Ample {
 
     TabletMutator deleteSuspension();
 
+    TabletMutator putExternalCompaction(ExternalCompactionId ecid,
+        ExternalCompactionMetadata ecMeta);
+
+    TabletMutator deleteExternalCompaction(ExternalCompactionId ecid);
+
     /**
      * This method persist (or queues for persisting) previous put and deletes against this object.
      * Unless this method is called, previous calls will never be persisted. The purpose of this
diff --git a/core/src/main/java/org/apache/accumulo/core/metadata/schema/AmpleImpl.java b/core/src/main/java/org/apache/accumulo/core/metadata/schema/AmpleImpl.java
index 863a6eb..b8cc498 100644
--- a/core/src/main/java/org/apache/accumulo/core/metadata/schema/AmpleImpl.java
+++ b/core/src/main/java/org/apache/accumulo/core/metadata/schema/AmpleImpl.java
@@ -18,6 +18,8 @@
  */
 package org.apache.accumulo.core.metadata.schema;
 
+import java.util.NoSuchElementException;
+
 import org.apache.accumulo.core.client.AccumuloClient;
 import org.apache.accumulo.core.dataImpl.KeyExtent;
 import org.apache.accumulo.core.metadata.schema.TabletMetadata.ColumnType;
@@ -43,6 +45,8 @@ public class AmpleImpl implements Ample {
 
     try (TabletsMetadata tablets = builder.build()) {
       return Iterables.getOnlyElement(tablets);
+    } catch (NoSuchElementException e) {
+      return null;
     }
   }
 
diff --git a/core/src/main/java/org/apache/accumulo/core/metadata/schema/ExternalCompactionFinalState.java b/core/src/main/java/org/apache/accumulo/core/metadata/schema/ExternalCompactionFinalState.java
new file mode 100644
index 0000000..4a01d24
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/metadata/schema/ExternalCompactionFinalState.java
@@ -0,0 +1,139 @@
+/*
+ * 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
+ *
+ *   http://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.schema;
+
+import java.util.Base64;
+
+import org.apache.accumulo.core.data.TableId;
+import org.apache.accumulo.core.dataImpl.KeyExtent;
+import org.apache.accumulo.core.util.TextUtil;
+import org.apache.hadoop.io.Text;
+
+import com.google.common.base.Preconditions;
+import com.google.gson.Gson;
+import com.google.gson.GsonBuilder;
+
+public class ExternalCompactionFinalState {
+
+  private static final Gson GSON = new GsonBuilder().create();
+
+  public enum FinalState {
+    FINISHED, FAILED
+  }
+
+  private final ExternalCompactionId ecid;
+  private final KeyExtent extent;
+  private final FinalState state;
+  private final long fileSize;
+  private final long fileEntries;
+
+  public ExternalCompactionFinalState(ExternalCompactionId ecid, KeyExtent extent, FinalState state,
+      long fileSize, long fileEntries) {
+    this.ecid = ecid;
+    this.extent = extent;
+    this.state = state;
+    this.fileSize = fileSize;
+    this.fileEntries = fileEntries;
+  }
+
+  public ExternalCompactionId getExternalCompactionId() {
+    return ecid;
+  }
+
+  public FinalState getFinalState() {
+    return state;
+  }
+
+  public KeyExtent getExtent() {
+    return extent;
+  }
+
+  public long getFileSize() {
+    Preconditions.checkState(state == FinalState.FINISHED);
+    return fileSize;
+  }
+
+  public long getEntries() {
+    Preconditions.checkState(state == FinalState.FINISHED);
+    return fileEntries;
+  }
+
+  // This class is used to serialize and deserialize this class using GSon. Any changes to this
+  // class must consider persisted data.
+  private static class Extent {
+
+    final String tableId;
+    final String er;
+    final String per;
+
+    Extent(KeyExtent extent) {
+      this.tableId = extent.tableId().canonical();
+      if (extent.endRow() != null) {
+        er = Base64.getEncoder().encodeToString(TextUtil.getBytes(extent.endRow()));
+      } else {
+        er = null;
+      }
+
+      if (extent.prevEndRow() != null) {
+        per = Base64.getEncoder().encodeToString(TextUtil.getBytes(extent.prevEndRow()));
+      } else {
+        per = null;
+      }
+    }
+
+    private Text decode(String s) {
+      if (s == null)
+        return null;
+      return new Text(Base64.getDecoder().decode(s));
+    }
+
+    KeyExtent toKeyExtent() {
+      return new KeyExtent(TableId.of(tableId), decode(er), decode(per));
+    }
+  }
+
+  // This class is used to serialize and deserialize this class using GSon. Any changes to this
+  // class must consider persisted data.
+  private static class JsonData {
+    Extent extent;
+    String state;
+    long fileSize;
+    long entries;
+  }
+
+  public String toJson() {
+    JsonData jd = new JsonData();
+    jd.state = state.name();
+    jd.fileSize = fileSize;
+    jd.entries = fileEntries;
+    jd.extent = new Extent(extent);
+    return GSON.toJson(jd);
+  }
+
+  public static ExternalCompactionFinalState fromJson(ExternalCompactionId ecid, String json) {
+    JsonData jd = GSON.fromJson(json, JsonData.class);
+    return new ExternalCompactionFinalState(ecid, jd.extent.toKeyExtent(),
+        FinalState.valueOf(jd.state), jd.fileSize, jd.entries);
+  }
+
+  @Override
+  public String toString() {
+    return toJson();
+  }
+}
diff --git a/core/src/main/java/org/apache/accumulo/core/metadata/schema/ExternalCompactionId.java b/core/src/main/java/org/apache/accumulo/core/metadata/schema/ExternalCompactionId.java
new file mode 100644
index 0000000..8050422
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/metadata/schema/ExternalCompactionId.java
@@ -0,0 +1,59 @@
+/*
+ * 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
+ *
+ *   http://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.schema;
+
+import java.util.UUID;
+
+import org.apache.accumulo.core.data.AbstractId;
+
+public class ExternalCompactionId extends AbstractId<ExternalCompactionId> {
+
+  // A common prefix is nice when grepping logs for external compaction ids. The prefix also serves
+  // as a nice sanity check on data coming in over the network and from persistent storage.
+  private static final String PREFIX = "ECID:";
+
+  private ExternalCompactionId(UUID uuid) {
+    super(PREFIX + uuid);
+  }
+
+  private ExternalCompactionId(String id) {
+    super(id);
+  }
+
+  private static final long serialVersionUID = 1L;
+
+  public static ExternalCompactionId generate(UUID uuid) {
+    return new ExternalCompactionId(uuid);
+  }
+
+  public static ExternalCompactionId of(String id) {
+    if (!id.startsWith(PREFIX)) {
+      throw new IllegalArgumentException("Not a valid external compaction id " + id);
+    }
+
+    try {
+      UUID.fromString(id.substring(PREFIX.length()));
+    } catch (IllegalArgumentException e) {
+      throw new IllegalArgumentException("Not a valid external compaction id " + id, e);
+    }
+
+    return new ExternalCompactionId(id);
+  }
+
+}
diff --git a/core/src/main/java/org/apache/accumulo/core/metadata/schema/ExternalCompactionMetadata.java b/core/src/main/java/org/apache/accumulo/core/metadata/schema/ExternalCompactionMetadata.java
new file mode 100644
index 0000000..df2d98c
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/metadata/schema/ExternalCompactionMetadata.java
@@ -0,0 +1,165 @@
+/*
+ * 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
+ *
+ *   http://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.schema;
+
+import static java.util.stream.Collectors.toList;
+import static java.util.stream.Collectors.toSet;
+
+import java.util.List;
+import java.util.Objects;
+import java.util.Set;
+
+import org.apache.accumulo.core.metadata.StoredTabletFile;
+import org.apache.accumulo.core.metadata.TabletFile;
+import org.apache.accumulo.core.spi.compaction.CompactionExecutorId;
+import org.apache.accumulo.core.spi.compaction.CompactionKind;
+import org.apache.accumulo.core.util.compaction.CompactionExecutorIdImpl;
+import org.apache.hadoop.fs.Path;
+
+import com.google.gson.Gson;
+import com.google.gson.GsonBuilder;
+
+public class ExternalCompactionMetadata {
+
+  private static final Gson GSON = new GsonBuilder().create();
+
+  private final Set<StoredTabletFile> jobFiles;
+  private final Set<StoredTabletFile> nextFiles;
+  private final TabletFile compactTmpName;
+  private final TabletFile newFile;
+  private final String compactorId;
+  private final CompactionKind kind;
+  private final long priority;
+  private final CompactionExecutorId ceid;
+  private final boolean propagateDeletes;
+  private final boolean initiallySelectedAll;
+  private final Long compactionId;
+
+  public ExternalCompactionMetadata(Set<StoredTabletFile> jobFiles, Set<StoredTabletFile> nextFiles,
+      TabletFile compactTmpName, TabletFile newFile, String compactorId, CompactionKind kind,
+      long priority, CompactionExecutorId ceid, boolean propagateDeletes,
+      boolean initiallySelectedAll, Long compactionId) {
+    this.jobFiles = Objects.requireNonNull(jobFiles);
+    this.nextFiles = Objects.requireNonNull(nextFiles);
+    this.compactTmpName = Objects.requireNonNull(compactTmpName);
+    this.newFile = Objects.requireNonNull(newFile);
+    this.compactorId = Objects.requireNonNull(compactorId);
+    this.kind = Objects.requireNonNull(kind);
+    this.priority = priority;
+    this.ceid = Objects.requireNonNull(ceid);
+    this.propagateDeletes = propagateDeletes;
+    this.initiallySelectedAll = initiallySelectedAll;
+    this.compactionId = compactionId;
+  }
+
+  public Set<StoredTabletFile> getJobFiles() {
+    return jobFiles;
+  }
+
+  public Set<StoredTabletFile> getNextFiles() {
+    return nextFiles;
+  }
+
+  public TabletFile getCompactTmpName() {
+    return compactTmpName;
+  }
+
+  public TabletFile getNewFile() {
+    return newFile;
+  }
+
+  public String getCompactorId() {
+    return compactorId;
+  }
+
+  public CompactionKind getKind() {
+    return kind;
+  }
+
+  public long getPriority() {
+    return priority;
+  }
+
+  public CompactionExecutorId getCompactionExecutorId() {
+    return ceid;
+  }
+
+  public boolean getPropogateDeletes() {
+    return propagateDeletes;
+  }
+
+  public boolean getInitiallySelecteAll() {
+    return initiallySelectedAll;
+  }
+
+  public Long getCompactionId() {
+    return compactionId;
+  }
+
+  // This class is used to serialize and deserialize this class using GSon. Any changes to this
+  // class must consider persisted data.
+  private static class GSonData {
+    List<String> inputs;
+    List<String> nextFiles;
+    String tmp;
+    String dest;
+    String compactor;
+    String kind;
+    String executorId;
+    long priority;
+    boolean propDels;
+    boolean selectedAll;
+    Long compactionId;
+  }
+
+  public String toJson() {
+    GSonData jData = new GSonData();
+
+    jData.inputs = jobFiles.stream().map(StoredTabletFile::getMetaUpdateDelete).collect(toList());
+    jData.nextFiles =
+        nextFiles.stream().map(StoredTabletFile::getMetaUpdateDelete).collect(toList());
+    jData.tmp = compactTmpName.getMetaInsert();
+    jData.dest = newFile.getMetaInsert();
+    jData.compactor = compactorId;
+    jData.kind = kind.name();
+    jData.executorId = ((CompactionExecutorIdImpl) ceid).getExternalName();
+    jData.priority = priority;
+    jData.propDels = propagateDeletes;
+    jData.selectedAll = initiallySelectedAll;
+    jData.compactionId = compactionId;
+    return GSON.toJson(jData);
+  }
+
+  public static ExternalCompactionMetadata fromJson(String json) {
+    GSonData jData = GSON.fromJson(json, GSonData.class);
+
+    return new ExternalCompactionMetadata(
+        jData.inputs.stream().map(StoredTabletFile::new).collect(toSet()),
+        jData.nextFiles.stream().map(StoredTabletFile::new).collect(toSet()),
+        new TabletFile(new Path(jData.tmp)), new TabletFile(new Path(jData.dest)), jData.compactor,
+        CompactionKind.valueOf(jData.kind), jData.priority,
+        CompactionExecutorIdImpl.externalId(jData.executorId), jData.propDels, jData.selectedAll,
+        jData.compactionId);
+  }
+
+  @Override
+  public String toString() {
+    return toJson();
+  }
+}
diff --git a/core/src/main/java/org/apache/accumulo/core/metadata/schema/MetadataSchema.java b/core/src/main/java/org/apache/accumulo/core/metadata/schema/MetadataSchema.java
index 795ea03..5f5abde 100644
--- a/core/src/main/java/org/apache/accumulo/core/metadata/schema/MetadataSchema.java
+++ b/core/src/main/java/org/apache/accumulo/core/metadata/schema/MetadataSchema.java
@@ -327,6 +327,11 @@ public class MetadataSchema {
       public static final Text NAME = new Text("chopped");
       public static final ColumnFQ CHOPPED_COLUMN = new ColumnFQ(NAME, new Text("chopped"));
     }
+
+    public static class ExternalCompactionColumnFamily {
+      public static final String STR_NAME = "ecomp";
+      public static final Text NAME = new Text(STR_NAME);
+    }
   }
 
   /**
@@ -449,4 +454,16 @@ public class MetadataSchema {
     }
   }
 
+  public static class ExternalCompactionSection {
+    private static final Section section =
+        new Section(RESERVED_PREFIX + "ecomp", true, RESERVED_PREFIX + "ecomq", false);
+
+    public static Range getRange() {
+      return section.getRange();
+    }
+
+    public static String getRowPrefix() {
+      return section.getRowPrefix();
+    }
+  }
 }
diff --git a/core/src/main/java/org/apache/accumulo/core/metadata/schema/TabletMetadata.java b/core/src/main/java/org/apache/accumulo/core/metadata/schema/TabletMetadata.java
index a7554fd..741288d 100644
--- a/core/src/main/java/org/apache/accumulo/core/metadata/schema/TabletMetadata.java
+++ b/core/src/main/java/org/apache/accumulo/core/metadata/schema/TabletMetadata.java
@@ -61,6 +61,7 @@ import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.Bu
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.ClonedColumnFamily;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.CurrentLocationColumnFamily;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.DataFileColumnFamily;
+import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.ExternalCompactionColumnFamily;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.FutureLocationColumnFamily;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.LastLocationColumnFamily;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.LogColumnFamily;
@@ -109,6 +110,7 @@ public class TabletMetadata {
   private List<LogEntry> logs;
   private OptionalLong compact = OptionalLong.empty();
   private Double splitRatio = null;
+  private Map<ExternalCompactionId,ExternalCompactionMetadata> extCompactions;
 
   public enum LocationType {
     CURRENT, FUTURE, LAST
@@ -129,7 +131,8 @@ public class TabletMetadata {
     LOGS,
     COMPACT_ID,
     SPLIT_RATIO,
-    SUSPEND
+    SUSPEND,
+    ECOMP
   }
 
   public static class Location extends TServerInstance {
@@ -292,6 +295,11 @@ public class TabletMetadata {
     }
   }
 
+  public Map<ExternalCompactionId,ExternalCompactionMetadata> getExternalCompactions() {
+    ensureFetched(ColumnType.ECOMP);
+    return extCompactions;
+  }
+
   @VisibleForTesting
   public static TabletMetadata convertRow(Iterator<Entry<Key,Value>> rowIter,
       EnumSet<ColumnType> fetchedColumns, boolean buildKeyValueMap) {
@@ -304,6 +312,8 @@ public class TabletMetadata {
     final var filesBuilder = ImmutableMap.<StoredTabletFile,DataFileValue>builder();
     final var scansBuilder = ImmutableList.<StoredTabletFile>builder();
     final var logsBuilder = ImmutableList.<LogEntry>builder();
+    final var extCompBuilder =
+        ImmutableMap.<ExternalCompactionId,ExternalCompactionMetadata>builder();
     final var loadedFilesBuilder = ImmutableMap.<TabletFile,Long>builder();
     ByteSequence row = null;
 
@@ -390,6 +400,10 @@ public class TabletMetadata {
         case LogColumnFamily.STR_NAME:
           logsBuilder.add(LogEntry.fromMetaWalEntry(kv));
           break;
+        case ExternalCompactionColumnFamily.STR_NAME:
+          extCompBuilder.put(ExternalCompactionId.of(qual),
+              ExternalCompactionMetadata.fromJson(val));
+          break;
         default:
           throw new IllegalStateException("Unexpected family " + fam);
       }
@@ -400,6 +414,7 @@ public class TabletMetadata {
     te.fetchedCols = fetchedColumns;
     te.scans = scansBuilder.build();
     te.logs = logsBuilder.build();
+    te.extCompactions = extCompBuilder.build();
     if (buildKeyValueMap) {
       te.keyValues = kvBuilder.build();
     }
diff --git a/core/src/main/java/org/apache/accumulo/core/metadata/schema/TabletsMetadata.java b/core/src/main/java/org/apache/accumulo/core/metadata/schema/TabletsMetadata.java
index f67f81a..1d6fc89 100644
--- a/core/src/main/java/org/apache/accumulo/core/metadata/schema/TabletsMetadata.java
+++ b/core/src/main/java/org/apache/accumulo/core/metadata/schema/TabletsMetadata.java
@@ -51,6 +51,7 @@ import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.Bu
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.ClonedColumnFamily;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.CurrentLocationColumnFamily;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.DataFileColumnFamily;
+import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.ExternalCompactionColumnFamily;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.FutureLocationColumnFamily;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.LastLocationColumnFamily;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.LogColumnFamily;
@@ -198,6 +199,9 @@ public class TabletsMetadata implements Iterable<TabletMetadata>, AutoCloseable
           case TIME:
             qualifiers.add(TIME_COLUMN);
             break;
+          case ECOMP:
+            families.add(ExternalCompactionColumnFamily.NAME);
+            break;
           default:
             throw new IllegalArgumentException("Unknown col type " + colToFetch);
 
diff --git a/core/src/main/java/org/apache/accumulo/core/spi/compaction/CompactionExecutorId.java b/core/src/main/java/org/apache/accumulo/core/spi/compaction/CompactionExecutorId.java
index 91e8bb0..0707392 100644
--- a/core/src/main/java/org/apache/accumulo/core/spi/compaction/CompactionExecutorId.java
+++ b/core/src/main/java/org/apache/accumulo/core/spi/compaction/CompactionExecutorId.java
@@ -30,11 +30,7 @@ import org.apache.accumulo.core.data.AbstractId;
 public class CompactionExecutorId extends AbstractId<CompactionExecutorId> {
   private static final long serialVersionUID = 1L;
 
-  private CompactionExecutorId(String canonical) {
+  protected CompactionExecutorId(String canonical) {
     super(canonical);
   }
-
-  public static CompactionExecutorId of(String canonical) {
-    return new CompactionExecutorId(canonical);
-  }
 }
diff --git a/core/src/main/java/org/apache/accumulo/core/spi/compaction/DefaultCompactionPlanner.java b/core/src/main/java/org/apache/accumulo/core/spi/compaction/DefaultCompactionPlanner.java
index fe95bcf..def506a 100644
--- a/core/src/main/java/org/apache/accumulo/core/spi/compaction/DefaultCompactionPlanner.java
+++ b/core/src/main/java/org/apache/accumulo/core/spi/compaction/DefaultCompactionPlanner.java
@@ -51,17 +51,58 @@ import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
  *
  * <ul>
  * <li>{@code tserver.compaction.major.service.<service>.opts.executors} This is a json array of
- * objects where each object has the fields: name, maxSize, and numThreads. The maxSize field
- * determine the maximum size of compaction that will run on an executor. The maxSize field can have
- * a suffix of K,M,G for kilobytes, megabytes, or gigabytes. One executor can have no max size and
- * it will run everything that is too large for the other executors. If all executors have a max
- * size, then system compactions will only run for compactions smaller than the largest max size.
- * User, chop, and selector compactions will always run, even if there is no executor for their
- * size. These compaction will run on the executor with the largest max size. The value for this
- * field should look like
- * {@code [{"name":"executor1","maxSize":"100M","numThreads":3},{"name":"executor2","maxSize":"500M","numThreads":3},{"executor3":"huge","numThreads":3}]}.
- * This configuration would run compactions less than 100M on executor1, compactions less than 500M
- * on executor2, and all other on executor3.
+ * objects where each object has the fields:
+ * <table>
+ * <caption>Default Compaction Planner Executor options</caption>
+ * <tr>
+ * <th>Field Name</th>
+ * <th>Description</th>
+ * </tr>
+ * <tr>
+ * <td>name</td>
+ * <td>name or alias of the executor (required)</td>
+ * </tr>
+ * <tr>
+ * <td>type</td>
+ * <td>valid values 'internal' or 'external' (required)</td>
+ * </tr>
+ * <tr>
+ * <td>maxSize</td>
+ * <td>threshold sum of the input files (required for all but one of the configs)</td>
+ * </tr>
+ * <tr>
+ * <td>numThreads</td>
+ * <td>number of threads for this executor configuration (required for 'internal', cannot be
+ * specified for 'external')</td>
+ * </tr>
+ * <tr>
+ * <td>queue</td>
+ * <td>name of the external compaction queue (required for 'external', cannot be specified for
+ * 'internal')</td>
+ * </tr>
+ * </table>
+ * <br>
+ * The maxSize field determines the maximum size of compaction that will run on an executor. The
+ * maxSize field can have a suffix of K,M,G for kilobytes, megabytes, or gigabytes and represents
+ * the sum of the input files for a given compaction. One executor can have no max size and it will
+ * run everything that is too large for the other executors. If all executors have a max size, then
+ * system compactions will only run for compactions smaller than the largest max size. User, chop,
+ * and selector compactions will always run, even if there is no executor for their size. These
+ * compactions will run on the executor with the largest max size. The following example value for
+ * this property will create 3 threads to run compactions of files whose file size sum is less than
+ * 100M, 3 threads to run compactions of files whose file size sum is less than 500M, and run all
+ * other compactions on Compactors configured to run compactions for Queue1:
+ *
+ * <pre>
+ * {@code
+ * [{"name":"small", "type": "internal", "maxSize":"100M","numThreads":3},
+ *  {"name":"medium", "type": "internal", "maxSize":"500M","numThreads":3},
+ *  {"name: "large", "type": "external", "queue", "Queue1"}
+ * ]}
+ * </pre>
+ *
+ * Note that the use of 'external' requires that the CompactionCoordinator and at least one
+ * Compactor for Queue1 is running.
  * <li>{@code tserver.compaction.major.service.<service>.opts.maxOpen} This determines the maximum
  * number of files that will be included in a single compaction.
  * </ul>
@@ -75,9 +116,11 @@ public class DefaultCompactionPlanner implements CompactionPlanner {
   private static Logger log = LoggerFactory.getLogger(DefaultCompactionPlanner.class);
 
   public static class ExecutorConfig {
+    String type;
     String name;
     String maxSize;
-    int numThreads;
+    Integer numThreads;
+    String queue;
   }
 
   private static class Executor {
@@ -103,7 +146,8 @@ public class DefaultCompactionPlanner implements CompactionPlanner {
   private List<Executor> executors;
   private int maxFilesToCompact;
 
-  @SuppressFBWarnings(value = "UWF_UNWRITTEN_FIELD", justification = "Field is written by Gson")
+  @SuppressFBWarnings(value = {"UWF_UNWRITTEN_FIELD", "NP_UNWRITTEN_FIELD"},
+      justification = "Field is written by Gson")
   @Override
   public void init(InitParameters params) {
     ExecutorConfig[] execConfigs =
@@ -112,10 +156,32 @@ public class DefaultCompactionPlanner implements CompactionPlanner {
     List<Executor> tmpExec = new ArrayList<>();
 
     for (ExecutorConfig executorConfig : execConfigs) {
-      var ceid = params.getExecutorManager().createExecutor(executorConfig.name,
-          executorConfig.numThreads);
       Long maxSize = executorConfig.maxSize == null ? null
           : ConfigurationTypeHelper.getFixedMemoryAsBytes(executorConfig.maxSize);
+
+      CompactionExecutorId ceid;
+
+      Objects.requireNonNull(executorConfig.type,
+          "'type' is a required and must be 'internal' or 'external'");
+      switch (executorConfig.type) {
+        case "internal":
+          Preconditions.checkArgument(null == executorConfig.queue,
+              "'queue' should not be specified for internal compactions");
+          Objects.requireNonNull(executorConfig.numThreads,
+              "'numThreads' must be specified for internal type");
+          ceid = params.getExecutorManager().createExecutor(executorConfig.name,
+              executorConfig.numThreads);
+          break;
+        case "external":
+          Preconditions.checkArgument(null == executorConfig.numThreads,
+              "'numThreads' should not be specified for external compactions");
+          Objects.requireNonNull(executorConfig.queue,
+              "'queue' must be specified for external type");
+          ceid = params.getExecutorManager().getExternalExecutor(executorConfig.queue);
+          break;
+        default:
+          throw new IllegalArgumentException("type must be 'internal' or 'external'");
+      }
       tmpExec.add(new Executor(ceid, maxSize));
     }
 
diff --git a/core/src/main/java/org/apache/accumulo/core/spi/compaction/ExecutorManager.java b/core/src/main/java/org/apache/accumulo/core/spi/compaction/ExecutorManager.java
index 4740ee6..3e39e55 100644
--- a/core/src/main/java/org/apache/accumulo/core/spi/compaction/ExecutorManager.java
+++ b/core/src/main/java/org/apache/accumulo/core/spi/compaction/ExecutorManager.java
@@ -31,4 +31,9 @@ public interface ExecutorManager {
    * Create a thread pool executor within a compaction service.
    */
   public CompactionExecutorId createExecutor(String name, int threads);
+
+  /**
+   * @return an id for a configured external execution queue.
+   */
+  public CompactionExecutorId getExternalExecutor(String name);
 }
diff --git a/core/src/main/java/org/apache/accumulo/core/util/ServerServices.java b/core/src/main/java/org/apache/accumulo/core/util/ServerServices.java
index 4874665..a25a75e 100644
--- a/core/src/main/java/org/apache/accumulo/core/util/ServerServices.java
+++ b/core/src/main/java/org/apache/accumulo/core/util/ServerServices.java
@@ -22,7 +22,7 @@ import java.util.EnumMap;
 
 public class ServerServices implements Comparable<ServerServices> {
   public static enum Service {
-    TSERV_CLIENT, GC_CLIENT
+    TSERV_CLIENT, GC_CLIENT, COMPACTOR_CLIENT
   }
 
   public static final String SERVICE_SEPARATOR = ";";
diff --git a/core/src/main/java/org/apache/accumulo/core/spi/compaction/CompactionExecutorId.java b/core/src/main/java/org/apache/accumulo/core/util/compaction/CompactionExecutorIdImpl.java
similarity index 50%
copy from core/src/main/java/org/apache/accumulo/core/spi/compaction/CompactionExecutorId.java
copy to core/src/main/java/org/apache/accumulo/core/util/compaction/CompactionExecutorIdImpl.java
index 91e8bb0..2dcb44c 100644
--- a/core/src/main/java/org/apache/accumulo/core/spi/compaction/CompactionExecutorId.java
+++ b/core/src/main/java/org/apache/accumulo/core/util/compaction/CompactionExecutorIdImpl.java
@@ -16,25 +16,36 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.accumulo.core.spi.compaction;
+package org.apache.accumulo.core.util.compaction;
 
-import org.apache.accumulo.core.data.AbstractId;
+import org.apache.accumulo.core.spi.compaction.CompactionExecutorId;
+import org.apache.accumulo.core.spi.compaction.CompactionServiceId;
 
-/**
- * A unique identifier for a a compaction executor that a {@link CompactionPlanner} can schedule
- * compactions on using a {@link CompactionJob}.
- *
- * @since 2.1.0
- * @see org.apache.accumulo.core.spi.compaction
- */
-public class CompactionExecutorId extends AbstractId<CompactionExecutorId> {
-  private static final long serialVersionUID = 1L;
+import com.google.common.base.Preconditions;
+
+public class CompactionExecutorIdImpl extends CompactionExecutorId {
 
-  private CompactionExecutorId(String canonical) {
+  protected CompactionExecutorIdImpl(String canonical) {
     super(canonical);
   }
 
-  public static CompactionExecutorId of(String canonical) {
-    return new CompactionExecutorId(canonical);
+  private static final long serialVersionUID = 1L;
+
+  public boolean isExternalId() {
+    return canonical().startsWith("e.");
   }
+
+  public String getExternalName() {
+    Preconditions.checkState(isExternalId());
+    return canonical().substring("e.".length());
+  }
+
+  public static CompactionExecutorId internalId(CompactionServiceId csid, String executorName) {
+    return new CompactionExecutorIdImpl("i." + csid + "." + executorName);
+  }
+
+  public static CompactionExecutorId externalId(String executorName) {
+    return new CompactionExecutorIdImpl("e." + executorName);
+  }
+
 }
diff --git a/core/src/main/java/org/apache/accumulo/core/util/compaction/CompactionJobImpl.java b/core/src/main/java/org/apache/accumulo/core/util/compaction/CompactionJobImpl.java
index 62ba955..9fbbe6a 100644
--- a/core/src/main/java/org/apache/accumulo/core/util/compaction/CompactionJobImpl.java
+++ b/core/src/main/java/org/apache/accumulo/core/util/compaction/CompactionJobImpl.java
@@ -20,6 +20,7 @@ package org.apache.accumulo.core.util.compaction;
 
 import java.util.Collection;
 import java.util.Objects;
+import java.util.Optional;
 import java.util.Set;
 
 import org.apache.accumulo.core.client.admin.compaction.CompactableFile;
@@ -39,15 +40,23 @@ public class CompactionJobImpl implements CompactionJob {
   private final CompactionExecutorId executor;
   private final Set<CompactableFile> files;
   private final CompactionKind kind;
-  private boolean selectedAll;
+  // Tracks if a job selected all of the tablets files that existed at the time the job was created.
+  private final Optional<Boolean> jobSelectedAll;
 
-  CompactionJobImpl(long priority, CompactionExecutorId executor, Collection<CompactableFile> files,
-      CompactionKind kind, boolean selectedAllFiles) {
+  /**
+   *
+   * @param jobSelectedAll
+   *          This parameters only needs to be non-empty for job objects that are used to start
+   *          compaction. After a job is running, its not used. So when a job object is recreated
+   *          for a running external compaction this parameter can be empty.
+   */
+  public CompactionJobImpl(long priority, CompactionExecutorId executor,
+      Collection<CompactableFile> files, CompactionKind kind, Optional<Boolean> jobSelectedAll) {
     this.priority = priority;
     this.executor = Objects.requireNonNull(executor);
     this.files = Set.copyOf(files);
-    this.kind = kind;
-    this.selectedAll = selectedAllFiles;
+    this.kind = Objects.requireNonNull(kind);
+    this.jobSelectedAll = Objects.requireNonNull(jobSelectedAll);
   }
 
   @Override
@@ -85,7 +94,7 @@ public class CompactionJobImpl implements CompactionJob {
   }
 
   public boolean selectedAll() {
-    return selectedAll;
+    return jobSelectedAll.get();
   }
 
   @Override
diff --git a/core/src/main/java/org/apache/accumulo/core/util/compaction/CompactionPlanImpl.java b/core/src/main/java/org/apache/accumulo/core/util/compaction/CompactionPlanImpl.java
index 4b678d9..ee96648 100644
--- a/core/src/main/java/org/apache/accumulo/core/util/compaction/CompactionPlanImpl.java
+++ b/core/src/main/java/org/apache/accumulo/core/util/compaction/CompactionPlanImpl.java
@@ -23,6 +23,7 @@ import java.util.Collection;
 import java.util.Collections;
 import java.util.HashSet;
 import java.util.List;
+import java.util.Optional;
 import java.util.Set;
 
 import org.apache.accumulo.core.client.admin.compaction.CompactableFile;
@@ -79,8 +80,8 @@ public class CompactionPlanImpl implements CompactionPlan {
 
       seenFiles.addAll(filesSet);
 
-      jobs.add(
-          new CompactionJobImpl(priority, executor, filesSet, kind, filesSet.equals(allFiles)));
+      jobs.add(new CompactionJobImpl(priority, executor, filesSet, kind,
+          Optional.of(filesSet.equals(allFiles))));
       return this;
     }
 
diff --git a/core/src/main/java/org/apache/accumulo/core/util/compaction/ExternalCompactionUtil.java b/core/src/main/java/org/apache/accumulo/core/util/compaction/ExternalCompactionUtil.java
new file mode 100644
index 0000000..2e42021
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/util/compaction/ExternalCompactionUtil.java
@@ -0,0 +1,270 @@
+/*
+ * 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
+ *
+ *   http://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.util.compaction;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
+
+import org.apache.accumulo.core.Constants;
+import org.apache.accumulo.core.clientImpl.ClientContext;
+import org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException;
+import org.apache.accumulo.core.compaction.thrift.CompactorService;
+import org.apache.accumulo.core.metadata.schema.ExternalCompactionId;
+import org.apache.accumulo.core.rpc.ThriftUtil;
+import org.apache.accumulo.core.tabletserver.thrift.ActiveCompaction;
+import org.apache.accumulo.core.tabletserver.thrift.TExternalCompactionJob;
+import org.apache.accumulo.core.trace.TraceUtil;
+import org.apache.accumulo.core.util.HostAndPort;
+import org.apache.accumulo.core.util.Pair;
+import org.apache.accumulo.core.util.threads.ThreadPools;
+import org.apache.accumulo.fate.zookeeper.ServiceLock;
+import org.apache.accumulo.fate.zookeeper.ZooReader;
+import org.apache.accumulo.fate.zookeeper.ZooSession;
+import org.apache.thrift.TException;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.KeeperException.NoNodeException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class ExternalCompactionUtil {
+
+  private static final Logger LOG = LoggerFactory.getLogger(ExternalCompactionUtil.class);
+
+  /**
+   * Utility for returning the address of a service in the form host:port
+   *
+   * @param address
+   *          HostAndPort of service
+   * @return host and port
+   */
+  public static String getHostPortString(HostAndPort address) {
+    if (address == null) {
+      return null;
+    }
+    return address.toString();
+  }
+
+  /**
+   *
+   * @return null if Coordinator node not found, else HostAndPort
+   */
+  public static HostAndPort findCompactionCoordinator(ClientContext context) {
+    final String lockPath = context.getZooKeeperRoot() + Constants.ZCOORDINATOR_LOCK;
+    try {
+      var zk = ZooSession.getAnonymousSession(context.getZooKeepers(),
+          context.getZooKeepersSessionTimeOut());
+      byte[] address = ServiceLock.getLockData(zk, ServiceLock.path(lockPath));
+      if (null == address) {
+        return null;
+      }
+      String coordinatorAddress = new String(address);
+      return HostAndPort.fromString(coordinatorAddress);
+    } catch (KeeperException | InterruptedException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  /**
+   * @return list of Compactors
+   */
+  public static List<HostAndPort> getCompactorAddrs(ClientContext context) {
+    try {
+      final List<HostAndPort> compactAddrs = new ArrayList<>();
+      final String compactorQueuesPath = context.getZooKeeperRoot() + Constants.ZCOMPACTORS;
+      ZooReader zooReader =
+          new ZooReader(context.getZooKeepers(), context.getZooKeepersSessionTimeOut());
+      List<String> queues = zooReader.getChildren(compactorQueuesPath);
+      for (String queue : queues) {
+        try {
+          List<String> compactors = zooReader.getChildren(compactorQueuesPath + "/" + queue);
+          for (String compactor : compactors) {
+            // compactor is the address, we are checking to see if there is a child node which
+            // represents the compactor's lock as a check that it's alive.
+            List<String> children =
+                zooReader.getChildren(compactorQueuesPath + "/" + queue + "/" + compactor);
+            if (!children.isEmpty()) {
+              LOG.trace("Found live compactor {} ", compactor);
+              compactAddrs.add(HostAndPort.fromString(compactor));
+            }
+          }
+        } catch (NoNodeException e) {
+          LOG.trace("Ignoring node that went missing", e);
+        }
+      }
+
+      return compactAddrs;
+    } catch (KeeperException e) {
+      throw new RuntimeException(e);
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      throw new RuntimeException(e);
+    }
+  }
+
+  public static List<ActiveCompaction> getActiveCompaction(HostAndPort compactor,
+      ClientContext context) throws ThriftSecurityException {
+    CompactorService.Client client = null;
+    try {
+      client = ThriftUtil.getClient(new CompactorService.Client.Factory(), compactor, context);
+      return client.getActiveCompactions(TraceUtil.traceInfo(), context.rpcCreds());
+    } catch (ThriftSecurityException e) {
+      throw e;
+    } catch (TException e) {
+      LOG.debug("Failed to contact compactor {}", compactor, e);
+    } finally {
+      ThriftUtil.returnClient(client);
+    }
+    return List.of();
+  }
+
+  /**
+   * Get the compaction currently running on the Compactor
+   *
+   * @param compactorAddr
+   *          compactor address
+   * @param context
+   *          context
+   * @return external compaction job or null if none running
+   */
+  public static TExternalCompactionJob getRunningCompaction(HostAndPort compactorAddr,
+      ClientContext context) {
+
+    CompactorService.Client client = null;
+    try {
+      client = ThriftUtil.getClient(new CompactorService.Client.Factory(), compactorAddr, context);
+      TExternalCompactionJob job =
+          client.getRunningCompaction(TraceUtil.traceInfo(), context.rpcCreds());
+      if (job.getExternalCompactionId() != null) {
+        LOG.debug("Compactor is running {}", job.getExternalCompactionId());
+        return job;
+      }
+    } catch (TException e) {
+      LOG.debug("Failed to contact compactor {}", compactorAddr, e);
+    } finally {
+      ThriftUtil.returnClient(client);
+    }
+    return null;
+  }
+
+  private static ExternalCompactionId getRunningCompactionId(HostAndPort compactorAddr,
+      ClientContext context) {
+    CompactorService.Client client = null;
+    try {
+      client = ThriftUtil.getClient(new CompactorService.Client.Factory(), compactorAddr, context);
+      String secid = client.getRunningCompactionId(TraceUtil.traceInfo(), context.rpcCreds());
+      if (!secid.isEmpty()) {
+        return ExternalCompactionId.of(secid);
+      }
+    } catch (TException e) {
+      LOG.debug("Failed to contact compactor {}", compactorAddr, e);
+    } finally {
+      ThriftUtil.returnClient(client);
+    }
+    return null;
+  }
+
+  /**
+   * @param context
+   *          server context
+   * @return map of compactor and external compaction jobs
+   */
+  public static Map<HostAndPort,TExternalCompactionJob>
+      getCompactionsRunningOnCompactors(ClientContext context) {
+
+    final List<Pair<HostAndPort,Future<TExternalCompactionJob>>> running = new ArrayList<>();
+    final ExecutorService executor =
+        ThreadPools.createFixedThreadPool(16, "CompactorRunningCompactions", false);
+
+    getCompactorAddrs(context).forEach(hp -> {
+      running.add(new Pair<HostAndPort,Future<TExternalCompactionJob>>(hp,
+          executor.submit(() -> getRunningCompaction(hp, context))));
+    });
+    executor.shutdown();
+
+    final Map<HostAndPort,TExternalCompactionJob> results = new HashMap<>();
+    running.forEach(p -> {
+      try {
+        TExternalCompactionJob job = p.getSecond().get();
+        if (null != job && null != job.getExternalCompactionId()) {
+          results.put(p.getFirst(), job);
+        }
+      } catch (InterruptedException e) {
+        throw new RuntimeException(e);
+      } catch (ExecutionException e) {
+        throw new RuntimeException(e);
+      }
+    });
+    return results;
+  }
+
+  public static Collection<ExternalCompactionId>
+      getCompactionIdsRunningOnCompactors(ClientContext context) {
+    final ExecutorService executor =
+        ThreadPools.createFixedThreadPool(16, "CompactorRunningCompactions", false);
+
+    List<Future<ExternalCompactionId>> futures = new ArrayList<>();
+
+    getCompactorAddrs(context).forEach(hp -> {
+      futures.add(executor.submit(() -> getRunningCompactionId(hp, context)));
+    });
+    executor.shutdown();
+
+    HashSet<ExternalCompactionId> runningIds = new HashSet<>();
+
+    futures.forEach(future -> {
+      try {
+        ExternalCompactionId ceid = future.get();
+        if (ceid != null) {
+          runningIds.add(ceid);
+        }
+      } catch (InterruptedException e) {
+        throw new RuntimeException(e);
+      } catch (ExecutionException e) {
+        throw new RuntimeException(e);
+      }
+    });
+
+    return runningIds;
+  }
+
+  public static int countCompactors(String queueName, ClientContext context) {
+    String queueRoot = context.getZooKeeperRoot() + Constants.ZCOMPACTORS + "/" + queueName;
+    List<String> children = context.getZooCache().getChildren(queueRoot);
+    if (children == null)
+      return 0;
+
+    int count = 0;
+
+    for (String child : children) {
+      List<String> children2 = context.getZooCache().getChildren(queueRoot + "/" + child);
+      if (children2 != null && !children2.isEmpty()) {
+        count++;
+      }
+    }
+
+    return count;
+  }
+}
diff --git a/core/src/main/java/org/apache/accumulo/fate/zookeeper/ServiceLock.java b/core/src/main/java/org/apache/accumulo/fate/zookeeper/ServiceLock.java
index f33a93f..40ed6f6 100644
--- a/core/src/main/java/org/apache/accumulo/fate/zookeeper/ServiceLock.java
+++ b/core/src/main/java/org/apache/accumulo/fate/zookeeper/ServiceLock.java
@@ -135,7 +135,9 @@ public class ServiceLock implements Watcher {
     }
 
     @Override
-    public void failedToAcquireLock(Exception e) {}
+    public void failedToAcquireLock(Exception e) {
+      LOG.debug("Failed to acquire lock", e);
+    }
 
     @Override
     public void lostLock(LockLossReason reason) {
diff --git a/core/src/main/scripts/generate-thrift.sh b/core/src/main/scripts/generate-thrift.sh
index 1ecfa33..b973481 100755
--- a/core/src/main/scripts/generate-thrift.sh
+++ b/core/src/main/scripts/generate-thrift.sh
@@ -32,7 +32,7 @@
 [[ -z $REQUIRED_THRIFT_VERSION ]] && REQUIRED_THRIFT_VERSION='0.12.0'
 [[ -z $INCLUDED_MODULES ]]        && INCLUDED_MODULES=(../server/tracer)
 [[ -z $BASE_OUTPUT_PACKAGE ]]     && BASE_OUTPUT_PACKAGE='org.apache.accumulo.core'
-[[ -z $PACKAGES_TO_GENERATE ]]    && PACKAGES_TO_GENERATE=(gc master manager tabletserver securityImpl clientImpl dataImpl replication trace)
+[[ -z $PACKAGES_TO_GENERATE ]]    && PACKAGES_TO_GENERATE=(gc master manager tabletserver securityImpl clientImpl dataImpl replication trace compaction)
 [[ -z $BUILD_DIR ]]               && BUILD_DIR='target'
 [[ -z $LANGUAGES_TO_GENERATE ]]   && LANGUAGES_TO_GENERATE=(java)
 [[ -z $FINAL_DIR ]]               && FINAL_DIR='src/main'
diff --git a/core/src/main/spotbugs/exclude-filter.xml b/core/src/main/spotbugs/exclude-filter.xml
index 68d14e7..b8e221b 100644
--- a/core/src/main/spotbugs/exclude-filter.xml
+++ b/core/src/main/spotbugs/exclude-filter.xml
@@ -26,6 +26,7 @@
   <Match>
     <!-- ignore thrift-generated files -->
     <Or>
+      <Package name="org.apache.accumulo.core.compaction.thrift" />
       <Package name="org.apache.accumulo.core.clientImpl.thrift" />
       <Package name="org.apache.accumulo.core.dataImpl.thrift" />
       <Package name="org.apache.accumulo.core.gc.thrift" />
diff --git a/core/src/main/thrift-gen-java/org/apache/accumulo/core/compaction/thrift/CompactionCoordinatorService.java b/core/src/main/thrift-gen-java/org/apache/accumulo/core/compaction/thrift/CompactionCoordinatorService.java
new file mode 100644
index 0000000..02a8526
--- /dev/null
+++ b/core/src/main/thrift-gen-java/org/apache/accumulo/core/compaction/thrift/CompactionCoordinatorService.java
@@ -0,0 +1,5477 @@
+/*
+ * 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
+ *
+ *   http://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.12.0)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package org.apache.accumulo.core.compaction.thrift;
+
+@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"})
+public class CompactionCoordinatorService {
+
+  public interface Iface {
+
+    public void compactionCompleted(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String externalCompactionId, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent, org.apache.accumulo.core.tabletserver.thrift.TCompactionStats stats) throws UnknownCompactionIdException, org.apache.thrift.TException;
+
+    public org.apache.accumulo.core.tabletserver.thrift.TExternalCompactionJob getCompactionJob(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String queueName, java.lang.String compactor, java.lang.String externalCompactionId) throws org.apache.thrift.TException;
+
+    public void updateCompactionStatus(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String externalCompactionId, TCompactionState state, java.lang.String message, long timestamp) throws UnknownCompactionIdException, org.apache.thrift.TException;
+
+    public void compactionFailed(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String externalCompactionId, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent) throws UnknownCompactionIdException, org.apache.thrift.TException;
+
+  }
+
+  public interface AsyncIface {
+
+    public void compactionCompleted(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String externalCompactionId, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent, org.apache.accumulo.core.tabletserver.thrift.TCompactionStats stats, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
+
+    public void getCompactionJob(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String queueName, java.lang.String compactor, java.lang.String externalCompactionId, org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.tabletserver.thrift.TExternalCompactionJob> resultHandler) throws org.apache.thrift.TException;
+
+    public void updateCompactionStatus(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String externalCompactionId, TCompactionState state, java.lang.String message, long timestamp, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
+
+    public void compactionFailed(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String externalCompactionId, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
+
+  }
+
+  public static class Client extends org.apache.thrift.TServiceClient implements Iface {
+    public static class Factory implements org.apache.thrift.TServiceClientFactory<Client> {
+      public Factory() {}
+      public Client getClient(org.apache.thrift.protocol.TProtocol prot) {
+        return new Client(prot);
+      }
+      public Client getClient(org.apache.thrift.protocol.TProtocol iprot, org.apache.thrift.protocol.TProtocol oprot) {
+        return new Client(iprot, oprot);
+      }
+    }
+
+    public Client(org.apache.thrift.protocol.TProtocol prot)
+    {
+      super(prot, prot);
+    }
+
+    public Client(org.apache.thrift.protocol.TProtocol iprot, org.apache.thrift.protocol.TProtocol oprot) {
+      super(iprot, oprot);
+    }
+
+    public void compactionCompleted(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String externalCompactionId, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent, org.apache.accumulo.core.tabletserver.thrift.TCompactionStats stats) throws UnknownCompactionIdException, org.apache.thrift.TException
+    {
+      send_compactionCompleted(tinfo, credentials, externalCompactionId, extent, stats);
+      recv_compactionCompleted();
+    }
+
+    public void send_compactionCompleted(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String externalCompactionId, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent, org.apache.accumulo.core.tabletserver.thrift.TCompactionStats stats) throws org.apache.thrift.TException
+    {
+      compactionCompleted_args args = new compactionCompleted_args();
+      args.setTinfo(tinfo);
+      args.setCredentials(credentials);
+      args.setExternalCompactionId(externalCompactionId);
+      args.setExtent(extent);
+      args.setStats(stats);
+      sendBase("compactionCompleted", args);
+    }
+
+    public void recv_compactionCompleted() throws UnknownCompactionIdException, org.apache.thrift.TException
+    {
+      compactionCompleted_result result = new compactionCompleted_result();
+      receiveBase(result, "compactionCompleted");
+      if (result.e != null) {
+        throw result.e;
+      }
+      return;
+    }
+
+    public org.apache.accumulo.core.tabletserver.thrift.TExternalCompactionJob getCompactionJob(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String queueName, java.lang.String compactor, java.lang.String externalCompactionId) throws org.apache.thrift.TException
+    {
+      send_getCompactionJob(tinfo, credentials, queueName, compactor, externalCompactionId);
+      return recv_getCompactionJob();
+    }
+
+    public void send_getCompactionJob(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String queueName, java.lang.String compactor, java.lang.String externalCompactionId) throws org.apache.thrift.TException
+    {
+      getCompactionJob_args args = new getCompactionJob_args();
+      args.setTinfo(tinfo);
+      args.setCredentials(credentials);
+      args.setQueueName(queueName);
+      args.setCompactor(compactor);
+      args.setExternalCompactionId(externalCompactionId);
+      sendBase("getCompactionJob", args);
+    }
+
+    public org.apache.accumulo.core.tabletserver.thrift.TExternalCompactionJob recv_getCompactionJob() throws org.apache.thrift.TException
+    {
+      getCompactionJob_result result = new getCompactionJob_result();
+      receiveBase(result, "getCompactionJob");
+      if (result.isSetSuccess()) {
+        return result.success;
+      }
+      throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "getCompactionJob failed: unknown result");
+    }
+
+    public void updateCompactionStatus(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String externalCompactionId, TCompactionState state, java.lang.String message, long timestamp) throws UnknownCompactionIdException, org.apache.thrift.TException
+    {
+      send_updateCompactionStatus(tinfo, credentials, externalCompactionId, state, message, timestamp);
+      recv_updateCompactionStatus();
+    }
+
+    public void send_updateCompactionStatus(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String externalCompactionId, TCompactionState state, java.lang.String message, long timestamp) throws org.apache.thrift.TException
+    {
+      updateCompactionStatus_args args = new updateCompactionStatus_args();
+      args.setTinfo(tinfo);
+      args.setCredentials(credentials);
+      args.setExternalCompactionId(externalCompactionId);
+      args.setState(state);
+      args.setMessage(message);
+      args.setTimestamp(timestamp);
+      sendBase("updateCompactionStatus", args);
+    }
+
+    public void recv_updateCompactionStatus() throws UnknownCompactionIdException, org.apache.thrift.TException
+    {
+      updateCompactionStatus_result result = new updateCompactionStatus_result();
+      receiveBase(result, "updateCompactionStatus");
+      if (result.e != null) {
+        throw result.e;
+      }
+      return;
+    }
+
+    public void compactionFailed(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String externalCompactionId, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent) throws UnknownCompactionIdException, org.apache.thrift.TException
+    {
+      send_compactionFailed(tinfo, credentials, externalCompactionId, extent);
+      recv_compactionFailed();
+    }
+
+    public void send_compactionFailed(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String externalCompactionId, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent) throws org.apache.thrift.TException
+    {
+      compactionFailed_args args = new compactionFailed_args();
+      args.setTinfo(tinfo);
+      args.setCredentials(credentials);
+      args.setExternalCompactionId(externalCompactionId);
+      args.setExtent(extent);
+      sendBase("compactionFailed", args);
+    }
+
+    public void recv_compactionFailed() throws UnknownCompactionIdException, org.apache.thrift.TException
+    {
+      compactionFailed_result result = new compactionFailed_result();
+      receiveBase(result, "compactionFailed");
+      if (result.e != null) {
+        throw result.e;
+      }
+      return;
+    }
+
+  }
+  public static class AsyncClient extends org.apache.thrift.async.TAsyncClient implements AsyncIface {
+    public static class Factory implements org.apache.thrift.async.TAsyncClientFactory<AsyncClient> {
+      private org.apache.thrift.async.TAsyncClientManager clientManager;
+      private org.apache.thrift.protocol.TProtocolFactory protocolFactory;
+      public Factory(org.apache.thrift.async.TAsyncClientManager clientManager, org.apache.thrift.protocol.TProtocolFactory protocolFactory) {
+        this.clientManager = clientManager;
+        this.protocolFactory = protocolFactory;
+      }
+      public AsyncClient getAsyncClient(org.apache.thrift.transport.TNonblockingTransport transport) {
+        return new AsyncClient(protocolFactory, clientManager, transport);
+      }
+    }
+
+    public AsyncClient(org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.async.TAsyncClientManager clientManager, org.apache.thrift.transport.TNonblockingTransport transport) {
+      super(protocolFactory, clientManager, transport);
+    }
+
+    public void compactionCompleted(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String externalCompactionId, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent, org.apache.accumulo.core.tabletserver.thrift.TCompactionStats stats, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
+      checkReady();
+      compactionCompleted_call method_call = new compactionCompleted_call(tinfo, credentials, externalCompactionId, extent, stats, resultHandler, this, ___protocolFactory, ___transport);
+      this.___currentMethod = method_call;
+      ___manager.call(method_call);
+    }
+
+    public static class compactionCompleted_call extends org.apache.thrift.async.TAsyncMethodCall<Void> {
+      private org.apache.accumulo.core.trace.thrift.TInfo tinfo;
+      private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials;
+      private java.lang.String externalCompactionId;
+      private org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent;
+      private org.apache.accumulo.core.tabletserver.thrift.TCompactionStats stats;
+      public compactionCompleted_call(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String externalCompactionId, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent, org.apache.accumulo.core.tabletserver.thrift.TCompactionStats stats, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFa [...]
+        super(client, protocolFactory, transport, resultHandler, false);
+        this.tinfo = tinfo;
+        this.credentials = credentials;
+        this.externalCompactionId = externalCompactionId;
+        this.extent = extent;
+        this.stats = stats;
+      }
+
+      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("compactionCompleted", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        compactionCompleted_args args = new compactionCompleted_args();
+        args.setTinfo(tinfo);
+        args.setCredentials(credentials);
+        args.setExternalCompactionId(externalCompactionId);
+        args.setExtent(extent);
+        args.setStats(stats);
+        args.write(prot);
+        prot.writeMessageEnd();
+      }
+
+      public Void getResult() throws UnknownCompactionIdException, org.apache.thrift.TException {
+        if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
+          throw new java.lang.IllegalStateException("Method call not finished!");
+        }
+        org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
+        org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
+        return null;
+      }
+    }
+
+    public void getCompactionJob(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String queueName, java.lang.String compactor, java.lang.String externalCompactionId, org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.tabletserver.thrift.TExternalCompactionJob> resultHandler) throws org.apache.thrift.TException {
+      checkReady();
+      getCompactionJob_call method_call = new getCompactionJob_call(tinfo, credentials, queueName, compactor, externalCompactionId, resultHandler, this, ___protocolFactory, ___transport);
+      this.___currentMethod = method_call;
+      ___manager.call(method_call);
+    }
+
+    public static class getCompactionJob_call extends org.apache.thrift.async.TAsyncMethodCall<org.apache.accumulo.core.tabletserver.thrift.TExternalCompactionJob> {
+      private org.apache.accumulo.core.trace.thrift.TInfo tinfo;
+      private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials;
+      private java.lang.String queueName;
+      private java.lang.String compactor;
+      private java.lang.String externalCompactionId;
+      public getCompactionJob_call(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String queueName, java.lang.String compactor, java.lang.String externalCompactionId, org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.tabletserver.thrift.TExternalCompactionJob> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.ap [...]
+        super(client, protocolFactory, transport, resultHandler, false);
+        this.tinfo = tinfo;
+        this.credentials = credentials;
+        this.queueName = queueName;
+        this.compactor = compactor;
+        this.externalCompactionId = externalCompactionId;
+      }
+
+      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("getCompactionJob", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        getCompactionJob_args args = new getCompactionJob_args();
+        args.setTinfo(tinfo);
+        args.setCredentials(credentials);
+        args.setQueueName(queueName);
+        args.setCompactor(compactor);
+        args.setExternalCompactionId(externalCompactionId);
+        args.write(prot);
+        prot.writeMessageEnd();
+      }
+
+      public org.apache.accumulo.core.tabletserver.thrift.TExternalCompactionJob getResult() throws org.apache.thrift.TException {
+        if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
+          throw new java.lang.IllegalStateException("Method call not finished!");
+        }
+        org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
+        org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
+        return (new Client(prot)).recv_getCompactionJob();
+      }
+    }
+
+    public void updateCompactionStatus(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String externalCompactionId, TCompactionState state, java.lang.String message, long timestamp, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
+      checkReady();
+      updateCompactionStatus_call method_call = new updateCompactionStatus_call(tinfo, credentials, externalCompactionId, state, message, timestamp, resultHandler, this, ___protocolFactory, ___transport);
+      this.___currentMethod = method_call;
+      ___manager.call(method_call);
+    }
+
+    public static class updateCompactionStatus_call extends org.apache.thrift.async.TAsyncMethodCall<Void> {
+      private org.apache.accumulo.core.trace.thrift.TInfo tinfo;
+      private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials;
+      private java.lang.String externalCompactionId;
+      private TCompactionState state;
+      private java.lang.String message;
+      private long timestamp;
+      public updateCompactionStatus_call(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String externalCompactionId, TCompactionState state, java.lang.String message, long timestamp, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport tra [...]
+        super(client, protocolFactory, transport, resultHandler, false);
+        this.tinfo = tinfo;
+        this.credentials = credentials;
+        this.externalCompactionId = externalCompactionId;
+        this.state = state;
+        this.message = message;
+        this.timestamp = timestamp;
+      }
+
+      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("updateCompactionStatus", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        updateCompactionStatus_args args = new updateCompactionStatus_args();
+        args.setTinfo(tinfo);
+        args.setCredentials(credentials);
+        args.setExternalCompactionId(externalCompactionId);
+        args.setState(state);
+        args.setMessage(message);
+        args.setTimestamp(timestamp);
+        args.write(prot);
+        prot.writeMessageEnd();
+      }
+
+      public Void getResult() throws UnknownCompactionIdException, org.apache.thrift.TException {
+        if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
+          throw new java.lang.IllegalStateException("Method call not finished!");
+        }
+        org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
+        org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
+        return null;
+      }
+    }
+
+    public void compactionFailed(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String externalCompactionId, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
+      checkReady();
+      compactionFailed_call method_call = new compactionFailed_call(tinfo, credentials, externalCompactionId, extent, resultHandler, this, ___protocolFactory, ___transport);
+      this.___currentMethod = method_call;
+      ___manager.call(method_call);
+    }
+
+    public static class compactionFailed_call extends org.apache.thrift.async.TAsyncMethodCall<Void> {
+      private org.apache.accumulo.core.trace.thrift.TInfo tinfo;
+      private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials;
+      private java.lang.String externalCompactionId;
+      private org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent;
+      public compactionFailed_call(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String externalCompactionId, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent, 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) thro [...]
+        super(client, protocolFactory, transport, resultHandler, false);
+        this.tinfo = tinfo;
+        this.credentials = credentials;
+        this.externalCompactionId = externalCompactionId;
+        this.extent = extent;
+      }
+
+      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("compactionFailed", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        compactionFailed_args args = new compactionFailed_args();
+        args.setTinfo(tinfo);
+        args.setCredentials(credentials);
+        args.setExternalCompactionId(externalCompactionId);
+        args.setExtent(extent);
+        args.write(prot);
+        prot.writeMessageEnd();
+      }
+
+      public Void getResult() throws UnknownCompactionIdException, org.apache.thrift.TException {
+        if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
+          throw new java.lang.IllegalStateException("Method call not finished!");
+        }
+        org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
+        org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
+        return null;
+      }
+    }
+
+  }
+
+  public static class Processor<I extends Iface> extends org.apache.thrift.TBaseProcessor<I> implements org.apache.thrift.TProcessor {
+    private static final org.slf4j.Logger _LOGGER = org.slf4j.LoggerFactory.getLogger(Processor.class.getName());
+    public Processor(I iface) {
+      super(iface, getProcessMap(new java.util.HashMap<java.lang.String, org.apache.thrift.ProcessFunction<I, ? extends org.apache.thrift.TBase>>()));
+    }
+
+    protected Processor(I iface, java.util.Map<java.lang.String, org.apache.thrift.ProcessFunction<I, ? extends org.apache.thrift.TBase>> processMap) {
+      super(iface, getProcessMap(processMap));
+    }
+
+    private static <I extends Iface> java.util.Map<java.lang.String,  org.apache.thrift.ProcessFunction<I, ? extends org.apache.thrift.TBase>> getProcessMap(java.util.Map<java.lang.String, org.apache.thrift.ProcessFunction<I, ? extends  org.apache.thrift.TBase>> processMap) {
+      processMap.put("compactionCompleted", new compactionCompleted());
+      processMap.put("getCompactionJob", new getCompactionJob());
+      processMap.put("updateCompactionStatus", new updateCompactionStatus());
+      processMap.put("compactionFailed", new compactionFailed());
+      return processMap;
+    }
+
+    public static class compactionCompleted<I extends Iface> extends org.apache.thrift.ProcessFunction<I, compactionCompleted_args> {
+      public compactionCompleted() {
+        super("compactionCompleted");
+      }
+
+      public compactionCompleted_args getEmptyArgsInstance() {
+        return new compactionCompleted_args();
+      }
+
+      protected boolean isOneway() {
+        return false;
+      }
+
+      @Override
+      protected boolean rethrowUnhandledExceptions() {
+        return false;
+      }
+
+      public compactionCompleted_result getResult(I iface, compactionCompleted_args args) throws org.apache.thrift.TException {
+        compactionCompleted_result result = new compactionCompleted_result();
+        try {
+          iface.compactionCompleted(args.tinfo, args.credentials, args.externalCompactionId, args.extent, args.stats);
+        } catch (UnknownCompactionIdException e) {
+          result.e = e;
+        }
+        return result;
+      }
+    }
+
+    public static class getCompactionJob<I extends Iface> extends org.apache.thrift.ProcessFunction<I, getCompactionJob_args> {
+      public getCompactionJob() {
+        super("getCompactionJob");
+      }
+
+      public getCompactionJob_args getEmptyArgsInstance() {
+        return new getCompactionJob_args();
+      }
+
+      protected boolean isOneway() {
+        return false;
+      }
+
+      @Override
+      protected boolean rethrowUnhandledExceptions() {
+        return false;
+      }
+
+      public getCompactionJob_result getResult(I iface, getCompactionJob_args args) throws org.apache.thrift.TException {
+        getCompactionJob_result result = new getCompactionJob_result();
+        result.success = iface.getCompactionJob(args.tinfo, args.credentials, args.queueName, args.compactor, args.externalCompactionId);
+        return result;
+      }
+    }
+
+    public static class updateCompactionStatus<I extends Iface> extends org.apache.thrift.ProcessFunction<I, updateCompactionStatus_args> {
+      public updateCompactionStatus() {
+        super("updateCompactionStatus");
+      }
+
+      public updateCompactionStatus_args getEmptyArgsInstance() {
+        return new updateCompactionStatus_args();
+      }
+
+      protected boolean isOneway() {
+        return false;
+      }
+
+      @Override
+      protected boolean rethrowUnhandledExceptions() {
+        return false;
+      }
+
+      public updateCompactionStatus_result getResult(I iface, updateCompactionStatus_args args) throws org.apache.thrift.TException {
+        updateCompactionStatus_result result = new updateCompactionStatus_result();
+        try {
+          iface.updateCompactionStatus(args.tinfo, args.credentials, args.externalCompactionId, args.state, args.message, args.timestamp);
+        } catch (UnknownCompactionIdException e) {
+          result.e = e;
+        }
+        return result;
+      }
+    }
+
+    public static class compactionFailed<I extends Iface> extends org.apache.thrift.ProcessFunction<I, compactionFailed_args> {
+      public compactionFailed() {
+        super("compactionFailed");
+      }
+
+      public compactionFailed_args getEmptyArgsInstance() {
+        return new compactionFailed_args();
+      }
+
+      protected boolean isOneway() {
+        return false;
+      }
+
+      @Override
+      protected boolean rethrowUnhandledExceptions() {
+        return false;
+      }
+
+      public compactionFailed_result getResult(I iface, compactionFailed_args args) throws org.apache.thrift.TException {
+        compactionFailed_result result = new compactionFailed_result();
+        try {
+          iface.compactionFailed(args.tinfo, args.credentials, args.externalCompactionId, args.extent);
+        } catch (UnknownCompactionIdException e) {
+          result.e = e;
+        }
+        return result;
+      }
+    }
+
+  }
+
+  public static class AsyncProcessor<I extends AsyncIface> extends org.apache.thrift.TBaseAsyncProcessor<I> {
+    private static final org.slf4j.Logger _LOGGER = org.slf4j.LoggerFactory.getLogger(AsyncProcessor.class.getName());
+    public AsyncProcessor(I iface) {
+      super(iface, getProcessMap(new java.util.HashMap<java.lang.String, org.apache.thrift.AsyncProcessFunction<I, ? extends org.apache.thrift.TBase, ?>>()));
+    }
+
+    protected AsyncProcessor(I iface, java.util.Map<java.lang.String,  org.apache.thrift.AsyncProcessFunction<I, ? extends  org.apache.thrift.TBase, ?>> processMap) {
+      super(iface, getProcessMap(processMap));
+    }
+
+    private static <I extends AsyncIface> java.util.Map<java.lang.String,  org.apache.thrift.AsyncProcessFunction<I, ? extends  org.apache.thrift.TBase,?>> getProcessMap(java.util.Map<java.lang.String,  org.apache.thrift.AsyncProcessFunction<I, ? extends  org.apache.thrift.TBase, ?>> processMap) {
+      processMap.put("compactionCompleted", new compactionCompleted());
+      processMap.put("getCompactionJob", new getCompactionJob());
+      processMap.put("updateCompactionStatus", new updateCompactionStatus());
+      processMap.put("compactionFailed", new compactionFailed());
+      return processMap;
+    }
+
+    public static class compactionCompleted<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, compactionCompleted_args, Void> {
+      public compactionCompleted() {
+        super("compactionCompleted");
+      }
+
+      public compactionCompleted_args getEmptyArgsInstance() {
+        return new compactionCompleted_args();
+      }
+
+      public org.apache.thrift.async.AsyncMethodCallback<Void> 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<Void>() { 
+          public void onComplete(Void o) {
+            compactionCompleted_result result = new compactionCompleted_result();
+            try {
+              fcall.sendResponse(fb, result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
+            } catch (org.apache.thrift.transport.TTransportException e) {
+              _LOGGER.error("TTransportException writing to internal frame buffer", e);
+              fb.close();
+            } catch (java.lang.Exception e) {
+              _LOGGER.error("Exception writing to internal frame buffer", e);
+              onError(e);
+            }
+          }
+          public void onError(java.lang.Exception e) {
+            byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
+            org.apache.thrift.TSerializable msg;
+            compactionCompleted_result result = new compactionCompleted_result();
+            if (e instanceof UnknownCompactionIdException) {
+              result.e = (UnknownCompactionIdException) e;
+              result.setEIsSet(true);
+              msg = result;
+            } else if (e instanceof org.apache.thrift.transport.TTransportException) {
+              _LOGGER.error("TTransportException inside handler", e);
+              fb.close();
+              return;
+            } else if (e instanceof org.apache.thrift.TApplicationException) {
+              _LOGGER.error("TApplicationException inside handler", e);
+              msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
+              msg = (org.apache.thrift.TApplicationException)e;
+            } else {
+              _LOGGER.error("Exception inside handler", e);
+              msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
+              msg = new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage());
+            }
+            try {
+              fcall.sendResponse(fb,msg,msgType,seqid);
+            } catch (java.lang.Exception ex) {
+              _LOGGER.error("Exception writing to internal frame buffer", ex);
+              fb.close();
+            }
+          }
+        };
+      }
+
+      protected boolean isOneway() {
+        return false;
+      }
+
+      public void start(I iface, compactionCompleted_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
+        iface.compactionCompleted(args.tinfo, args.credentials, args.externalCompactionId, args.extent, args.stats,resultHandler);
+      }
+    }
+
+    public static class getCompactionJob<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, getCompactionJob_args, org.apache.accumulo.core.tabletserver.thrift.TExternalCompactionJob> {
+      public getCompactionJob() {
+        super("getCompactionJob");
+      }
+
+      public getCompactionJob_args getEmptyArgsInstance() {
+        return new getCompactionJob_args();
+      }
+
+      public org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.tabletserver.thrift.TExternalCompactionJob> 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<org.apache.accumulo.core.tabletserver.thrift.TExternalCompactionJob>() { 
+          public void onComplete(org.apache.accumulo.core.tabletserver.thrift.TExternalCompactionJob o) {
+            getCompactionJob_result result = new getCompactionJob_result();
+            result.success = o;
+            try {
+              fcall.sendResponse(fb, result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
+            } catch (org.apache.thrift.transport.TTransportException e) {
+              _LOGGER.error("TTransportException writing to internal frame buffer", e);
+              fb.close();
+            } catch (java.lang.Exception e) {
+              _LOGGER.error("Exception writing to internal frame buffer", e);
+              onError(e);
+            }
+          }
+          public void onError(java.lang.Exception e) {
+            byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
+            org.apache.thrift.TSerializable msg;
+            getCompactionJob_result result = new getCompactionJob_result();
+            if (e instanceof org.apache.thrift.transport.TTransportException) {
+              _LOGGER.error("TTransportException inside handler", e);
+              fb.close();
+              return;
+            } else if (e instanceof org.apache.thrift.TApplicationException) {
+              _LOGGER.error("TApplicationException inside handler", e);
+              msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
+              msg = (org.apache.thrift.TApplicationException)e;
+            } else {
+              _LOGGER.error("Exception inside handler", e);
+              msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
+              msg = new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage());
+            }
+            try {
+              fcall.sendResponse(fb,msg,msgType,seqid);
+            } catch (java.lang.Exception ex) {
+              _LOGGER.error("Exception writing to internal frame buffer", ex);
+              fb.close();
+            }
+          }
+        };
+      }
+
+      protected boolean isOneway() {
+        return false;
+      }
+
+      public void start(I iface, getCompactionJob_args args, org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.tabletserver.thrift.TExternalCompactionJob> resultHandler) throws org.apache.thrift.TException {
+        iface.getCompactionJob(args.tinfo, args.credentials, args.queueName, args.compactor, args.externalCompactionId,resultHandler);
+      }
+    }
+
+    public static class updateCompactionStatus<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, updateCompactionStatus_args, Void> {
+      public updateCompactionStatus() {
+        super("updateCompactionStatus");
+      }
+
+      public updateCompactionStatus_args getEmptyArgsInstance() {
+        return new updateCompactionStatus_args();
+      }
+
+      public org.apache.thrift.async.AsyncMethodCallback<Void> 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<Void>() { 
+          public void onComplete(Void o) {
+            updateCompactionStatus_result result = new updateCompactionStatus_result();
+            try {
+              fcall.sendResponse(fb, result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
+            } catch (org.apache.thrift.transport.TTransportException e) {
+              _LOGGER.error("TTransportException writing to internal frame buffer", e);
+              fb.close();
+            } catch (java.lang.Exception e) {
+              _LOGGER.error("Exception writing to internal frame buffer", e);
+              onError(e);
+            }
+          }
+          public void onError(java.lang.Exception e) {
+            byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
+            org.apache.thrift.TSerializable msg;
+            updateCompactionStatus_result result = new updateCompactionStatus_result();
+            if (e instanceof UnknownCompactionIdException) {
+              result.e = (UnknownCompactionIdException) e;
+              result.setEIsSet(true);
+              msg = result;
+            } else if (e instanceof org.apache.thrift.transport.TTransportException) {
+              _LOGGER.error("TTransportException inside handler", e);
+              fb.close();
+              return;
+            } else if (e instanceof org.apache.thrift.TApplicationException) {
+              _LOGGER.error("TApplicationException inside handler", e);
+              msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
+              msg = (org.apache.thrift.TApplicationException)e;
+            } else {
+              _LOGGER.error("Exception inside handler", e);
+              msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
+              msg = new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage());
+            }
+            try {
+              fcall.sendResponse(fb,msg,msgType,seqid);
+            } catch (java.lang.Exception ex) {
+              _LOGGER.error("Exception writing to internal frame buffer", ex);
+              fb.close();
+            }
+          }
+        };
+      }
+
+      protected boolean isOneway() {
+        return false;
+      }
+
+      public void start(I iface, updateCompactionStatus_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
+        iface.updateCompactionStatus(args.tinfo, args.credentials, args.externalCompactionId, args.state, args.message, args.timestamp,resultHandler);
+      }
+    }
+
+    public static class compactionFailed<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, compactionFailed_args, Void> {
+      public compactionFailed() {
+        super("compactionFailed");
+      }
+
+      public compactionFailed_args getEmptyArgsInstance() {
+        return new compactionFailed_args();
+      }
+
+      public org.apache.thrift.async.AsyncMethodCallback<Void> 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<Void>() { 
+          public void onComplete(Void o) {
+            compactionFailed_result result = new compactionFailed_result();
+            try {
+              fcall.sendResponse(fb, result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
+            } catch (org.apache.thrift.transport.TTransportException e) {
+              _LOGGER.error("TTransportException writing to internal frame buffer", e);
+              fb.close();
+            } catch (java.lang.Exception e) {
+              _LOGGER.error("Exception writing to internal frame buffer", e);
+              onError(e);
+            }
+          }
+          public void onError(java.lang.Exception e) {
+            byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
+            org.apache.thrift.TSerializable msg;
+            compactionFailed_result result = new compactionFailed_result();
+            if (e instanceof UnknownCompactionIdException) {
+              result.e = (UnknownCompactionIdException) e;
+              result.setEIsSet(true);
+              msg = result;
+            } else if (e instanceof org.apache.thrift.transport.TTransportException) {
+              _LOGGER.error("TTransportException inside handler", e);
+              fb.close();
+              return;
+            } else if (e instanceof org.apache.thrift.TApplicationException) {
+              _LOGGER.error("TApplicationException inside handler", e);
+              msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
+              msg = (org.apache.thrift.TApplicationException)e;
+            } else {
+              _LOGGER.error("Exception inside handler", e);
+              msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
+              msg = new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage());
+            }
+            try {
+              fcall.sendResponse(fb,msg,msgType,seqid);
+            } catch (java.lang.Exception ex) {
+              _LOGGER.error("Exception writing to internal frame buffer", ex);
+              fb.close();
+            }
+          }
+        };
+      }
+
+      protected boolean isOneway() {
+        return false;
+      }
+
+      public void start(I iface, compactionFailed_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
+        iface.compactionFailed(args.tinfo, args.credentials, args.externalCompactionId, args.extent,resultHandler);
+      }
+    }
+
+  }
+
+  public static class compactionCompleted_args implements org.apache.thrift.TBase<compactionCompleted_args, compactionCompleted_args._Fields>, java.io.Serializable, Cloneable, Comparable<compactionCompleted_args>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("compactionCompleted_args");
+
+    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 EXTERNAL_COMPACTION_ID_FIELD_DESC = new org.apache.thrift.protocol.TField("externalCompactionId", org.apache.thrift.protocol.TType.STRING, (short)3);
+    private static final org.apache.thrift.protocol.TField EXTENT_FIELD_DESC = new org.apache.thrift.protocol.TField("extent", org.apache.thrift.protocol.TType.STRUCT, (short)4);
+    private static final org.apache.thrift.protocol.TField STATS_FIELD_DESC = new org.apache.thrift.protocol.TField("stats", org.apache.thrift.protocol.TType.STRUCT, (short)5);
+
+    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new compactionCompleted_argsStandardSchemeFactory();
+    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new compactionCompleted_argsTupleSchemeFactory();
+
+    public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo; // required
+    public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials; // required
+    public @org.apache.thrift.annotation.Nullable java.lang.String externalCompactionId; // required
+    public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent; // required
+    public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.tabletserver.thrift.TCompactionStats stats; // 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"),
+      EXTERNAL_COMPACTION_ID((short)3, "externalCompactionId"),
+      EXTENT((short)4, "extent"),
+      STATS((short)5, "stats");
+
+      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: // TINFO
+            return TINFO;
+          case 2: // CREDENTIALS
+            return CREDENTIALS;
+          case 3: // EXTERNAL_COMPACTION_ID
+            return EXTERNAL_COMPACTION_ID;
+          case 4: // EXTENT
+            return EXTENT;
+          case 5: // STATS
+            return STATS;
+          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;
+      }
+
+      public short getThriftFieldId() {
+        return _thriftId;
+      }
+
+      public java.lang.String getFieldName() {
+        return _fieldName;
+      }
+    }
+
+    // isset id assignments
+    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.TINFO, new org.apache.thrift.meta_data.FieldMetaData("tinfo", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.trace.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.EXTERNAL_COMPACTION_ID, new org.apache.thrift.meta_data.FieldMetaData("externalCompactionId", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+      tmpMap.put(_Fields.EXTENT, new org.apache.thrift.meta_data.FieldMetaData("extent", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent.class)));
+      tmpMap.put(_Fields.STATS, new org.apache.thrift.meta_data.FieldMetaData("stats", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.tabletserver.thrift.TCompactionStats.class)));
+      metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(compactionCompleted_args.class, metaDataMap);
+    }
+
+    public compactionCompleted_args() {
+    }
+
+    public compactionCompleted_args(
+      org.apache.accumulo.core.trace.thrift.TInfo tinfo,
+      org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials,
+      java.lang.String externalCompactionId,
+      org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent,
+      org.apache.accumulo.core.tabletserver.thrift.TCompactionStats stats)
+    {
+      this();
+      this.tinfo = tinfo;
+      this.credentials = credentials;
+      this.externalCompactionId = externalCompactionId;
+      this.extent = extent;
+      this.stats = stats;
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public compactionCompleted_args(compactionCompleted_args other) {
+      if (other.isSetTinfo()) {
+        this.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo(other.tinfo);
+      }
+      if (other.isSetCredentials()) {
+        this.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(other.credentials);
+      }
+      if (other.isSetExternalCompactionId()) {
+        this.externalCompactionId = other.externalCompactionId;
+      }
+      if (other.isSetExtent()) {
+        this.extent = new org.apache.accumulo.core.dataImpl.thrift.TKeyExtent(other.extent);
+      }
+      if (other.isSetStats()) {
+        this.stats = new org.apache.accumulo.core.tabletserver.thrift.TCompactionStats(other.stats);
+      }
+    }
+
+    public compactionCompleted_args deepCopy() {
+      return new compactionCompleted_args(this);
+    }
+
+    @Override
+    public void clear() {
+      this.tinfo = null;
+      this.credentials = null;
+      this.externalCompactionId = null;
+      this.extent = null;
+      this.stats = null;
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public org.apache.accumulo.core.trace.thrift.TInfo getTinfo() {
+      return this.tinfo;
+    }
+
+    public compactionCompleted_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo) {
+      this.tinfo = tinfo;
+      return this;
+    }
+
+    public void unsetTinfo() {
+      this.tinfo = null;
+    }
+
+    /** Returns true if field tinfo is set (has been assigned a value) and false otherwise */
+    public boolean isSetTinfo() {
+      return this.tinfo != null;
+    }
+
+    public void setTinfoIsSet(boolean value) {
+      if (!value) {
+        this.tinfo = null;
+      }
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public org.apache.accumulo.core.securityImpl.thrift.TCredentials getCredentials() {
+      return this.credentials;
+    }
+
+    public compactionCompleted_args setCredentials(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) {
+      this.credentials = credentials;
+      return this;
+    }
+
+    public void unsetCredentials() {
+      this.credentials = null;
+    }
+
+    /** Returns true if field credentials is set (has been assigned a value) and false otherwise */
+    public boolean isSetCredentials() {
+      return this.credentials != null;
+    }
+
+    public void setCredentialsIsSet(boolean value) {
+      if (!value) {
+        this.credentials = null;
+      }
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public java.lang.String getExternalCompactionId() {
+      return this.externalCompactionId;
+    }
+
+    public compactionCompleted_args setExternalCompactionId(@org.apache.thrift.annotation.Nullable java.lang.String externalCompactionId) {
+      this.externalCompactionId = externalCompactionId;
+      return this;
+    }
+
+    public void unsetExternalCompactionId() {
+      this.externalCompactionId = null;
+    }
+
+    /** Returns true if field externalCompactionId is set (has been assigned a value) and false otherwise */
+    public boolean isSetExternalCompactionId() {
+      return this.externalCompactionId != null;
+    }
+
+    public void setExternalCompactionIdIsSet(boolean value) {
+      if (!value) {
+        this.externalCompactionId = null;
+      }
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public org.apache.accumulo.core.dataImpl.thrift.TKeyExtent getExtent() {
+      return this.extent;
+    }
+
+    public compactionCompleted_args setExtent(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent) {
+      this.extent = extent;
+      return this;
+    }
+
+    public void unsetExtent() {
+      this.extent = null;
+    }
+
+    /** Returns true if field extent is set (has been assigned a value) and false otherwise */
+    public boolean isSetExtent() {
+      return this.extent != null;
+    }
+
+    public void setExtentIsSet(boolean value) {
+      if (!value) {
+        this.extent = null;
+      }
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public org.apache.accumulo.core.tabletserver.thrift.TCompactionStats getStats() {
+      return this.stats;
+    }
+
+    public compactionCompleted_args setStats(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.tabletserver.thrift.TCompactionStats stats) {
+      this.stats = stats;
+      return this;
+    }
+
+    public void unsetStats() {
+      this.stats = null;
+    }
+
+    /** Returns true if field stats is set (has been assigned a value) and false otherwise */
+    public boolean isSetStats() {
+      return this.stats != null;
+    }
+
+    public void setStatsIsSet(boolean value) {
+      if (!value) {
+        this.stats = null;
+      }
+    }
+
+    public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable java.lang.Object value) {
+      switch (field) {
+      case TINFO:
+        if (value == null) {
+          unsetTinfo();
+        } else {
+          setTinfo((org.apache.accumulo.core.trace.thrift.TInfo)value);
+        }
+        break;
+
+      case CREDENTIALS:
+        if (value == null) {
+          unsetCredentials();
+        } else {
+          setCredentials((org.apache.accumulo.core.securityImpl.thrift.TCredentials)value);
+        }
+        break;
+
+      case EXTERNAL_COMPACTION_ID:
+        if (value == null) {
+          unsetExternalCompactionId();
+        } else {
+          setExternalCompactionId((java.lang.String)value);
+        }
+        break;
+
+      case EXTENT:
+        if (value == null) {
+          unsetExtent();
+        } else {
+          setExtent((org.apache.accumulo.core.dataImpl.thrift.TKeyExtent)value);
+        }
+        break;
+
+      case STATS:
+        if (value == null) {
+          unsetStats();
+        } else {
+          setStats((org.apache.accumulo.core.tabletserver.thrift.TCompactionStats)value);
+        }
+        break;
+
+      }
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public java.lang.Object getFieldValue(_Fields field) {
+      switch (field) {
+      case TINFO:
+        return getTinfo();
+
+      case CREDENTIALS:
+        return getCredentials();
+
+      case EXTERNAL_COMPACTION_ID:
+        return getExternalCompactionId();
+
+      case EXTENT:
+        return getExtent();
+
+      case STATS:
+        return getStats();
+
+      }
+      throw new java.lang.IllegalStateException();
+    }
+
+    /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+    public boolean isSet(_Fields field) {
+      if (field == null) {
+        throw new java.lang.IllegalArgumentException();
+      }
+
+      switch (field) {
+      case TINFO:
+        return isSetTinfo();
+      case CREDENTIALS:
+        return isSetCredentials();
+      case EXTERNAL_COMPACTION_ID:
+        return isSetExternalCompactionId();
+      case EXTENT:
+        return isSetExtent();
+      case STATS:
+        return isSetStats();
+      }
+      throw new java.lang.IllegalStateException();
+    }
+
+    @Override
+    public boolean equals(java.lang.Object that) {
+      if (that == null)
+        return false;
+      if (that instanceof compactionCompleted_args)
+        return this.equals((compactionCompleted_args)that);
+      return false;
+    }
+
+    public boolean equals(compactionCompleted_args that) {
+      if (that == null)
+        return false;
+      if (this == that)
+        return true;
+
+      boolean this_present_tinfo = true && this.isSetTinfo();
+      boolean that_present_tinfo = true && that.isSetTinfo();
+      if (this_present_tinfo || that_present_tinfo) {
+        if (!(this_present_tinfo && that_present_tinfo))
+          return false;
+        if (!this.tinfo.equals(that.tinfo))
+          return false;
+      }
+
+      boolean this_present_credentials = true && this.isSetCredentials();
+      boolean that_present_credentials = true && that.isSetCredentials();
+      if (this_present_credentials || that_present_credentials) {
+        if (!(this_present_credentials && that_present_credentials))
+          return false;
+        if (!this.credentials.equals(that.credentials))
+          return false;
+      }
+
+      boolean this_present_externalCompactionId = true && this.isSetExternalCompactionId();
+      boolean that_present_externalCompactionId = true && that.isSetExternalCompactionId();
+      if (this_present_externalCompactionId || that_present_externalCompactionId) {
+        if (!(this_present_externalCompactionId && that_present_externalCompactionId))
+          return false;
+        if (!this.externalCompactionId.equals(that.externalCompactionId))
+          return false;
+      }
+
+      boolean this_present_extent = true && this.isSetExtent();
+      boolean that_present_extent = true && that.isSetExtent();
+      if (this_present_extent || that_present_extent) {
+        if (!(this_present_extent && that_present_extent))
+          return false;
+        if (!this.extent.equals(that.extent))
+          return false;
+      }
+
+      boolean this_present_stats = true && this.isSetStats();
+      boolean that_present_stats = true && that.isSetStats();
+      if (this_present_stats || that_present_stats) {
+        if (!(this_present_stats && that_present_stats))
+          return false;
+        if (!this.stats.equals(that.stats))
+          return false;
+      }
+
+      return true;
+    }
+
+    @Override
+    public int hashCode() {
+      int hashCode = 1;
+
+      hashCode = hashCode * 8191 + ((isSetTinfo()) ? 131071 : 524287);
+      if (isSetTinfo())
+        hashCode = hashCode * 8191 + tinfo.hashCode();
+
+      hashCode = hashCode * 8191 + ((isSetCredentials()) ? 131071 : 524287);
+      if (isSetCredentials())
+        hashCode = hashCode * 8191 + credentials.hashCode();
+
+      hashCode = hashCode * 8191 + ((isSetExternalCompactionId()) ? 131071 : 524287);
+      if (isSetExternalCompactionId())
+        hashCode = hashCode * 8191 + externalCompactionId.hashCode();
+
+      hashCode = hashCode * 8191 + ((isSetExtent()) ? 131071 : 524287);
+      if (isSetExtent())
+        hashCode = hashCode * 8191 + extent.hashCode();
+
+      hashCode = hashCode * 8191 + ((isSetStats()) ? 131071 : 524287);
+      if (isSetStats())
+        hashCode = hashCode * 8191 + stats.hashCode();
+
+      return hashCode;
+    }
+
+    @Override
+    public int compareTo(compactionCompleted_args other) {
+      if (!getClass().equals(other.getClass())) {
+        return getClass().getName().compareTo(other.getClass().getName());
+      }
+
+      int lastComparison = 0;
+
+      lastComparison = java.lang.Boolean.valueOf(isSetTinfo()).compareTo(other.isSetTinfo());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetTinfo()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tinfo, other.tinfo);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      lastComparison = java.lang.Boolean.valueOf(isSetCredentials()).compareTo(other.isSetCredentials());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetCredentials()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.credentials, other.credentials);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      lastComparison = java.lang.Boolean.valueOf(isSetExternalCompactionId()).compareTo(other.isSetExternalCompactionId());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetExternalCompactionId()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.externalCompactionId, other.externalCompactionId);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      lastComparison = java.lang.Boolean.valueOf(isSetExtent()).compareTo(other.isSetExtent());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetExtent()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.extent, other.extent);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      lastComparison = java.lang.Boolean.valueOf(isSetStats()).compareTo(other.isSetStats());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetStats()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.stats, other.stats);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      return 0;
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public _Fields fieldForId(int fieldId) {
+      return _Fields.findByThriftId(fieldId);
+    }
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+      scheme(iprot).read(iprot, this);
+    }
+
+    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("compactionCompleted_args(");
+      boolean first = true;
+
+      sb.append("tinfo:");
+      if (this.tinfo == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.tinfo);
+      }
+      first = false;
+      if (!first) sb.append(", ");
+      sb.append("credentials:");
+      if (this.credentials == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.credentials);
+      }
+      first = false;
+      if (!first) sb.append(", ");
+      sb.append("externalCompactionId:");
+      if (this.externalCompactionId == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.externalCompactionId);
+      }
+      first = false;
+      if (!first) sb.append(", ");
+      sb.append("extent:");
+      if (this.extent == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.extent);
+      }
+      first = false;
+      if (!first) sb.append(", ");
+      sb.append("stats:");
+      if (this.stats == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.stats);
+      }
+      first = false;
+      sb.append(")");
+      return sb.toString();
+    }
+
+    public void validate() throws org.apache.thrift.TException {
+      // check for required fields
+      // check for sub-struct validity
+      if (tinfo != null) {
+        tinfo.validate();
+      }
+      if (credentials != null) {
+        credentials.validate();
+      }
+      if (extent != null) {
+        extent.validate();
+      }
+      if (stats != null) {
+        stats.validate();
+      }
+    }
+
+    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 {
+        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 compactionCompleted_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public compactionCompleted_argsStandardScheme getScheme() {
+        return new compactionCompleted_argsStandardScheme();
+      }
+    }
+
+    private static class compactionCompleted_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme<compactionCompleted_args> {
+
+      public void read(org.apache.thrift.protocol.TProtocol iprot, compactionCompleted_args 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: // TINFO
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+                struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+                struct.tinfo.read(iprot);
+                struct.setTinfoIsSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            case 2: // CREDENTIALS
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+                struct.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials();
+                struct.credentials.read(iprot);
+                struct.setCredentialsIsSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            case 3: // EXTERNAL_COMPACTION_ID
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+                struct.externalCompactionId = iprot.readString();
+                struct.setExternalCompactionIdIsSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            case 4: // EXTENT
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+                struct.extent = new org.apache.accumulo.core.dataImpl.thrift.TKeyExtent();
+                struct.extent.read(iprot);
+                struct.setExtentIsSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            case 5: // STATS
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+                struct.stats = new org.apache.accumulo.core.tabletserver.thrift.TCompactionStats();
+                struct.stats.read(iprot);
+                struct.setStatsIsSet(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();
+      }
+
+      public void write(org.apache.thrift.protocol.TProtocol oprot, compactionCompleted_args struct) throws org.apache.thrift.TException {
+        struct.validate();
+
+        oprot.writeStructBegin(STRUCT_DESC);
+        if (struct.tinfo != null) {
+          oprot.writeFieldBegin(TINFO_FIELD_DESC);
+          struct.tinfo.write(oprot);
+          oprot.writeFieldEnd();
+        }
+        if (struct.credentials != null) {
+          oprot.writeFieldBegin(CREDENTIALS_FIELD_DESC);
+          struct.credentials.write(oprot);
+          oprot.writeFieldEnd();
+        }
+        if (struct.externalCompactionId != null) {
+          oprot.writeFieldBegin(EXTERNAL_COMPACTION_ID_FIELD_DESC);
+          oprot.writeString(struct.externalCompactionId);
+          oprot.writeFieldEnd();
+        }
+        if (struct.extent != null) {
+          oprot.writeFieldBegin(EXTENT_FIELD_DESC);
+          struct.extent.write(oprot);
+          oprot.writeFieldEnd();
+        }
+        if (struct.stats != null) {
+          oprot.writeFieldBegin(STATS_FIELD_DESC);
+          struct.stats.write(oprot);
+          oprot.writeFieldEnd();
+        }
+        oprot.writeFieldStop();
+        oprot.writeStructEnd();
+      }
+
+    }
+
+    private static class compactionCompleted_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public compactionCompleted_argsTupleScheme getScheme() {
+        return new compactionCompleted_argsTupleScheme();
+      }
+    }
+
+    private static class compactionCompleted_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme<compactionCompleted_args> {
+
+      @Override
+      public void write(org.apache.thrift.protocol.TProtocol prot, compactionCompleted_args 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.isSetTinfo()) {
+          optionals.set(0);
+        }
+        if (struct.isSetCredentials()) {
+          optionals.set(1);
+        }
+        if (struct.isSetExternalCompactionId()) {
+          optionals.set(2);
+        }
+        if (struct.isSetExtent()) {
+          optionals.set(3);
+        }
+        if (struct.isSetStats()) {
+          optionals.set(4);
+        }
+        oprot.writeBitSet(optionals, 5);
+        if (struct.isSetTinfo()) {
+          struct.tinfo.write(oprot);
+        }
+        if (struct.isSetCredentials()) {
+          struct.credentials.write(oprot);
+        }
+        if (struct.isSetExternalCompactionId()) {
+          oprot.writeString(struct.externalCompactionId);
+        }
+        if (struct.isSetExtent()) {
+          struct.extent.write(oprot);
+        }
+        if (struct.isSetStats()) {
+          struct.stats.write(oprot);
+        }
+      }
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol prot, compactionCompleted_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(5);
+        if (incoming.get(0)) {
+          struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+          struct.tinfo.read(iprot);
+          struct.setTinfoIsSet(true);
+        }
+        if (incoming.get(1)) {
+          struct.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials();
+          struct.credentials.read(iprot);
+          struct.setCredentialsIsSet(true);
+        }
+        if (incoming.get(2)) {
+          struct.externalCompactionId = iprot.readString();
+          struct.setExternalCompactionIdIsSet(true);
+        }
+        if (incoming.get(3)) {
+          struct.extent = new org.apache.accumulo.core.dataImpl.thrift.TKeyExtent();
+          struct.extent.read(iprot);
+          struct.setExtentIsSet(true);
+        }
+        if (incoming.get(4)) {
+          struct.stats = new org.apache.accumulo.core.tabletserver.thrift.TCompactionStats();
+          struct.stats.read(iprot);
+          struct.setStatsIsSet(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();
+    }
+  }
+
+  public static class compactionCompleted_result implements org.apache.thrift.TBase<compactionCompleted_result, compactionCompleted_result._Fields>, java.io.Serializable, Cloneable, Comparable<compactionCompleted_result>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("compactionCompleted_result");
+
+    private static final org.apache.thrift.protocol.TField E_FIELD_DESC = new org.apache.thrift.protocol.TField("e", org.apache.thrift.protocol.TType.STRUCT, (short)1);
+
+    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new compactionCompleted_resultStandardSchemeFactory();
+    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new compactionCompleted_resultTupleSchemeFactory();
+
+    public @org.apache.thrift.annotation.Nullable UnknownCompactionIdException e; // 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 {
+      E((short)1, "e");
+
+      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: // E
+            return E;
+          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;
+      }
+
+      public short getThriftFieldId() {
+        return _thriftId;
+      }
+
+      public java.lang.String getFieldName() {
+        return _fieldName;
+      }
+    }
+
+    // isset id assignments
+    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.E, new org.apache.thrift.meta_data.FieldMetaData("e", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, UnknownCompactionIdException.class)));
+      metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(compactionCompleted_result.class, metaDataMap);
+    }
+
+    public compactionCompleted_result() {
+    }
+
+    public compactionCompleted_result(
+      UnknownCompactionIdException e)
+    {
+      this();
+      this.e = e;
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public compactionCompleted_result(compactionCompleted_result other) {
+      if (other.isSetE()) {
+        this.e = new UnknownCompactionIdException(other.e);
+      }
+    }
+
+    public compactionCompleted_result deepCopy() {
+      return new compactionCompleted_result(this);
+    }
+
+    @Override
+    public void clear() {
+      this.e = null;
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public UnknownCompactionIdException getE() {
+      return this.e;
+    }
+
+    public compactionCompleted_result setE(@org.apache.thrift.annotation.Nullable UnknownCompactionIdException e) {
+      this.e = e;
+      return this;
+    }
+
+    public void unsetE() {
+      this.e = null;
+    }
+
+    /** Returns true if field e is set (has been assigned a value) and false otherwise */
+    public boolean isSetE() {
+      return this.e != null;
+    }
+
+    public void setEIsSet(boolean value) {
+      if (!value) {
+        this.e = null;
+      }
+    }
+
+    public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable java.lang.Object value) {
+      switch (field) {
+      case E:
+        if (value == null) {
+          unsetE();
+        } else {
+          setE((UnknownCompactionIdException)value);
+        }
+        break;
+
+      }
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public java.lang.Object getFieldValue(_Fields field) {
+      switch (field) {
+      case E:
+        return getE();
+
+      }
+      throw new java.lang.IllegalStateException();
+    }
+
+    /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+    public boolean isSet(_Fields field) {
+      if (field == null) {
+        throw new java.lang.IllegalArgumentException();
+      }
+
+      switch (field) {
+      case E:
+        return isSetE();
+      }
+      throw new java.lang.IllegalStateException();
+    }
+
+    @Override
+    public boolean equals(java.lang.Object that) {
+      if (that == null)
+        return false;
+      if (that instanceof compactionCompleted_result)
+        return this.equals((compactionCompleted_result)that);
+      return false;
+    }
+
+    public boolean equals(compactionCompleted_result that) {
+      if (that == null)
+        return false;
+      if (this == that)
+        return true;
+
+      boolean this_present_e = true && this.isSetE();
+      boolean that_present_e = true && that.isSetE();
+      if (this_present_e || that_present_e) {
+        if (!(this_present_e && that_present_e))
+          return false;
+        if (!this.e.equals(that.e))
+          return false;
+      }
+
+      return true;
+    }
+
+    @Override
+    public int hashCode() {
+      int hashCode = 1;
+
+      hashCode = hashCode * 8191 + ((isSetE()) ? 131071 : 524287);
+      if (isSetE())
+        hashCode = hashCode * 8191 + e.hashCode();
+
+      return hashCode;
+    }
+
+    @Override
+    public int compareTo(compactionCompleted_result other) {
+      if (!getClass().equals(other.getClass())) {
+        return getClass().getName().compareTo(other.getClass().getName());
+      }
+
+      int lastComparison = 0;
+
+      lastComparison = java.lang.Boolean.valueOf(isSetE()).compareTo(other.isSetE());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetE()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.e, other.e);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      return 0;
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public _Fields fieldForId(int fieldId) {
+      return _Fields.findByThriftId(fieldId);
+    }
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+      scheme(iprot).read(iprot, this);
+    }
+
+    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("compactionCompleted_result(");
+      boolean first = true;
+
+      sb.append("e:");
+      if (this.e == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.e);
+      }
+      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 {
+        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 compactionCompleted_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public compactionCompleted_resultStandardScheme getScheme() {
+        return new compactionCompleted_resultStandardScheme();
+      }
+    }
+
+    private static class compactionCompleted_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme<compactionCompleted_result> {
+
+      public void read(org.apache.thrift.protocol.TProtocol iprot, compactionCompleted_result 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: // E
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+                struct.e = new UnknownCompactionIdException();
+                struct.e.read(iprot);
+                struct.setEIsSet(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();
+      }
+
+      public void write(org.apache.thrift.protocol.TProtocol oprot, compactionCompleted_result struct) throws org.apache.thrift.TException {
+        struct.validate();
+
+        oprot.writeStructBegin(STRUCT_DESC);
+        if (struct.e != null) {
+          oprot.writeFieldBegin(E_FIELD_DESC);
+          struct.e.write(oprot);
+          oprot.writeFieldEnd();
+        }
+        oprot.writeFieldStop();
+        oprot.writeStructEnd();
+      }
+
+    }
+
+    private static class compactionCompleted_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public compactionCompleted_resultTupleScheme getScheme() {
+        return new compactionCompleted_resultTupleScheme();
+      }
+    }
+
+    private static class compactionCompleted_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme<compactionCompleted_result> {
+
+      @Override
+      public void write(org.apache.thrift.protocol.TProtocol prot, compactionCompleted_result 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.isSetE()) {
+          optionals.set(0);
+        }
+        oprot.writeBitSet(optionals, 1);
+        if (struct.isSetE()) {
+          struct.e.write(oprot);
+        }
+      }
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol prot, compactionCompleted_result struct) throws org.apache.thrift.TException {
+        org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
+        java.util.BitSet incoming = iprot.readBitSet(1);
+        if (incoming.get(0)) {
+          struct.e = new UnknownCompactionIdException();
+          struct.e.read(iprot);
+          struct.setEIsSet(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();
+    }
+  }
+
+  public static class getCompactionJob_args implements org.apache.thrift.TBase<getCompactionJob_args, getCompactionJob_args._Fields>, java.io.Serializable, Cloneable, Comparable<getCompactionJob_args>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getCompactionJob_args");
+
+    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 QUEUE_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("queueName", org.apache.thrift.protocol.TType.STRING, (short)3);
+    private static final org.apache.thrift.protocol.TField COMPACTOR_FIELD_DESC = new org.apache.thrift.protocol.TField("compactor", org.apache.thrift.protocol.TType.STRING, (short)4);
+    private static final org.apache.thrift.protocol.TField EXTERNAL_COMPACTION_ID_FIELD_DESC = new org.apache.thrift.protocol.TField("externalCompactionId", org.apache.thrift.protocol.TType.STRING, (short)5);
+
+    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new getCompactionJob_argsStandardSchemeFactory();
+    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new getCompactionJob_argsTupleSchemeFactory();
+
+    public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo; // required
+    public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials; // required
+    public @org.apache.thrift.annotation.Nullable java.lang.String queueName; // required
+    public @org.apache.thrift.annotation.Nullable java.lang.String compactor; // required
+    public @org.apache.thrift.annotation.Nullable java.lang.String externalCompactionId; // 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"),
+      QUEUE_NAME((short)3, "queueName"),
+      COMPACTOR((short)4, "compactor"),
+      EXTERNAL_COMPACTION_ID((short)5, "externalCompactionId");
+
+      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: // TINFO
+            return TINFO;
+          case 2: // CREDENTIALS
+            return CREDENTIALS;
+          case 3: // QUEUE_NAME
+            return QUEUE_NAME;
+          case 4: // COMPACTOR
+            return COMPACTOR;
+          case 5: // EXTERNAL_COMPACTION_ID
+            return EXTERNAL_COMPACTION_ID;
+          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;
+      }
+
+      public short getThriftFieldId() {
+        return _thriftId;
+      }
+
+      public java.lang.String getFieldName() {
+        return _fieldName;
+      }
+    }
+
+    // isset id assignments
+    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.TINFO, new org.apache.thrift.meta_data.FieldMetaData("tinfo", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.trace.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.QUEUE_NAME, new org.apache.thrift.meta_data.FieldMetaData("queueName", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+      tmpMap.put(_Fields.COMPACTOR, new org.apache.thrift.meta_data.FieldMetaData("compactor", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+      tmpMap.put(_Fields.EXTERNAL_COMPACTION_ID, new org.apache.thrift.meta_data.FieldMetaData("externalCompactionId", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+      metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(getCompactionJob_args.class, metaDataMap);
+    }
+
+    public getCompactionJob_args() {
+    }
+
+    public getCompactionJob_args(
+      org.apache.accumulo.core.trace.thrift.TInfo tinfo,
+      org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials,
+      java.lang.String queueName,
+      java.lang.String compactor,
+      java.lang.String externalCompactionId)
+    {
+      this();
+      this.tinfo = tinfo;
+      this.credentials = credentials;
+      this.queueName = queueName;
+      this.compactor = compactor;
+      this.externalCompactionId = externalCompactionId;
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public getCompactionJob_args(getCompactionJob_args other) {
+      if (other.isSetTinfo()) {
+        this.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo(other.tinfo);
+      }
+      if (other.isSetCredentials()) {
+        this.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(other.credentials);
+      }
+      if (other.isSetQueueName()) {
+        this.queueName = other.queueName;
+      }
+      if (other.isSetCompactor()) {
+        this.compactor = other.compactor;
+      }
+      if (other.isSetExternalCompactionId()) {
+        this.externalCompactionId = other.externalCompactionId;
+      }
+    }
+
+    public getCompactionJob_args deepCopy() {
+      return new getCompactionJob_args(this);
+    }
+
+    @Override
+    public void clear() {
+      this.tinfo = null;
+      this.credentials = null;
+      this.queueName = null;
+      this.compactor = null;
+      this.externalCompactionId = null;
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public org.apache.accumulo.core.trace.thrift.TInfo getTinfo() {
+      return this.tinfo;
+    }
+
+    public getCompactionJob_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo) {
+      this.tinfo = tinfo;
+      return this;
+    }
+
+    public void unsetTinfo() {
+      this.tinfo = null;
+    }
+
+    /** Returns true if field tinfo is set (has been assigned a value) and false otherwise */
+    public boolean isSetTinfo() {
+      return this.tinfo != null;
+    }
+
+    public void setTinfoIsSet(boolean value) {
+      if (!value) {
+        this.tinfo = null;
+      }
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public org.apache.accumulo.core.securityImpl.thrift.TCredentials getCredentials() {
+      return this.credentials;
+    }
+
+    public getCompactionJob_args setCredentials(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) {
+      this.credentials = credentials;
+      return this;
+    }
+
+    public void unsetCredentials() {
+      this.credentials = null;
+    }
+
+    /** Returns true if field credentials is set (has been assigned a value) and false otherwise */
+    public boolean isSetCredentials() {
+      return this.credentials != null;
+    }
+
+    public void setCredentialsIsSet(boolean value) {
+      if (!value) {
+        this.credentials = null;
+      }
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public java.lang.String getQueueName() {
+      return this.queueName;
+    }
+
+    public getCompactionJob_args setQueueName(@org.apache.thrift.annotation.Nullable java.lang.String queueName) {
+      this.queueName = queueName;
+      return this;
+    }
+
+    public void unsetQueueName() {
+      this.queueName = null;
+    }
+
+    /** Returns true if field queueName is set (has been assigned a value) and false otherwise */
+    public boolean isSetQueueName() {
+      return this.queueName != null;
+    }
+
+    public void setQueueNameIsSet(boolean value) {
+      if (!value) {
+        this.queueName = null;
+      }
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public java.lang.String getCompactor() {
+      return this.compactor;
+    }
+
+    public getCompactionJob_args setCompactor(@org.apache.thrift.annotation.Nullable java.lang.String compactor) {
+      this.compactor = compactor;
+      return this;
+    }
+
+    public void unsetCompactor() {
+      this.compactor = null;
+    }
+
+    /** Returns true if field compactor is set (has been assigned a value) and false otherwise */
+    public boolean isSetCompactor() {
+      return this.compactor != null;
+    }
+
+    public void setCompactorIsSet(boolean value) {
+      if (!value) {
+        this.compactor = null;
+      }
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public java.lang.String getExternalCompactionId() {
+      return this.externalCompactionId;
+    }
+
+    public getCompactionJob_args setExternalCompactionId(@org.apache.thrift.annotation.Nullable java.lang.String externalCompactionId) {
+      this.externalCompactionId = externalCompactionId;
+      return this;
+    }
+
+    public void unsetExternalCompactionId() {
+      this.externalCompactionId = null;
+    }
+
+    /** Returns true if field externalCompactionId is set (has been assigned a value) and false otherwise */
+    public boolean isSetExternalCompactionId() {
+      return this.externalCompactionId != null;
+    }
+
+    public void setExternalCompactionIdIsSet(boolean value) {
+      if (!value) {
+        this.externalCompactionId = null;
+      }
+    }
+
+    public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable java.lang.Object value) {
+      switch (field) {
+      case TINFO:
+        if (value == null) {
+          unsetTinfo();
+        } else {
+          setTinfo((org.apache.accumulo.core.trace.thrift.TInfo)value);
+        }
+        break;
+
+      case CREDENTIALS:
+        if (value == null) {
+          unsetCredentials();
+        } else {
+          setCredentials((org.apache.accumulo.core.securityImpl.thrift.TCredentials)value);
+        }
+        break;
+
+      case QUEUE_NAME:
+        if (value == null) {
+          unsetQueueName();
+        } else {
+          setQueueName((java.lang.String)value);
+        }
+        break;
+
+      case COMPACTOR:
+        if (value == null) {
+          unsetCompactor();
+        } else {
+          setCompactor((java.lang.String)value);
+        }
+        break;
+
+      case EXTERNAL_COMPACTION_ID:
+        if (value == null) {
+          unsetExternalCompactionId();
+        } else {
+          setExternalCompactionId((java.lang.String)value);
+        }
+        break;
+
+      }
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public java.lang.Object getFieldValue(_Fields field) {
+      switch (field) {
+      case TINFO:
+        return getTinfo();
+
+      case CREDENTIALS:
+        return getCredentials();
+
+      case QUEUE_NAME:
+        return getQueueName();
+
+      case COMPACTOR:
+        return getCompactor();
+
+      case EXTERNAL_COMPACTION_ID:
+        return getExternalCompactionId();
+
+      }
+      throw new java.lang.IllegalStateException();
+    }
+
+    /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+    public boolean isSet(_Fields field) {
+      if (field == null) {
+        throw new java.lang.IllegalArgumentException();
+      }
+
+      switch (field) {
+      case TINFO:
+        return isSetTinfo();
+      case CREDENTIALS:
+        return isSetCredentials();
+      case QUEUE_NAME:
+        return isSetQueueName();
+      case COMPACTOR:
+        return isSetCompactor();
+      case EXTERNAL_COMPACTION_ID:
+        return isSetExternalCompactionId();
+      }
+      throw new java.lang.IllegalStateException();
+    }
+
+    @Override
+    public boolean equals(java.lang.Object that) {
+      if (that == null)
+        return false;
+      if (that instanceof getCompactionJob_args)
+        return this.equals((getCompactionJob_args)that);
+      return false;
+    }
+
+    public boolean equals(getCompactionJob_args that) {
+      if (that == null)
+        return false;
+      if (this == that)
+        return true;
+
+      boolean this_present_tinfo = true && this.isSetTinfo();
+      boolean that_present_tinfo = true && that.isSetTinfo();
+      if (this_present_tinfo || that_present_tinfo) {
+        if (!(this_present_tinfo && that_present_tinfo))
+          return false;
+        if (!this.tinfo.equals(that.tinfo))
+          return false;
+      }
+
+      boolean this_present_credentials = true && this.isSetCredentials();
+      boolean that_present_credentials = true && that.isSetCredentials();
+      if (this_present_credentials || that_present_credentials) {
+        if (!(this_present_credentials && that_present_credentials))
+          return false;
+        if (!this.credentials.equals(that.credentials))
+          return false;
+      }
+
+      boolean this_present_queueName = true && this.isSetQueueName();
+      boolean that_present_queueName = true && that.isSetQueueName();
+      if (this_present_queueName || that_present_queueName) {
+        if (!(this_present_queueName && that_present_queueName))
+          return false;
+        if (!this.queueName.equals(that.queueName))
+          return false;
+      }
+
+      boolean this_present_compactor = true && this.isSetCompactor();
+      boolean that_present_compactor = true && that.isSetCompactor();
+      if (this_present_compactor || that_present_compactor) {
+        if (!(this_present_compactor && that_present_compactor))
+          return false;
+        if (!this.compactor.equals(that.compactor))
+          return false;
+      }
+
+      boolean this_present_externalCompactionId = true && this.isSetExternalCompactionId();
+      boolean that_present_externalCompactionId = true && that.isSetExternalCompactionId();
+      if (this_present_externalCompactionId || that_present_externalCompactionId) {
+        if (!(this_present_externalCompactionId && that_present_externalCompactionId))
+          return false;
+        if (!this.externalCompactionId.equals(that.externalCompactionId))
+          return false;
+      }
+
+      return true;
+    }
+
+    @Override
+    public int hashCode() {
+      int hashCode = 1;
+
+      hashCode = hashCode * 8191 + ((isSetTinfo()) ? 131071 : 524287);
+      if (isSetTinfo())
+        hashCode = hashCode * 8191 + tinfo.hashCode();
+
+      hashCode = hashCode * 8191 + ((isSetCredentials()) ? 131071 : 524287);
+      if (isSetCredentials())
+        hashCode = hashCode * 8191 + credentials.hashCode();
+
+      hashCode = hashCode * 8191 + ((isSetQueueName()) ? 131071 : 524287);
+      if (isSetQueueName())
+        hashCode = hashCode * 8191 + queueName.hashCode();
+
+      hashCode = hashCode * 8191 + ((isSetCompactor()) ? 131071 : 524287);
+      if (isSetCompactor())
+        hashCode = hashCode * 8191 + compactor.hashCode();
+
+      hashCode = hashCode * 8191 + ((isSetExternalCompactionId()) ? 131071 : 524287);
+      if (isSetExternalCompactionId())
+        hashCode = hashCode * 8191 + externalCompactionId.hashCode();
+
+      return hashCode;
+    }
+
+    @Override
+    public int compareTo(getCompactionJob_args other) {
+      if (!getClass().equals(other.getClass())) {
+        return getClass().getName().compareTo(other.getClass().getName());
+      }
+
+      int lastComparison = 0;
+
+      lastComparison = java.lang.Boolean.valueOf(isSetTinfo()).compareTo(other.isSetTinfo());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetTinfo()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tinfo, other.tinfo);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      lastComparison = java.lang.Boolean.valueOf(isSetCredentials()).compareTo(other.isSetCredentials());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetCredentials()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.credentials, other.credentials);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      lastComparison = java.lang.Boolean.valueOf(isSetQueueName()).compareTo(other.isSetQueueName());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetQueueName()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.queueName, other.queueName);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      lastComparison = java.lang.Boolean.valueOf(isSetCompactor()).compareTo(other.isSetCompactor());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetCompactor()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.compactor, other.compactor);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      lastComparison = java.lang.Boolean.valueOf(isSetExternalCompactionId()).compareTo(other.isSetExternalCompactionId());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetExternalCompactionId()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.externalCompactionId, other.externalCompactionId);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      return 0;
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public _Fields fieldForId(int fieldId) {
+      return _Fields.findByThriftId(fieldId);
+    }
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+      scheme(iprot).read(iprot, this);
+    }
+
+    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("getCompactionJob_args(");
+      boolean first = true;
+
+      sb.append("tinfo:");
+      if (this.tinfo == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.tinfo);
+      }
+      first = false;
+      if (!first) sb.append(", ");
+      sb.append("credentials:");
+      if (this.credentials == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.credentials);
+      }
+      first = false;
+      if (!first) sb.append(", ");
+      sb.append("queueName:");
+      if (this.queueName == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.queueName);
+      }
+      first = false;
+      if (!first) sb.append(", ");
+      sb.append("compactor:");
+      if (this.compactor == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.compactor);
+      }
+      first = false;
+      if (!first) sb.append(", ");
+      sb.append("externalCompactionId:");
+      if (this.externalCompactionId == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.externalCompactionId);
+      }
+      first = false;
+      sb.append(")");
+      return sb.toString();
+    }
+
+    public void validate() throws org.apache.thrift.TException {
+      // check for required fields
+      // check for sub-struct validity
+      if (tinfo != null) {
+        tinfo.validate();
+      }
+      if (credentials != null) {
+        credentials.validate();
+      }
+    }
+
+    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 {
+        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 getCompactionJob_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public getCompactionJob_argsStandardScheme getScheme() {
+        return new getCompactionJob_argsStandardScheme();
+      }
+    }
+
+    private static class getCompactionJob_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme<getCompactionJob_args> {
+
+      public void read(org.apache.thrift.protocol.TProtocol iprot, getCompactionJob_args 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: // TINFO
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+                struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+                struct.tinfo.read(iprot);
+                struct.setTinfoIsSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            case 2: // CREDENTIALS
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+                struct.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials();
+                struct.credentials.read(iprot);
+                struct.setCredentialsIsSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            case 3: // QUEUE_NAME
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+                struct.queueName = iprot.readString();
+                struct.setQueueNameIsSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            case 4: // COMPACTOR
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+                struct.compactor = iprot.readString();
+                struct.setCompactorIsSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            case 5: // EXTERNAL_COMPACTION_ID
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+                struct.externalCompactionId = iprot.readString();
+                struct.setExternalCompactionIdIsSet(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();
+      }
+
+      public void write(org.apache.thrift.protocol.TProtocol oprot, getCompactionJob_args struct) throws org.apache.thrift.TException {
+        struct.validate();
+
+        oprot.writeStructBegin(STRUCT_DESC);
+        if (struct.tinfo != null) {
+          oprot.writeFieldBegin(TINFO_FIELD_DESC);
+          struct.tinfo.write(oprot);
+          oprot.writeFieldEnd();
+        }
+        if (struct.credentials != null) {
+          oprot.writeFieldBegin(CREDENTIALS_FIELD_DESC);
+          struct.credentials.write(oprot);
+          oprot.writeFieldEnd();
+        }
+        if (struct.queueName != null) {
+          oprot.writeFieldBegin(QUEUE_NAME_FIELD_DESC);
+          oprot.writeString(struct.queueName);
+          oprot.writeFieldEnd();
+        }
+        if (struct.compactor != null) {
+          oprot.writeFieldBegin(COMPACTOR_FIELD_DESC);
+          oprot.writeString(struct.compactor);
+          oprot.writeFieldEnd();
+        }
+        if (struct.externalCompactionId != null) {
+          oprot.writeFieldBegin(EXTERNAL_COMPACTION_ID_FIELD_DESC);
+          oprot.writeString(struct.externalCompactionId);
+          oprot.writeFieldEnd();
+        }
+        oprot.writeFieldStop();
+        oprot.writeStructEnd();
+      }
+
+    }
+
+    private static class getCompactionJob_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public getCompactionJob_argsTupleScheme getScheme() {
+        return new getCompactionJob_argsTupleScheme();
+      }
+    }
+
+    private static class getCompactionJob_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme<getCompactionJob_args> {
+
+      @Override
+      public void write(org.apache.thrift.protocol.TProtocol prot, getCompactionJob_args 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.isSetTinfo()) {
+          optionals.set(0);
+        }
+        if (struct.isSetCredentials()) {
+          optionals.set(1);
+        }
+        if (struct.isSetQueueName()) {
+          optionals.set(2);
+        }
+        if (struct.isSetCompactor()) {
+          optionals.set(3);
+        }
+        if (struct.isSetExternalCompactionId()) {
+          optionals.set(4);
+        }
+        oprot.writeBitSet(optionals, 5);
+        if (struct.isSetTinfo()) {
+          struct.tinfo.write(oprot);
+        }
+        if (struct.isSetCredentials()) {
+          struct.credentials.write(oprot);
+        }
+        if (struct.isSetQueueName()) {
+          oprot.writeString(struct.queueName);
+        }
+        if (struct.isSetCompactor()) {
+          oprot.writeString(struct.compactor);
+        }
+        if (struct.isSetExternalCompactionId()) {
+          oprot.writeString(struct.externalCompactionId);
+        }
+      }
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol prot, getCompactionJob_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(5);
+        if (incoming.get(0)) {
+          struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+          struct.tinfo.read(iprot);
+          struct.setTinfoIsSet(true);
+        }
+        if (incoming.get(1)) {
+          struct.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials();
+          struct.credentials.read(iprot);
+          struct.setCredentialsIsSet(true);
+        }
+        if (incoming.get(2)) {
+          struct.queueName = iprot.readString();
+          struct.setQueueNameIsSet(true);
+        }
+        if (incoming.get(3)) {
+          struct.compactor = iprot.readString();
+          struct.setCompactorIsSet(true);
+        }
+        if (incoming.get(4)) {
+          struct.externalCompactionId = iprot.readString();
+          struct.setExternalCompactionIdIsSet(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();
+    }
+  }
+
+  public static class getCompactionJob_result implements org.apache.thrift.TBase<getCompactionJob_result, getCompactionJob_result._Fields>, java.io.Serializable, Cloneable, Comparable<getCompactionJob_result>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getCompactionJob_result");
+
+    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.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new getCompactionJob_resultStandardSchemeFactory();
+    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new getCompactionJob_resultTupleSchemeFactory();
+
+    public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.tabletserver.thrift.TExternalCompactionJob success; // 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 {
+      SUCCESS((short)0, "success");
+
+      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 0: // SUCCESS
+            return SUCCESS;
+          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;
+      }
+
+      public short getThriftFieldId() {
+        return _thriftId;
+      }
+
+      public java.lang.String getFieldName() {
+        return _fieldName;
+      }
+    }
+
+    // isset id assignments
+    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.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.tabletserver.thrift.TExternalCompactionJob.class)));
+      metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(getCompactionJob_result.class, metaDataMap);
+    }
+
+    public getCompactionJob_result() {
+    }
+
+    public getCompactionJob_result(
+      org.apache.accumulo.core.tabletserver.thrift.TExternalCompactionJob success)
+    {
+      this();
+      this.success = success;
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public getCompactionJob_result(getCompactionJob_result other) {
+      if (other.isSetSuccess()) {
+        this.success = new org.apache.accumulo.core.tabletserver.thrift.TExternalCompactionJob(other.success);
+      }
+    }
+
+    public getCompactionJob_result deepCopy() {
+      return new getCompactionJob_result(this);
+    }
+
+    @Override
+    public void clear() {
+      this.success = null;
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public org.apache.accumulo.core.tabletserver.thrift.TExternalCompactionJob getSuccess() {
+      return this.success;
+    }
+
+    public getCompactionJob_result setSuccess(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.tabletserver.thrift.TExternalCompactionJob success) {
+      this.success = success;
+      return this;
+    }
+
+    public void unsetSuccess() {
+      this.success = null;
+    }
+
+    /** Returns true if field success is set (has been assigned a value) and false otherwise */
+    public boolean isSetSuccess() {
+      return this.success != null;
+    }
+
+    public void setSuccessIsSet(boolean value) {
+      if (!value) {
+        this.success = null;
+      }
+    }
+
+    public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable java.lang.Object value) {
+      switch (field) {
+      case SUCCESS:
+        if (value == null) {
+          unsetSuccess();
+        } else {
+          setSuccess((org.apache.accumulo.core.tabletserver.thrift.TExternalCompactionJob)value);
+        }
+        break;
+
+      }
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public java.lang.Object getFieldValue(_Fields field) {
+      switch (field) {
+      case SUCCESS:
+        return getSuccess();
+
+      }
+      throw new java.lang.IllegalStateException();
+    }
+
+    /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+    public boolean isSet(_Fields field) {
+      if (field == null) {
+        throw new java.lang.IllegalArgumentException();
+      }
+
+      switch (field) {
+      case SUCCESS:
+        return isSetSuccess();
+      }
+      throw new java.lang.IllegalStateException();
+    }
+
+    @Override
+    public boolean equals(java.lang.Object that) {
+      if (that == null)
+        return false;
+      if (that instanceof getCompactionJob_result)
+        return this.equals((getCompactionJob_result)that);
+      return false;
+    }
+
+    public boolean equals(getCompactionJob_result that) {
+      if (that == null)
+        return false;
+      if (this == that)
+        return 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.equals(that.success))
+          return false;
+      }
+
+      return true;
+    }
+
+    @Override
+    public int hashCode() {
+      int hashCode = 1;
+
+      hashCode = hashCode * 8191 + ((isSetSuccess()) ? 131071 : 524287);
+      if (isSetSuccess())
+        hashCode = hashCode * 8191 + success.hashCode();
+
+      return hashCode;
+    }
+
+    @Override
+    public int compareTo(getCompactionJob_result other) {
+      if (!getClass().equals(other.getClass())) {
+        return getClass().getName().compareTo(other.getClass().getName());
+      }
+
+      int lastComparison = 0;
+
+      lastComparison = java.lang.Boolean.valueOf(isSetSuccess()).compareTo(other.isSetSuccess());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetSuccess()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, other.success);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      return 0;
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public _Fields fieldForId(int fieldId) {
+      return _Fields.findByThriftId(fieldId);
+    }
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+      scheme(iprot).read(iprot, this);
+    }
+
+    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("getCompactionJob_result(");
+      boolean first = true;
+
+      sb.append("success:");
+      if (this.success == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.success);
+      }
+      first = false;
+      sb.append(")");
+      return sb.toString();
+    }
+
+    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 {
+      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 {
+        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 getCompactionJob_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public getCompactionJob_resultStandardScheme getScheme() {
+        return new getCompactionJob_resultStandardScheme();
+      }
+    }
+
+    private static class getCompactionJob_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme<getCompactionJob_result> {
+
+      public void read(org.apache.thrift.protocol.TProtocol iprot, getCompactionJob_result 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 0: // SUCCESS
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+                struct.success = new org.apache.accumulo.core.tabletserver.thrift.TExternalCompactionJob();
+                struct.success.read(iprot);
+                struct.setSuccessIsSet(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();
+      }
+
+      public void write(org.apache.thrift.protocol.TProtocol oprot, getCompactionJob_result struct) throws org.apache.thrift.TException {
+        struct.validate();
+
+        oprot.writeStructBegin(STRUCT_DESC);
+        if (struct.success != null) {
+          oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
+          struct.success.write(oprot);
+          oprot.writeFieldEnd();
+        }
+        oprot.writeFieldStop();
+        oprot.writeStructEnd();
+      }
+
+    }
+
+    private static class getCompactionJob_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public getCompactionJob_resultTupleScheme getScheme() {
+        return new getCompactionJob_resultTupleScheme();
+      }
+    }
+
+    private static class getCompactionJob_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme<getCompactionJob_result> {
+
+      @Override
+      public void write(org.apache.thrift.protocol.TProtocol prot, getCompactionJob_result 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.isSetSuccess()) {
+          optionals.set(0);
+        }
+        oprot.writeBitSet(optionals, 1);
+        if (struct.isSetSuccess()) {
+          struct.success.write(oprot);
+        }
+      }
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol prot, getCompactionJob_result struct) throws org.apache.thrift.TException {
+        org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
+        java.util.BitSet incoming = iprot.readBitSet(1);
+        if (incoming.get(0)) {
+          struct.success = new org.apache.accumulo.core.tabletserver.thrift.TExternalCompactionJob();
+          struct.success.read(iprot);
+          struct.setSuccessIsSet(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();
+    }
+  }
+
+  public static class updateCompactionStatus_args implements org.apache.thrift.TBase<updateCompactionStatus_args, updateCompactionStatus_args._Fields>, java.io.Serializable, Cloneable, Comparable<updateCompactionStatus_args>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("updateCompactionStatus_args");
+
+    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 EXTERNAL_COMPACTION_ID_FIELD_DESC = new org.apache.thrift.protocol.TField("externalCompactionId", org.apache.thrift.protocol.TType.STRING, (short)3);
+    private static final org.apache.thrift.protocol.TField STATE_FIELD_DESC = new org.apache.thrift.protocol.TField("state", org.apache.thrift.protocol.TType.I32, (short)4);
+    private static final org.apache.thrift.protocol.TField MESSAGE_FIELD_DESC = new org.apache.thrift.protocol.TField("message", org.apache.thrift.protocol.TType.STRING, (short)5);
+    private static final org.apache.thrift.protocol.TField TIMESTAMP_FIELD_DESC = new org.apache.thrift.protocol.TField("timestamp", org.apache.thrift.protocol.TType.I64, (short)6);
+
+    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new updateCompactionStatus_argsStandardSchemeFactory();
+    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new updateCompactionStatus_argsTupleSchemeFactory();
+
+    public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo; // required
+    public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials; // required
+    public @org.apache.thrift.annotation.Nullable java.lang.String externalCompactionId; // required
+    /**
+     * 
+     * @see TCompactionState
+     */
+    public @org.apache.thrift.annotation.Nullable TCompactionState state; // required
+    public @org.apache.thrift.annotation.Nullable java.lang.String message; // required
+    public long timestamp; // 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"),
+      EXTERNAL_COMPACTION_ID((short)3, "externalCompactionId"),
+      /**
+       * 
+       * @see TCompactionState
+       */
+      STATE((short)4, "state"),
+      MESSAGE((short)5, "message"),
+      TIMESTAMP((short)6, "timestamp");
+
+      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: // TINFO
+            return TINFO;
+          case 2: // CREDENTIALS
+            return CREDENTIALS;
+          case 3: // EXTERNAL_COMPACTION_ID
+            return EXTERNAL_COMPACTION_ID;
+          case 4: // STATE
+            return STATE;
+          case 5: // MESSAGE
+            return MESSAGE;
+          case 6: // TIMESTAMP
+            return TIMESTAMP;
+          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;
+      }
+
+      public short getThriftFieldId() {
+        return _thriftId;
+      }
+
+      public java.lang.String getFieldName() {
+        return _fieldName;
+      }
+    }
+
+    // isset id assignments
+    private static final int __TIMESTAMP_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.TINFO, new org.apache.thrift.meta_data.FieldMetaData("tinfo", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.trace.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.EXTERNAL_COMPACTION_ID, new org.apache.thrift.meta_data.FieldMetaData("externalCompactionId", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+      tmpMap.put(_Fields.STATE, new org.apache.thrift.meta_data.FieldMetaData("state", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.EnumMetaData(org.apache.thrift.protocol.TType.ENUM, TCompactionState.class)));
+      tmpMap.put(_Fields.MESSAGE, new org.apache.thrift.meta_data.FieldMetaData("message", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+      tmpMap.put(_Fields.TIMESTAMP, new org.apache.thrift.meta_data.FieldMetaData("timestamp", 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(updateCompactionStatus_args.class, metaDataMap);
+    }
+
+    public updateCompactionStatus_args() {
+    }
+
+    public updateCompactionStatus_args(
+      org.apache.accumulo.core.trace.thrift.TInfo tinfo,
+      org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials,
+      java.lang.String externalCompactionId,
+      TCompactionState state,
+      java.lang.String message,
+      long timestamp)
+    {
+      this();
+      this.tinfo = tinfo;
+      this.credentials = credentials;
+      this.externalCompactionId = externalCompactionId;
+      this.state = state;
+      this.message = message;
+      this.timestamp = timestamp;
+      setTimestampIsSet(true);
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public updateCompactionStatus_args(updateCompactionStatus_args other) {
+      __isset_bitfield = other.__isset_bitfield;
+      if (other.isSetTinfo()) {
+        this.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo(other.tinfo);
+      }
+      if (other.isSetCredentials()) {
+        this.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(other.credentials);
+      }
+      if (other.isSetExternalCompactionId()) {
+        this.externalCompactionId = other.externalCompactionId;
+      }
+      if (other.isSetState()) {
+        this.state = other.state;
+      }
+      if (other.isSetMessage()) {
+        this.message = other.message;
+      }
+      this.timestamp = other.timestamp;
+    }
+
+    public updateCompactionStatus_args deepCopy() {
+      return new updateCompactionStatus_args(this);
+    }
+
+    @Override
+    public void clear() {
+      this.tinfo = null;
+      this.credentials = null;
+      this.externalCompactionId = null;
+      this.state = null;
+      this.message = null;
+      setTimestampIsSet(false);
+      this.timestamp = 0;
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public org.apache.accumulo.core.trace.thrift.TInfo getTinfo() {
+      return this.tinfo;
+    }
+
+    public updateCompactionStatus_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo) {
+      this.tinfo = tinfo;
+      return this;
+    }
+
+    public void unsetTinfo() {
+      this.tinfo = null;
+    }
+
+    /** Returns true if field tinfo is set (has been assigned a value) and false otherwise */
+    public boolean isSetTinfo() {
+      return this.tinfo != null;
+    }
+
+    public void setTinfoIsSet(boolean value) {
+      if (!value) {
+        this.tinfo = null;
+      }
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public org.apache.accumulo.core.securityImpl.thrift.TCredentials getCredentials() {
+      return this.credentials;
+    }
+
+    public updateCompactionStatus_args setCredentials(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) {
+      this.credentials = credentials;
+      return this;
+    }
+
+    public void unsetCredentials() {
+      this.credentials = null;
+    }
+
+    /** Returns true if field credentials is set (has been assigned a value) and false otherwise */
+    public boolean isSetCredentials() {
+      return this.credentials != null;
+    }
+
+    public void setCredentialsIsSet(boolean value) {
+      if (!value) {
+        this.credentials = null;
+      }
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public java.lang.String getExternalCompactionId() {
+      return this.externalCompactionId;
+    }
+
+    public updateCompactionStatus_args setExternalCompactionId(@org.apache.thrift.annotation.Nullable java.lang.String externalCompactionId) {
+      this.externalCompactionId = externalCompactionId;
+      return this;
+    }
+
+    public void unsetExternalCompactionId() {
+      this.externalCompactionId = null;
+    }
+
+    /** Returns true if field externalCompactionId is set (has been assigned a value) and false otherwise */
+    public boolean isSetExternalCompactionId() {
+      return this.externalCompactionId != null;
+    }
+
+    public void setExternalCompactionIdIsSet(boolean value) {
+      if (!value) {
+        this.externalCompactionId = null;
+      }
+    }
+
+    /**
+     * 
+     * @see TCompactionState
+     */
+    @org.apache.thrift.annotation.Nullable
+    public TCompactionState getState() {
+      return this.state;
+    }
+
+    /**
+     * 
+     * @see TCompactionState
+     */
+    public updateCompactionStatus_args setState(@org.apache.thrift.annotation.Nullable TCompactionState state) {
+      this.state = state;
+      return this;
+    }
+
+    public void unsetState() {
+      this.state = null;
+    }
+
+    /** Returns true if field state is set (has been assigned a value) and false otherwise */
+    public boolean isSetState() {
+      return this.state != null;
+    }
+
+    public void setStateIsSet(boolean value) {
+      if (!value) {
+        this.state = null;
+      }
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public java.lang.String getMessage() {
+      return this.message;
+    }
+
+    public updateCompactionStatus_args setMessage(@org.apache.thrift.annotation.Nullable java.lang.String message) {
+      this.message = message;
+      return this;
+    }
+
+    public void unsetMessage() {
+      this.message = null;
+    }
+
+    /** Returns true if field message is set (has been assigned a value) and false otherwise */
+    public boolean isSetMessage() {
+      return this.message != null;
+    }
+
+    public void setMessageIsSet(boolean value) {
+      if (!value) {
+        this.message = null;
+      }
+    }
+
+    public long getTimestamp() {
+      return this.timestamp;
+    }
+
+    public updateCompactionStatus_args setTimestamp(long timestamp) {
+      this.timestamp = timestamp;
+      setTimestampIsSet(true);
+      return this;
+    }
+
+    public void unsetTimestamp() {
+      __isset_bitfield = org.apache.thrift.EncodingUtils.clearBit(__isset_bitfield, __TIMESTAMP_ISSET_ID);
+    }
+
+    /** Returns true if field timestamp is set (has been assigned a value) and false otherwise */
+    public boolean isSetTimestamp() {
+      return org.apache.thrift.EncodingUtils.testBit(__isset_bitfield, __TIMESTAMP_ISSET_ID);
+    }
+
+    public void setTimestampIsSet(boolean value) {
+      __isset_bitfield = org.apache.thrift.EncodingUtils.setBit(__isset_bitfield, __TIMESTAMP_ISSET_ID, value);
+    }
+
+    public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable java.lang.Object value) {
+      switch (field) {
+      case TINFO:
+        if (value == null) {
+          unsetTinfo();
+        } else {
+          setTinfo((org.apache.accumulo.core.trace.thrift.TInfo)value);
+        }
+        break;
+
+      case CREDENTIALS:
+        if (value == null) {
+          unsetCredentials();
+        } else {
+          setCredentials((org.apache.accumulo.core.securityImpl.thrift.TCredentials)value);
+        }
+        break;
+
+      case EXTERNAL_COMPACTION_ID:
+        if (value == null) {
+          unsetExternalCompactionId();
+        } else {
+          setExternalCompactionId((java.lang.String)value);
+        }
+        break;
+
+      case STATE:
+        if (value == null) {
+          unsetState();
+        } else {
+          setState((TCompactionState)value);
+        }
+        break;
+
+      case MESSAGE:
+        if (value == null) {
+          unsetMessage();
+        } else {
+          setMessage((java.lang.String)value);
+        }
+        break;
+
+      case TIMESTAMP:
+        if (value == null) {
+          unsetTimestamp();
+        } else {
+          setTimestamp((java.lang.Long)value);
+        }
+        break;
+
+      }
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public java.lang.Object getFieldValue(_Fields field) {
+      switch (field) {
+      case TINFO:
+        return getTinfo();
+
+      case CREDENTIALS:
+        return getCredentials();
+
+      case EXTERNAL_COMPACTION_ID:
+        return getExternalCompactionId();
+
+      case STATE:
+        return getState();
+
+      case MESSAGE:
+        return getMessage();
+
+      case TIMESTAMP:
+        return getTimestamp();
+
+      }
+      throw new java.lang.IllegalStateException();
+    }
+
+    /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+    public boolean isSet(_Fields field) {
+      if (field == null) {
+        throw new java.lang.IllegalArgumentException();
+      }
+
+      switch (field) {
+      case TINFO:
+        return isSetTinfo();
+      case CREDENTIALS:
+        return isSetCredentials();
+      case EXTERNAL_COMPACTION_ID:
+        return isSetExternalCompactionId();
+      case STATE:
+        return isSetState();
+      case MESSAGE:
+        return isSetMessage();
+      case TIMESTAMP:
+        return isSetTimestamp();
+      }
+      throw new java.lang.IllegalStateException();
+    }
+
+    @Override
+    public boolean equals(java.lang.Object that) {
+      if (that == null)
+        return false;
+      if (that instanceof updateCompactionStatus_args)
+        return this.equals((updateCompactionStatus_args)that);
+      return false;
+    }
+
+    public boolean equals(updateCompactionStatus_args that) {
+      if (that == null)
+        return false;
+      if (this == that)
+        return true;
+
+      boolean this_present_tinfo = true && this.isSetTinfo();
+      boolean that_present_tinfo = true && that.isSetTinfo();
+      if (this_present_tinfo || that_present_tinfo) {
+        if (!(this_present_tinfo && that_present_tinfo))
+          return false;
+        if (!this.tinfo.equals(that.tinfo))
+          return false;
+      }
+
+      boolean this_present_credentials = true && this.isSetCredentials();
+      boolean that_present_credentials = true && that.isSetCredentials();
+      if (this_present_credentials || that_present_credentials) {
+        if (!(this_present_credentials && that_present_credentials))
+          return false;
+        if (!this.credentials.equals(that.credentials))
+          return false;
+      }
+
+      boolean this_present_externalCompactionId = true && this.isSetExternalCompactionId();
+      boolean that_present_externalCompactionId = true && that.isSetExternalCompactionId();
+      if (this_present_externalCompactionId || that_present_externalCompactionId) {
+        if (!(this_present_externalCompactionId && that_present_externalCompactionId))
+          return false;
+        if (!this.externalCompactionId.equals(that.externalCompactionId))
+          return false;
+      }
+
+      boolean this_present_state = true && this.isSetState();
+      boolean that_present_state = true && that.isSetState();
+      if (this_present_state || that_present_state) {
+        if (!(this_present_state && that_present_state))
+          return false;
+        if (!this.state.equals(that.state))
+          return false;
+      }
+
+      boolean this_present_message = true && this.isSetMessage();
+      boolean that_present_message = true && that.isSetMessage();
+      if (this_present_message || that_present_message) {
+        if (!(this_present_message && that_present_message))
+          return false;
+        if (!this.message.equals(that.message))
+          return false;
+      }
+
+      boolean this_present_timestamp = true;
+      boolean that_present_timestamp = true;
+      if (this_present_timestamp || that_present_timestamp) {
+        if (!(this_present_timestamp && that_present_timestamp))
+          return false;
+        if (this.timestamp != that.timestamp)
+          return false;
+      }
+
+      return true;
+    }
+
+    @Override
+    public int hashCode() {
+      int hashCode = 1;
+
+      hashCode = hashCode * 8191 + ((isSetTinfo()) ? 131071 : 524287);
+      if (isSetTinfo())
+        hashCode = hashCode * 8191 + tinfo.hashCode();
+
+      hashCode = hashCode * 8191 + ((isSetCredentials()) ? 131071 : 524287);
+      if (isSetCredentials())
+        hashCode = hashCode * 8191 + credentials.hashCode();
+
+      hashCode = hashCode * 8191 + ((isSetExternalCompactionId()) ? 131071 : 524287);
+      if (isSetExternalCompactionId())
+        hashCode = hashCode * 8191 + externalCompactionId.hashCode();
+
+      hashCode = hashCode * 8191 + ((isSetState()) ? 131071 : 524287);
+      if (isSetState())
+        hashCode = hashCode * 8191 + state.getValue();
+
+      hashCode = hashCode * 8191 + ((isSetMessage()) ? 131071 : 524287);
+      if (isSetMessage())
+        hashCode = hashCode * 8191 + message.hashCode();
+
+      hashCode = hashCode * 8191 + org.apache.thrift.TBaseHelper.hashCode(timestamp);
+
+      return hashCode;
+    }
+
+    @Override
+    public int compareTo(updateCompactionStatus_args other) {
+      if (!getClass().equals(other.getClass())) {
+        return getClass().getName().compareTo(other.getClass().getName());
+      }
+
+      int lastComparison = 0;
+
+      lastComparison = java.lang.Boolean.valueOf(isSetTinfo()).compareTo(other.isSetTinfo());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetTinfo()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tinfo, other.tinfo);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      lastComparison = java.lang.Boolean.valueOf(isSetCredentials()).compareTo(other.isSetCredentials());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetCredentials()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.credentials, other.credentials);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      lastComparison = java.lang.Boolean.valueOf(isSetExternalCompactionId()).compareTo(other.isSetExternalCompactionId());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetExternalCompactionId()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.externalCompactionId, other.externalCompactionId);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      lastComparison = java.lang.Boolean.valueOf(isSetState()).compareTo(other.isSetState());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetState()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.state, other.state);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      lastComparison = java.lang.Boolean.valueOf(isSetMessage()).compareTo(other.isSetMessage());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetMessage()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.message, other.message);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      lastComparison = java.lang.Boolean.valueOf(isSetTimestamp()).compareTo(other.isSetTimestamp());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetTimestamp()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.timestamp, other.timestamp);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      return 0;
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public _Fields fieldForId(int fieldId) {
+      return _Fields.findByThriftId(fieldId);
+    }
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+      scheme(iprot).read(iprot, this);
+    }
+
+    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("updateCompactionStatus_args(");
+      boolean first = true;
+
+      sb.append("tinfo:");
+      if (this.tinfo == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.tinfo);
+      }
+      first = false;
+      if (!first) sb.append(", ");
+      sb.append("credentials:");
+      if (this.credentials == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.credentials);
+      }
+      first = false;
+      if (!first) sb.append(", ");
+      sb.append("externalCompactionId:");
+      if (this.externalCompactionId == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.externalCompactionId);
+      }
+      first = false;
+      if (!first) sb.append(", ");
+      sb.append("state:");
+      if (this.state == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.state);
+      }
+      first = false;
+      if (!first) sb.append(", ");
+      sb.append("message:");
+      if (this.message == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.message);
+      }
+      first = false;
+      if (!first) sb.append(", ");
+      sb.append("timestamp:");
+      sb.append(this.timestamp);
+      first = false;
+      sb.append(")");
+      return sb.toString();
+    }
+
+    public void validate() throws org.apache.thrift.TException {
+      // check for required fields
+      // check for sub-struct validity
+      if (tinfo != null) {
+        tinfo.validate();
+      }
+      if (credentials != null) {
+        credentials.validate();
+      }
+    }
+
+    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 updateCompactionStatus_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public updateCompactionStatus_argsStandardScheme getScheme() {
+        return new updateCompactionStatus_argsStandardScheme();
+      }
+    }
+
+    private static class updateCompactionStatus_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme<updateCompactionStatus_args> {
+
+      public void read(org.apache.thrift.protocol.TProtocol iprot, updateCompactionStatus_args 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: // TINFO
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+                struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+                struct.tinfo.read(iprot);
+                struct.setTinfoIsSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            case 2: // CREDENTIALS
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+                struct.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials();
+                struct.credentials.read(iprot);
+                struct.setCredentialsIsSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            case 3: // EXTERNAL_COMPACTION_ID
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+                struct.externalCompactionId = iprot.readString();
+                struct.setExternalCompactionIdIsSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            case 4: // STATE
+              if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
+                struct.state = org.apache.accumulo.core.compaction.thrift.TCompactionState.findByValue(iprot.readI32());
+                struct.setStateIsSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            case 5: // MESSAGE
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+                struct.message = iprot.readString();
+                struct.setMessageIsSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            case 6: // TIMESTAMP
+              if (schemeField.type == org.apache.thrift.protocol.TType.I64) {
+                struct.timestamp = iprot.readI64();
+                struct.setTimestampIsSet(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();
+      }
+
+      public void write(org.apache.thrift.protocol.TProtocol oprot, updateCompactionStatus_args struct) throws org.apache.thrift.TException {
+        struct.validate();
+
+        oprot.writeStructBegin(STRUCT_DESC);
+        if (struct.tinfo != null) {
+          oprot.writeFieldBegin(TINFO_FIELD_DESC);
+          struct.tinfo.write(oprot);
+          oprot.writeFieldEnd();
+        }
+        if (struct.credentials != null) {
+          oprot.writeFieldBegin(CREDENTIALS_FIELD_DESC);
+          struct.credentials.write(oprot);
+          oprot.writeFieldEnd();
+        }
+        if (struct.externalCompactionId != null) {
+          oprot.writeFieldBegin(EXTERNAL_COMPACTION_ID_FIELD_DESC);
+          oprot.writeString(struct.externalCompactionId);
+          oprot.writeFieldEnd();
+        }
+        if (struct.state != null) {
+          oprot.writeFieldBegin(STATE_FIELD_DESC);
+          oprot.writeI32(struct.state.getValue());
+          oprot.writeFieldEnd();
+        }
+        if (struct.message != null) {
+          oprot.writeFieldBegin(MESSAGE_FIELD_DESC);
+          oprot.writeString(struct.message);
+          oprot.writeFieldEnd();
+        }
+        oprot.writeFieldBegin(TIMESTAMP_FIELD_DESC);
+        oprot.writeI64(struct.timestamp);
+        oprot.writeFieldEnd();
+        oprot.writeFieldStop();
+        oprot.writeStructEnd();
+      }
+
+    }
+
+    private static class updateCompactionStatus_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public updateCompactionStatus_argsTupleScheme getScheme() {
+        return new updateCompactionStatus_argsTupleScheme();
+      }
+    }
+
+    private static class updateCompactionStatus_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme<updateCompactionStatus_args> {
+
+      @Override
+      public void write(org.apache.thrift.protocol.TProtocol prot, updateCompactionStatus_args 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.isSetTinfo()) {
+          optionals.set(0);
+        }
+        if (struct.isSetCredentials()) {
+          optionals.set(1);
+        }
+        if (struct.isSetExternalCompactionId()) {
+          optionals.set(2);
+        }
+        if (struct.isSetState()) {
+          optionals.set(3);
+        }
+        if (struct.isSetMessage()) {
+          optionals.set(4);
+        }
+        if (struct.isSetTimestamp()) {
+          optionals.set(5);
+        }
+        oprot.writeBitSet(optionals, 6);
+        if (struct.isSetTinfo()) {
+          struct.tinfo.write(oprot);
+        }
+        if (struct.isSetCredentials()) {
+          struct.credentials.write(oprot);
+        }
+        if (struct.isSetExternalCompactionId()) {
+          oprot.writeString(struct.externalCompactionId);
+        }
+        if (struct.isSetState()) {
+          oprot.writeI32(struct.state.getValue());
+        }
+        if (struct.isSetMessage()) {
+          oprot.writeString(struct.message);
+        }
+        if (struct.isSetTimestamp()) {
+          oprot.writeI64(struct.timestamp);
+        }
+      }
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol prot, updateCompactionStatus_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(6);
+        if (incoming.get(0)) {
+          struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+          struct.tinfo.read(iprot);
+          struct.setTinfoIsSet(true);
+        }
+        if (incoming.get(1)) {
+          struct.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials();
+          struct.credentials.read(iprot);
+          struct.setCredentialsIsSet(true);
+        }
+        if (incoming.get(2)) {
+          struct.externalCompactionId = iprot.readString();
+          struct.setExternalCompactionIdIsSet(true);
+        }
+        if (incoming.get(3)) {
+          struct.state = org.apache.accumulo.core.compaction.thrift.TCompactionState.findByValue(iprot.readI32());
+          struct.setStateIsSet(true);
+        }
+        if (incoming.get(4)) {
+          struct.message = iprot.readString();
+          struct.setMessageIsSet(true);
+        }
+        if (incoming.get(5)) {
+          struct.timestamp = iprot.readI64();
+          struct.setTimestampIsSet(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();
+    }
+  }
+
+  public static class updateCompactionStatus_result implements org.apache.thrift.TBase<updateCompactionStatus_result, updateCompactionStatus_result._Fields>, java.io.Serializable, Cloneable, Comparable<updateCompactionStatus_result>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("updateCompactionStatus_result");
+
+    private static final org.apache.thrift.protocol.TField E_FIELD_DESC = new org.apache.thrift.protocol.TField("e", org.apache.thrift.protocol.TType.STRUCT, (short)1);
+
+    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new updateCompactionStatus_resultStandardSchemeFactory();
+    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new updateCompactionStatus_resultTupleSchemeFactory();
+
+    public @org.apache.thrift.annotation.Nullable UnknownCompactionIdException e; // 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 {
+      E((short)1, "e");
+
+      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: // E
+            return E;
+          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;
+      }
+
+      public short getThriftFieldId() {
+        return _thriftId;
+      }
+
+      public java.lang.String getFieldName() {
+        return _fieldName;
+      }
+    }
+
+    // isset id assignments
+    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.E, new org.apache.thrift.meta_data.FieldMetaData("e", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, UnknownCompactionIdException.class)));
+      metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(updateCompactionStatus_result.class, metaDataMap);
+    }
+
+    public updateCompactionStatus_result() {
+    }
+
+    public updateCompactionStatus_result(
+      UnknownCompactionIdException e)
+    {
+      this();
+      this.e = e;
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public updateCompactionStatus_result(updateCompactionStatus_result other) {
+      if (other.isSetE()) {
+        this.e = new UnknownCompactionIdException(other.e);
+      }
+    }
+
+    public updateCompactionStatus_result deepCopy() {
+      return new updateCompactionStatus_result(this);
+    }
+
+    @Override
+    public void clear() {
+      this.e = null;
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public UnknownCompactionIdException getE() {
+      return this.e;
+    }
+
+    public updateCompactionStatus_result setE(@org.apache.thrift.annotation.Nullable UnknownCompactionIdException e) {
+      this.e = e;
+      return this;
+    }
+
+    public void unsetE() {
+      this.e = null;
+    }
+
+    /** Returns true if field e is set (has been assigned a value) and false otherwise */
+    public boolean isSetE() {
+      return this.e != null;
+    }
+
+    public void setEIsSet(boolean value) {
+      if (!value) {
+        this.e = null;
+      }
+    }
+
+    public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable java.lang.Object value) {
+      switch (field) {
+      case E:
+        if (value == null) {
+          unsetE();
+        } else {
+          setE((UnknownCompactionIdException)value);
+        }
+        break;
+
+      }
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public java.lang.Object getFieldValue(_Fields field) {
+      switch (field) {
+      case E:
+        return getE();
+
+      }
+      throw new java.lang.IllegalStateException();
+    }
+
+    /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+    public boolean isSet(_Fields field) {
+      if (field == null) {
+        throw new java.lang.IllegalArgumentException();
+      }
+
+      switch (field) {
+      case E:
+        return isSetE();
+      }
+      throw new java.lang.IllegalStateException();
+    }
+
+    @Override
+    public boolean equals(java.lang.Object that) {
+      if (that == null)
+        return false;
+      if (that instanceof updateCompactionStatus_result)
+        return this.equals((updateCompactionStatus_result)that);
+      return false;
+    }
+
+    public boolean equals(updateCompactionStatus_result that) {
+      if (that == null)
+        return false;
+      if (this == that)
+        return true;
+
+      boolean this_present_e = true && this.isSetE();
+      boolean that_present_e = true && that.isSetE();
+      if (this_present_e || that_present_e) {
+        if (!(this_present_e && that_present_e))
+          return false;
+        if (!this.e.equals(that.e))
+          return false;
+      }
+
+      return true;
+    }
+
+    @Override
+    public int hashCode() {
+      int hashCode = 1;
+
+      hashCode = hashCode * 8191 + ((isSetE()) ? 131071 : 524287);
+      if (isSetE())
+        hashCode = hashCode * 8191 + e.hashCode();
+
+      return hashCode;
+    }
+
+    @Override
+    public int compareTo(updateCompactionStatus_result other) {
+      if (!getClass().equals(other.getClass())) {
+        return getClass().getName().compareTo(other.getClass().getName());
+      }
+
+      int lastComparison = 0;
+
+      lastComparison = java.lang.Boolean.valueOf(isSetE()).compareTo(other.isSetE());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetE()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.e, other.e);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      return 0;
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public _Fields fieldForId(int fieldId) {
+      return _Fields.findByThriftId(fieldId);
+    }
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+      scheme(iprot).read(iprot, this);
+    }
+
+    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("updateCompactionStatus_result(");
+      boolean first = true;
+
+      sb.append("e:");
+      if (this.e == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.e);
+      }
+      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 {
+        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 updateCompactionStatus_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public updateCompactionStatus_resultStandardScheme getScheme() {
+        return new updateCompactionStatus_resultStandardScheme();
+      }
+    }
+
+    private static class updateCompactionStatus_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme<updateCompactionStatus_result> {
+
+      public void read(org.apache.thrift.protocol.TProtocol iprot, updateCompactionStatus_result 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: // E
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+                struct.e = new UnknownCompactionIdException();
+                struct.e.read(iprot);
+                struct.setEIsSet(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();
+      }
+
+      public void write(org.apache.thrift.protocol.TProtocol oprot, updateCompactionStatus_result struct) throws org.apache.thrift.TException {
+        struct.validate();
+
+        oprot.writeStructBegin(STRUCT_DESC);
+        if (struct.e != null) {
+          oprot.writeFieldBegin(E_FIELD_DESC);
+          struct.e.write(oprot);
+          oprot.writeFieldEnd();
+        }
+        oprot.writeFieldStop();
+        oprot.writeStructEnd();
+      }
+
+    }
+
+    private static class updateCompactionStatus_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public updateCompactionStatus_resultTupleScheme getScheme() {
+        return new updateCompactionStatus_resultTupleScheme();
+      }
+    }
+
+    private static class updateCompactionStatus_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme<updateCompactionStatus_result> {
+
+      @Override
+      public void write(org.apache.thrift.protocol.TProtocol prot, updateCompactionStatus_result 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.isSetE()) {
+          optionals.set(0);
+        }
+        oprot.writeBitSet(optionals, 1);
+        if (struct.isSetE()) {
+          struct.e.write(oprot);
+        }
+      }
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol prot, updateCompactionStatus_result struct) throws org.apache.thrift.TException {
+        org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
+        java.util.BitSet incoming = iprot.readBitSet(1);
+        if (incoming.get(0)) {
+          struct.e = new UnknownCompactionIdException();
+          struct.e.read(iprot);
+          struct.setEIsSet(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();
+    }
+  }
+
+  public static class compactionFailed_args implements org.apache.thrift.TBase<compactionFailed_args, compactionFailed_args._Fields>, java.io.Serializable, Cloneable, Comparable<compactionFailed_args>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("compactionFailed_args");
+
+    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 EXTERNAL_COMPACTION_ID_FIELD_DESC = new org.apache.thrift.protocol.TField("externalCompactionId", org.apache.thrift.protocol.TType.STRING, (short)3);
+    private static final org.apache.thrift.protocol.TField EXTENT_FIELD_DESC = new org.apache.thrift.protocol.TField("extent", org.apache.thrift.protocol.TType.STRUCT, (short)4);
+
+    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new compactionFailed_argsStandardSchemeFactory();
+    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new compactionFailed_argsTupleSchemeFactory();
+
+    public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo; // required
+    public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials; // required
+    public @org.apache.thrift.annotation.Nullable java.lang.String externalCompactionId; // required
+    public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent; // 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"),
+      EXTERNAL_COMPACTION_ID((short)3, "externalCompactionId"),
+      EXTENT((short)4, "extent");
+
+      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: // TINFO
+            return TINFO;
+          case 2: // CREDENTIALS
+            return CREDENTIALS;
+          case 3: // EXTERNAL_COMPACTION_ID
+            return EXTERNAL_COMPACTION_ID;
+          case 4: // EXTENT
+            return EXTENT;
+          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;
+      }
+
+      public short getThriftFieldId() {
+        return _thriftId;
+      }
+
+      public java.lang.String getFieldName() {
+        return _fieldName;
+      }
+    }
+
+    // isset id assignments
+    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.TINFO, new org.apache.thrift.meta_data.FieldMetaData("tinfo", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.trace.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.EXTERNAL_COMPACTION_ID, new org.apache.thrift.meta_data.FieldMetaData("externalCompactionId", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+      tmpMap.put(_Fields.EXTENT, new org.apache.thrift.meta_data.FieldMetaData("extent", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent.class)));
+      metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(compactionFailed_args.class, metaDataMap);
+    }
+
+    public compactionFailed_args() {
+    }
+
+    public compactionFailed_args(
+      org.apache.accumulo.core.trace.thrift.TInfo tinfo,
+      org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials,
+      java.lang.String externalCompactionId,
+      org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent)
+    {
+      this();
+      this.tinfo = tinfo;
+      this.credentials = credentials;
+      this.externalCompactionId = externalCompactionId;
+      this.extent = extent;
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public compactionFailed_args(compactionFailed_args other) {
+      if (other.isSetTinfo()) {
+        this.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo(other.tinfo);
+      }
+      if (other.isSetCredentials()) {
+        this.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(other.credentials);
+      }
+      if (other.isSetExternalCompactionId()) {
+        this.externalCompactionId = other.externalCompactionId;
+      }
+      if (other.isSetExtent()) {
+        this.extent = new org.apache.accumulo.core.dataImpl.thrift.TKeyExtent(other.extent);
+      }
+    }
+
+    public compactionFailed_args deepCopy() {
+      return new compactionFailed_args(this);
+    }
+
+    @Override
+    public void clear() {
+      this.tinfo = null;
+      this.credentials = null;
+      this.externalCompactionId = null;
+      this.extent = null;
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public org.apache.accumulo.core.trace.thrift.TInfo getTinfo() {
+      return this.tinfo;
+    }
+
+    public compactionFailed_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo) {
+      this.tinfo = tinfo;
+      return this;
+    }
+
+    public void unsetTinfo() {
+      this.tinfo = null;
+    }
+
+    /** Returns true if field tinfo is set (has been assigned a value) and false otherwise */
+    public boolean isSetTinfo() {
+      return this.tinfo != null;
+    }
+
+    public void setTinfoIsSet(boolean value) {
+      if (!value) {
+        this.tinfo = null;
+      }
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public org.apache.accumulo.core.securityImpl.thrift.TCredentials getCredentials() {
+      return this.credentials;
+    }
+
+    public compactionFailed_args setCredentials(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) {
+      this.credentials = credentials;
+      return this;
+    }
+
+    public void unsetCredentials() {
+      this.credentials = null;
+    }
+
+    /** Returns true if field credentials is set (has been assigned a value) and false otherwise */
+    public boolean isSetCredentials() {
+      return this.credentials != null;
+    }
+
+    public void setCredentialsIsSet(boolean value) {
+      if (!value) {
+        this.credentials = null;
+      }
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public java.lang.String getExternalCompactionId() {
+      return this.externalCompactionId;
+    }
+
+    public compactionFailed_args setExternalCompactionId(@org.apache.thrift.annotation.Nullable java.lang.String externalCompactionId) {
+      this.externalCompactionId = externalCompactionId;
+      return this;
+    }
+
+    public void unsetExternalCompactionId() {
+      this.externalCompactionId = null;
+    }
+
+    /** Returns true if field externalCompactionId is set (has been assigned a value) and false otherwise */
+    public boolean isSetExternalCompactionId() {
+      return this.externalCompactionId != null;
+    }
+
+    public void setExternalCompactionIdIsSet(boolean value) {
+      if (!value) {
+        this.externalCompactionId = null;
+      }
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public org.apache.accumulo.core.dataImpl.thrift.TKeyExtent getExtent() {
+      return this.extent;
+    }
+
+    public compactionFailed_args setExtent(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent) {
+      this.extent = extent;
+      return this;
+    }
+
+    public void unsetExtent() {
+      this.extent = null;
+    }
+
+    /** Returns true if field extent is set (has been assigned a value) and false otherwise */
+    public boolean isSetExtent() {
+      return this.extent != null;
+    }
+
+    public void setExtentIsSet(boolean value) {
+      if (!value) {
+        this.extent = null;
+      }
+    }
+
+    public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable java.lang.Object value) {
+      switch (field) {
+      case TINFO:
+        if (value == null) {
+          unsetTinfo();
+        } else {
+          setTinfo((org.apache.accumulo.core.trace.thrift.TInfo)value);
+        }
+        break;
+
+      case CREDENTIALS:
+        if (value == null) {
+          unsetCredentials();
+        } else {
+          setCredentials((org.apache.accumulo.core.securityImpl.thrift.TCredentials)value);
+        }
+        break;
+
+      case EXTERNAL_COMPACTION_ID:
+        if (value == null) {
+          unsetExternalCompactionId();
+        } else {
+          setExternalCompactionId((java.lang.String)value);
+        }
+        break;
+
+      case EXTENT:
+        if (value == null) {
+          unsetExtent();
+        } else {
+          setExtent((org.apache.accumulo.core.dataImpl.thrift.TKeyExtent)value);
+        }
+        break;
+
+      }
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public java.lang.Object getFieldValue(_Fields field) {
+      switch (field) {
+      case TINFO:
+        return getTinfo();
+
+      case CREDENTIALS:
+        return getCredentials();
+
+      case EXTERNAL_COMPACTION_ID:
+        return getExternalCompactionId();
+
+      case EXTENT:
+        return getExtent();
+
+      }
+      throw new java.lang.IllegalStateException();
+    }
+
+    /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+    public boolean isSet(_Fields field) {
+      if (field == null) {
+        throw new java.lang.IllegalArgumentException();
+      }
+
+      switch (field) {
+      case TINFO:
+        return isSetTinfo();
+      case CREDENTIALS:
+        return isSetCredentials();
+      case EXTERNAL_COMPACTION_ID:
+        return isSetExternalCompactionId();
+      case EXTENT:
+        return isSetExtent();
+      }
+      throw new java.lang.IllegalStateException();
+    }
+
+    @Override
+    public boolean equals(java.lang.Object that) {
+      if (that == null)
+        return false;
+      if (that instanceof compactionFailed_args)
+        return this.equals((compactionFailed_args)that);
+      return false;
+    }
+
+    public boolean equals(compactionFailed_args that) {
+      if (that == null)
+        return false;
+      if (this == that)
+        return true;
+
+      boolean this_present_tinfo = true && this.isSetTinfo();
+      boolean that_present_tinfo = true && that.isSetTinfo();
+      if (this_present_tinfo || that_present_tinfo) {
+        if (!(this_present_tinfo && that_present_tinfo))
+          return false;
+        if (!this.tinfo.equals(that.tinfo))
+          return false;
+      }
+
+      boolean this_present_credentials = true && this.isSetCredentials();
+      boolean that_present_credentials = true && that.isSetCredentials();
+      if (this_present_credentials || that_present_credentials) {
+        if (!(this_present_credentials && that_present_credentials))
+          return false;
+        if (!this.credentials.equals(that.credentials))
+          return false;
+      }
+
+      boolean this_present_externalCompactionId = true && this.isSetExternalCompactionId();
+      boolean that_present_externalCompactionId = true && that.isSetExternalCompactionId();
+      if (this_present_externalCompactionId || that_present_externalCompactionId) {
+        if (!(this_present_externalCompactionId && that_present_externalCompactionId))
+          return false;
+        if (!this.externalCompactionId.equals(that.externalCompactionId))
+          return false;
+      }
+
+      boolean this_present_extent = true && this.isSetExtent();
+      boolean that_present_extent = true && that.isSetExtent();
+      if (this_present_extent || that_present_extent) {
+        if (!(this_present_extent && that_present_extent))
+          return false;
+        if (!this.extent.equals(that.extent))
+          return false;
+      }
+
+      return true;
+    }
+
+    @Override
+    public int hashCode() {
+      int hashCode = 1;
+
+      hashCode = hashCode * 8191 + ((isSetTinfo()) ? 131071 : 524287);
+      if (isSetTinfo())
+        hashCode = hashCode * 8191 + tinfo.hashCode();
+
+      hashCode = hashCode * 8191 + ((isSetCredentials()) ? 131071 : 524287);
+      if (isSetCredentials())
+        hashCode = hashCode * 8191 + credentials.hashCode();
+
+      hashCode = hashCode * 8191 + ((isSetExternalCompactionId()) ? 131071 : 524287);
+      if (isSetExternalCompactionId())
+        hashCode = hashCode * 8191 + externalCompactionId.hashCode();
+
+      hashCode = hashCode * 8191 + ((isSetExtent()) ? 131071 : 524287);
+      if (isSetExtent())
+        hashCode = hashCode * 8191 + extent.hashCode();
+
+      return hashCode;
+    }
+
+    @Override
+    public int compareTo(compactionFailed_args other) {
+      if (!getClass().equals(other.getClass())) {
+        return getClass().getName().compareTo(other.getClass().getName());
+      }
+
+      int lastComparison = 0;
+
+      lastComparison = java.lang.Boolean.valueOf(isSetTinfo()).compareTo(other.isSetTinfo());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetTinfo()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tinfo, other.tinfo);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      lastComparison = java.lang.Boolean.valueOf(isSetCredentials()).compareTo(other.isSetCredentials());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetCredentials()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.credentials, other.credentials);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      lastComparison = java.lang.Boolean.valueOf(isSetExternalCompactionId()).compareTo(other.isSetExternalCompactionId());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetExternalCompactionId()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.externalCompactionId, other.externalCompactionId);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      lastComparison = java.lang.Boolean.valueOf(isSetExtent()).compareTo(other.isSetExtent());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetExtent()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.extent, other.extent);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      return 0;
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public _Fields fieldForId(int fieldId) {
+      return _Fields.findByThriftId(fieldId);
+    }
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+      scheme(iprot).read(iprot, this);
+    }
+
+    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("compactionFailed_args(");
+      boolean first = true;
+
+      sb.append("tinfo:");
+      if (this.tinfo == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.tinfo);
+      }
+      first = false;
+      if (!first) sb.append(", ");
+      sb.append("credentials:");
+      if (this.credentials == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.credentials);
+      }
+      first = false;
+      if (!first) sb.append(", ");
+      sb.append("externalCompactionId:");
+      if (this.externalCompactionId == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.externalCompactionId);
+      }
+      first = false;
+      if (!first) sb.append(", ");
+      sb.append("extent:");
+      if (this.extent == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.extent);
+      }
+      first = false;
+      sb.append(")");
+      return sb.toString();
+    }
+
+    public void validate() throws org.apache.thrift.TException {
+      // check for required fields
+      // check for sub-struct validity
+      if (tinfo != null) {
+        tinfo.validate();
+      }
+      if (credentials != null) {
+        credentials.validate();
+      }
+      if (extent != null) {
+        extent.validate();
+      }
+    }
+
+    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 {
+        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 compactionFailed_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public compactionFailed_argsStandardScheme getScheme() {
+        return new compactionFailed_argsStandardScheme();
+      }
+    }
+
+    private static class compactionFailed_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme<compactionFailed_args> {
+
+      public void read(org.apache.thrift.protocol.TProtocol iprot, compactionFailed_args 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: // TINFO
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+                struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+                struct.tinfo.read(iprot);
+                struct.setTinfoIsSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            case 2: // CREDENTIALS
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+                struct.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials();
+                struct.credentials.read(iprot);
+                struct.setCredentialsIsSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            case 3: // EXTERNAL_COMPACTION_ID
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+                struct.externalCompactionId = iprot.readString();
+                struct.setExternalCompactionIdIsSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            case 4: // EXTENT
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+                struct.extent = new org.apache.accumulo.core.dataImpl.thrift.TKeyExtent();
+                struct.extent.read(iprot);
+                struct.setExtentIsSet(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();
+      }
+
+      public void write(org.apache.thrift.protocol.TProtocol oprot, compactionFailed_args struct) throws org.apache.thrift.TException {
+        struct.validate();
+
+        oprot.writeStructBegin(STRUCT_DESC);
+        if (struct.tinfo != null) {
+          oprot.writeFieldBegin(TINFO_FIELD_DESC);
+          struct.tinfo.write(oprot);
+          oprot.writeFieldEnd();
+        }
+        if (struct.credentials != null) {
+          oprot.writeFieldBegin(CREDENTIALS_FIELD_DESC);
+          struct.credentials.write(oprot);
+          oprot.writeFieldEnd();
+        }
+        if (struct.externalCompactionId != null) {
+          oprot.writeFieldBegin(EXTERNAL_COMPACTION_ID_FIELD_DESC);
+          oprot.writeString(struct.externalCompactionId);
+          oprot.writeFieldEnd();
+        }
+        if (struct.extent != null) {
+          oprot.writeFieldBegin(EXTENT_FIELD_DESC);
+          struct.extent.write(oprot);
+          oprot.writeFieldEnd();
+        }
+        oprot.writeFieldStop();
+        oprot.writeStructEnd();
+      }
+
+    }
+
+    private static class compactionFailed_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public compactionFailed_argsTupleScheme getScheme() {
+        return new compactionFailed_argsTupleScheme();
+      }
+    }
+
+    private static class compactionFailed_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme<compactionFailed_args> {
+
+      @Override
+      public void write(org.apache.thrift.protocol.TProtocol prot, compactionFailed_args 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.isSetTinfo()) {
+          optionals.set(0);
+        }
+        if (struct.isSetCredentials()) {
+          optionals.set(1);
+        }
+        if (struct.isSetExternalCompactionId()) {
+          optionals.set(2);
+        }
+        if (struct.isSetExtent()) {
+          optionals.set(3);
+        }
+        oprot.writeBitSet(optionals, 4);
+        if (struct.isSetTinfo()) {
+          struct.tinfo.write(oprot);
+        }
+        if (struct.isSetCredentials()) {
+          struct.credentials.write(oprot);
+        }
+        if (struct.isSetExternalCompactionId()) {
+          oprot.writeString(struct.externalCompactionId);
+        }
+        if (struct.isSetExtent()) {
+          struct.extent.write(oprot);
+        }
+      }
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol prot, compactionFailed_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(4);
+        if (incoming.get(0)) {
+          struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+          struct.tinfo.read(iprot);
+          struct.setTinfoIsSet(true);
+        }
+        if (incoming.get(1)) {
+          struct.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials();
+          struct.credentials.read(iprot);
+          struct.setCredentialsIsSet(true);
+        }
+        if (incoming.get(2)) {
+          struct.externalCompactionId = iprot.readString();
+          struct.setExternalCompactionIdIsSet(true);
+        }
+        if (incoming.get(3)) {
+          struct.extent = new org.apache.accumulo.core.dataImpl.thrift.TKeyExtent();
+          struct.extent.read(iprot);
+          struct.setExtentIsSet(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();
+    }
+  }
+
+  public static class compactionFailed_result implements org.apache.thrift.TBase<compactionFailed_result, compactionFailed_result._Fields>, java.io.Serializable, Cloneable, Comparable<compactionFailed_result>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("compactionFailed_result");
+
+    private static final org.apache.thrift.protocol.TField E_FIELD_DESC = new org.apache.thrift.protocol.TField("e", org.apache.thrift.protocol.TType.STRUCT, (short)1);
+
+    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new compactionFailed_resultStandardSchemeFactory();
+    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new compactionFailed_resultTupleSchemeFactory();
+
+    public @org.apache.thrift.annotation.Nullable UnknownCompactionIdException e; // 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 {
+      E((short)1, "e");
+
+      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: // E
+            return E;
+          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;
+      }
+
+      public short getThriftFieldId() {
+        return _thriftId;
+      }
+
+      public java.lang.String getFieldName() {
+        return _fieldName;
+      }
+    }
+
+    // isset id assignments
+    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.E, new org.apache.thrift.meta_data.FieldMetaData("e", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, UnknownCompactionIdException.class)));
+      metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(compactionFailed_result.class, metaDataMap);
+    }
+
+    public compactionFailed_result() {
+    }
+
+    public compactionFailed_result(
+      UnknownCompactionIdException e)
+    {
+      this();
+      this.e = e;
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public compactionFailed_result(compactionFailed_result other) {
+      if (other.isSetE()) {
+        this.e = new UnknownCompactionIdException(other.e);
+      }
+    }
+
+    public compactionFailed_result deepCopy() {
+      return new compactionFailed_result(this);
+    }
+
+    @Override
+    public void clear() {
+      this.e = null;
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public UnknownCompactionIdException getE() {
+      return this.e;
+    }
+
+    public compactionFailed_result setE(@org.apache.thrift.annotation.Nullable UnknownCompactionIdException e) {
+      this.e = e;
+      return this;
+    }
+
+    public void unsetE() {
+      this.e = null;
+    }
+
+    /** Returns true if field e is set (has been assigned a value) and false otherwise */
+    public boolean isSetE() {
+      return this.e != null;
+    }
+
+    public void setEIsSet(boolean value) {
+      if (!value) {
+        this.e = null;
+      }
+    }
+
+    public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable java.lang.Object value) {
+      switch (field) {
+      case E:
+        if (value == null) {
+          unsetE();
+        } else {
+          setE((UnknownCompactionIdException)value);
+        }
+        break;
+
+      }
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public java.lang.Object getFieldValue(_Fields field) {
+      switch (field) {
+      case E:
+        return getE();
+
+      }
+      throw new java.lang.IllegalStateException();
+    }
+
+    /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+    public boolean isSet(_Fields field) {
+      if (field == null) {
+        throw new java.lang.IllegalArgumentException();
+      }
+
+      switch (field) {
+      case E:
+        return isSetE();
+      }
+      throw new java.lang.IllegalStateException();
+    }
+
+    @Override
+    public boolean equals(java.lang.Object that) {
+      if (that == null)
+        return false;
+      if (that instanceof compactionFailed_result)
+        return this.equals((compactionFailed_result)that);
+      return false;
+    }
+
+    public boolean equals(compactionFailed_result that) {
+      if (that == null)
+        return false;
+      if (this == that)
+        return true;
+
+      boolean this_present_e = true && this.isSetE();
+      boolean that_present_e = true && that.isSetE();
+      if (this_present_e || that_present_e) {
+        if (!(this_present_e && that_present_e))
+          return false;
+        if (!this.e.equals(that.e))
+          return false;
+      }
+
+      return true;
+    }
+
+    @Override
+    public int hashCode() {
+      int hashCode = 1;
+
+      hashCode = hashCode * 8191 + ((isSetE()) ? 131071 : 524287);
+      if (isSetE())
+        hashCode = hashCode * 8191 + e.hashCode();
+
+      return hashCode;
+    }
+
+    @Override
+    public int compareTo(compactionFailed_result other) {
+      if (!getClass().equals(other.getClass())) {
+        return getClass().getName().compareTo(other.getClass().getName());
+      }
+
+      int lastComparison = 0;
+
+      lastComparison = java.lang.Boolean.valueOf(isSetE()).compareTo(other.isSetE());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetE()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.e, other.e);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      return 0;
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public _Fields fieldForId(int fieldId) {
+      return _Fields.findByThriftId(fieldId);
+    }
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+      scheme(iprot).read(iprot, this);
+    }
+
+    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("compactionFailed_result(");
+      boolean first = true;
+
+      sb.append("e:");
+      if (this.e == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.e);
+      }
+      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 {
+        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 compactionFailed_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public compactionFailed_resultStandardScheme getScheme() {
+        return new compactionFailed_resultStandardScheme();
+      }
+    }
+
+    private static class compactionFailed_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme<compactionFailed_result> {
+
+      public void read(org.apache.thrift.protocol.TProtocol iprot, compactionFailed_result 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: // E
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+                struct.e = new UnknownCompactionIdException();
+                struct.e.read(iprot);
+                struct.setEIsSet(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();
+      }
+
+      public void write(org.apache.thrift.protocol.TProtocol oprot, compactionFailed_result struct) throws org.apache.thrift.TException {
+        struct.validate();
+
+        oprot.writeStructBegin(STRUCT_DESC);
+        if (struct.e != null) {
+          oprot.writeFieldBegin(E_FIELD_DESC);
+          struct.e.write(oprot);
+          oprot.writeFieldEnd();
+        }
+        oprot.writeFieldStop();
+        oprot.writeStructEnd();
+      }
+
+    }
+
+    private static class compactionFailed_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public compactionFailed_resultTupleScheme getScheme() {
+        return new compactionFailed_resultTupleScheme();
+      }
+    }
+
+    private static class compactionFailed_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme<compactionFailed_result> {
+
+      @Override
+      public void write(org.apache.thrift.protocol.TProtocol prot, compactionFailed_result 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.isSetE()) {
+          optionals.set(0);
+        }
+        oprot.writeBitSet(optionals, 1);
+        if (struct.isSetE()) {
+          struct.e.write(oprot);
+        }
+      }
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol prot, compactionFailed_result struct) throws org.apache.thrift.TException {
+        org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
+        java.util.BitSet incoming = iprot.readBitSet(1);
+        if (incoming.get(0)) {
+          struct.e = new UnknownCompactionIdException();
+          struct.e.read(iprot);
+          struct.setEIsSet(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/compaction/thrift/CompactorService.java b/core/src/main/thrift-gen-java/org/apache/accumulo/core/compaction/thrift/CompactorService.java
new file mode 100644
index 0000000..d049564
--- /dev/null
+++ b/core/src/main/thrift-gen-java/org/apache/accumulo/core/compaction/thrift/CompactorService.java
@@ -0,0 +1,4643 @@
+/*
+ * 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
+ *
+ *   http://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.12.0)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package org.apache.accumulo.core.compaction.thrift;
+
+@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"})
+public class CompactorService {
+
+  public interface Iface {
+
+    public void cancel(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String externalCompactionId) throws UnknownCompactionIdException, org.apache.thrift.TException;
+
+    public org.apache.accumulo.core.tabletserver.thrift.TExternalCompactionJob getRunningCompaction(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.thrift.TException;
+
+    public java.lang.String getRunningCompactionId(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.thrift.TException;
+
+    public java.util.List<org.apache.accumulo.core.tabletserver.thrift.ActiveCompaction> getActiveCompactions(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.thrift.TException;
+
+  }
+
+  public interface AsyncIface {
+
+    public void cancel(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String externalCompactionId, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
+
+    public void getRunningCompaction(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.tabletserver.thrift.TExternalCompactionJob> resultHandler) throws org.apache.thrift.TException;
+
+    public void getRunningCompactionId(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<java.lang.String> resultHandler) throws org.apache.thrift.TException;
+
+    public void getActiveCompactions(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<java.util.List<org.apache.accumulo.core.tabletserver.thrift.ActiveCompaction>> resultHandler) throws org.apache.thrift.TException;
+
+  }
+
+  public static class Client extends org.apache.thrift.TServiceClient implements Iface {
+    public static class Factory implements org.apache.thrift.TServiceClientFactory<Client> {
+      public Factory() {}
+      public Client getClient(org.apache.thrift.protocol.TProtocol prot) {
+        return new Client(prot);
+      }
+      public Client getClient(org.apache.thrift.protocol.TProtocol iprot, org.apache.thrift.protocol.TProtocol oprot) {
+        return new Client(iprot, oprot);
+      }
+    }
+
+    public Client(org.apache.thrift.protocol.TProtocol prot)
+    {
+      super(prot, prot);
+    }
+
+    public Client(org.apache.thrift.protocol.TProtocol iprot, org.apache.thrift.protocol.TProtocol oprot) {
+      super(iprot, oprot);
+    }
+
+    public void cancel(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String externalCompactionId) throws UnknownCompactionIdException, org.apache.thrift.TException
+    {
+      send_cancel(tinfo, credentials, externalCompactionId);
+      recv_cancel();
+    }
+
+    public void send_cancel(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String externalCompactionId) throws org.apache.thrift.TException
+    {
+      cancel_args args = new cancel_args();
+      args.setTinfo(tinfo);
+      args.setCredentials(credentials);
+      args.setExternalCompactionId(externalCompactionId);
+      sendBase("cancel", args);
+    }
+
+    public void recv_cancel() throws UnknownCompactionIdException, org.apache.thrift.TException
+    {
+      cancel_result result = new cancel_result();
+      receiveBase(result, "cancel");
+      if (result.e != null) {
+        throw result.e;
+      }
+      return;
+    }
+
+    public org.apache.accumulo.core.tabletserver.thrift.TExternalCompactionJob getRunningCompaction(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.thrift.TException
+    {
+      send_getRunningCompaction(tinfo, credentials);
+      return recv_getRunningCompaction();
+    }
+
+    public void send_getRunningCompaction(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws org.apache.thrift.TException
+    {
+      getRunningCompaction_args args = new getRunningCompaction_args();
+      args.setTinfo(tinfo);
+      args.setCredentials(credentials);
+      sendBase("getRunningCompaction", args);
+    }
+
+    public org.apache.accumulo.core.tabletserver.thrift.TExternalCompactionJob recv_getRunningCompaction() throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.thrift.TException
+    {
+      getRunningCompaction_result result = new getRunningCompaction_result();
+      receiveBase(result, "getRunningCompaction");
+      if (result.isSetSuccess()) {
+        return result.success;
+      }
+      if (result.sec != null) {
+        throw result.sec;
+      }
+      throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "getRunningCompaction failed: unknown result");
+    }
+
+    public java.lang.String getRunningCompactionId(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.thrift.TException
+    {
+      send_getRunningCompactionId(tinfo, credentials);
+      return recv_getRunningCompactionId();
+    }
+
+    public void send_getRunningCompactionId(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws org.apache.thrift.TException
+    {
+      getRunningCompactionId_args args = new getRunningCompactionId_args();
+      args.setTinfo(tinfo);
+      args.setCredentials(credentials);
+      sendBase("getRunningCompactionId", args);
+    }
+
+    public java.lang.String recv_getRunningCompactionId() throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.thrift.TException
+    {
+      getRunningCompactionId_result result = new getRunningCompactionId_result();
+      receiveBase(result, "getRunningCompactionId");
+      if (result.isSetSuccess()) {
+        return result.success;
+      }
+      if (result.sec != null) {
+        throw result.sec;
+      }
+      throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "getRunningCompactionId failed: unknown result");
+    }
+
+    public java.util.List<org.apache.accumulo.core.tabletserver.thrift.ActiveCompaction> getActiveCompactions(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.thrift.TException
+    {
+      send_getActiveCompactions(tinfo, credentials);
+      return recv_getActiveCompactions();
+    }
+
+    public void send_getActiveCompactions(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws org.apache.thrift.TException
+    {
+      getActiveCompactions_args args = new getActiveCompactions_args();
+      args.setTinfo(tinfo);
+      args.setCredentials(credentials);
+      sendBase("getActiveCompactions", args);
+    }
+
+    public java.util.List<org.apache.accumulo.core.tabletserver.thrift.ActiveCompaction> recv_getActiveCompactions() throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.thrift.TException
+    {
+      getActiveCompactions_result result = new getActiveCompactions_result();
+      receiveBase(result, "getActiveCompactions");
+      if (result.isSetSuccess()) {
+        return result.success;
+      }
+      if (result.sec != null) {
+        throw result.sec;
+      }
+      throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "getActiveCompactions failed: unknown result");
+    }
+
+  }
+  public static class AsyncClient extends org.apache.thrift.async.TAsyncClient implements AsyncIface {
+    public static class Factory implements org.apache.thrift.async.TAsyncClientFactory<AsyncClient> {
+      private org.apache.thrift.async.TAsyncClientManager clientManager;
+      private org.apache.thrift.protocol.TProtocolFactory protocolFactory;
+      public Factory(org.apache.thrift.async.TAsyncClientManager clientManager, org.apache.thrift.protocol.TProtocolFactory protocolFactory) {
+        this.clientManager = clientManager;
+        this.protocolFactory = protocolFactory;
+      }
+      public AsyncClient getAsyncClient(org.apache.thrift.transport.TNonblockingTransport transport) {
+        return new AsyncClient(protocolFactory, clientManager, transport);
+      }
+    }
+
+    public AsyncClient(org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.async.TAsyncClientManager clientManager, org.apache.thrift.transport.TNonblockingTransport transport) {
+      super(protocolFactory, clientManager, transport);
+    }
+
+    public void cancel(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String externalCompactionId, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
+      checkReady();
+      cancel_call method_call = new cancel_call(tinfo, credentials, externalCompactionId, resultHandler, this, ___protocolFactory, ___transport);
+      this.___currentMethod = method_call;
+      ___manager.call(method_call);
+    }
+
+    public static class cancel_call extends org.apache.thrift.async.TAsyncMethodCall<Void> {
+      private org.apache.accumulo.core.trace.thrift.TInfo tinfo;
+      private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials;
+      private java.lang.String externalCompactionId;
+      public cancel_call(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String externalCompactionId, 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;
+        this.externalCompactionId = externalCompactionId;
+      }
+
+      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("cancel", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        cancel_args args = new cancel_args();
+        args.setTinfo(tinfo);
+        args.setCredentials(credentials);
+        args.setExternalCompactionId(externalCompactionId);
+        args.write(prot);
+        prot.writeMessageEnd();
+      }
+
+      public Void getResult() throws UnknownCompactionIdException, org.apache.thrift.TException {
+        if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
+          throw new java.lang.IllegalStateException("Method call not finished!");
+        }
+        org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
+        org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
+        return null;
+      }
+    }
+
+    public void getRunningCompaction(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.tabletserver.thrift.TExternalCompactionJob> resultHandler) throws org.apache.thrift.TException {
+      checkReady();
+      getRunningCompaction_call method_call = new getRunningCompaction_call(tinfo, credentials, resultHandler, this, ___protocolFactory, ___transport);
+      this.___currentMethod = method_call;
+      ___manager.call(method_call);
+    }
+
+    public static class getRunningCompaction_call extends org.apache.thrift.async.TAsyncMethodCall<org.apache.accumulo.core.tabletserver.thrift.TExternalCompactionJob> {
+      private org.apache.accumulo.core.trace.thrift.TInfo tinfo;
+      private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials;
+      public getRunningCompaction_call(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.tabletserver.thrift.TExternalCompactionJob> 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;
+      }
+
+      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("getRunningCompaction", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        getRunningCompaction_args args = new getRunningCompaction_args();
+        args.setTinfo(tinfo);
+        args.setCredentials(credentials);
+        args.write(prot);
+        prot.writeMessageEnd();
+      }
+
+      public org.apache.accumulo.core.tabletserver.thrift.TExternalCompactionJob getResult() throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.thrift.TException {
+        if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
+          throw new java.lang.IllegalStateException("Method call not finished!");
+        }
+        org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
+        org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
+        return (new Client(prot)).recv_getRunningCompaction();
+      }
+    }
+
+    public void getRunningCompactionId(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<java.lang.String> resultHandler) throws org.apache.thrift.TException {
+      checkReady();
+      getRunningCompactionId_call method_call = new getRunningCompactionId_call(tinfo, credentials, resultHandler, this, ___protocolFactory, ___transport);
+      this.___currentMethod = method_call;
+      ___manager.call(method_call);
+    }
+
+    public static class getRunningCompactionId_call extends org.apache.thrift.async.TAsyncMethodCall<java.lang.String> {
+      private org.apache.accumulo.core.trace.thrift.TInfo tinfo;
+      private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials;
+      public getRunningCompactionId_call(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, 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;
+      }
+
+      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("getRunningCompactionId", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        getRunningCompactionId_args args = new getRunningCompactionId_args();
+        args.setTinfo(tinfo);
+        args.setCredentials(credentials);
+        args.write(prot);
+        prot.writeMessageEnd();
+      }
+
+      public java.lang.String getResult() throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.thrift.TException {
+        if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
+          throw new java.lang.IllegalStateException("Method call not finished!");
+        }
+        org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
+        org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
+        return (new Client(prot)).recv_getRunningCompactionId();
+      }
+    }
+
+    public void getActiveCompactions(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<java.util.List<org.apache.accumulo.core.tabletserver.thrift.ActiveCompaction>> resultHandler) throws org.apache.thrift.TException {
+      checkReady();
+      getActiveCompactions_call method_call = new getActiveCompactions_call(tinfo, credentials, resultHandler, this, ___protocolFactory, ___transport);
+      this.___currentMethod = method_call;
+      ___manager.call(method_call);
+    }
+
+    public static class getActiveCompactions_call extends org.apache.thrift.async.TAsyncMethodCall<java.util.List<org.apache.accumulo.core.tabletserver.thrift.ActiveCompaction>> {
+      private org.apache.accumulo.core.trace.thrift.TInfo tinfo;
+      private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials;
+      public getActiveCompactions_call(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<java.util.List<org.apache.accumulo.core.tabletserver.thrift.ActiveCompaction>> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.T [...]
+        super(client, protocolFactory, transport, resultHandler, false);
+        this.tinfo = tinfo;
+        this.credentials = credentials;
+      }
+
+      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("getActiveCompactions", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        getActiveCompactions_args args = new getActiveCompactions_args();
+        args.setTinfo(tinfo);
+        args.setCredentials(credentials);
+        args.write(prot);
+        prot.writeMessageEnd();
+      }
+
+      public java.util.List<org.apache.accumulo.core.tabletserver.thrift.ActiveCompaction> getResult() throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.thrift.TException {
+        if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
+          throw new java.lang.IllegalStateException("Method call not finished!");
+        }
+        org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
+        org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
+        return (new Client(prot)).recv_getActiveCompactions();
+      }
+    }
+
+  }
+
+  public static class Processor<I extends Iface> extends org.apache.thrift.TBaseProcessor<I> implements org.apache.thrift.TProcessor {
+    private static final org.slf4j.Logger _LOGGER = org.slf4j.LoggerFactory.getLogger(Processor.class.getName());
+    public Processor(I iface) {
+      super(iface, getProcessMap(new java.util.HashMap<java.lang.String, org.apache.thrift.ProcessFunction<I, ? extends org.apache.thrift.TBase>>()));
+    }
+
+    protected Processor(I iface, java.util.Map<java.lang.String, org.apache.thrift.ProcessFunction<I, ? extends org.apache.thrift.TBase>> processMap) {
+      super(iface, getProcessMap(processMap));
+    }
+
+    private static <I extends Iface> java.util.Map<java.lang.String,  org.apache.thrift.ProcessFunction<I, ? extends org.apache.thrift.TBase>> getProcessMap(java.util.Map<java.lang.String, org.apache.thrift.ProcessFunction<I, ? extends  org.apache.thrift.TBase>> processMap) {
+      processMap.put("cancel", new cancel());
+      processMap.put("getRunningCompaction", new getRunningCompaction());
+      processMap.put("getRunningCompactionId", new getRunningCompactionId());
+      processMap.put("getActiveCompactions", new getActiveCompactions());
+      return processMap;
+    }
+
+    public static class cancel<I extends Iface> extends org.apache.thrift.ProcessFunction<I, cancel_args> {
+      public cancel() {
+        super("cancel");
+      }
+
+      public cancel_args getEmptyArgsInstance() {
+        return new cancel_args();
+      }
+
+      protected boolean isOneway() {
+        return false;
+      }
+
+      @Override
+      protected boolean rethrowUnhandledExceptions() {
+        return false;
+      }
+
+      public cancel_result getResult(I iface, cancel_args args) throws org.apache.thrift.TException {
+        cancel_result result = new cancel_result();
+        try {
+          iface.cancel(args.tinfo, args.credentials, args.externalCompactionId);
+        } catch (UnknownCompactionIdException e) {
+          result.e = e;
+        }
+        return result;
+      }
+    }
+
+    public static class getRunningCompaction<I extends Iface> extends org.apache.thrift.ProcessFunction<I, getRunningCompaction_args> {
+      public getRunningCompaction() {
+        super("getRunningCompaction");
+      }
+
+      public getRunningCompaction_args getEmptyArgsInstance() {
+        return new getRunningCompaction_args();
+      }
+
+      protected boolean isOneway() {
+        return false;
+      }
+
+      @Override
+      protected boolean rethrowUnhandledExceptions() {
+        return false;
+      }
+
+      public getRunningCompaction_result getResult(I iface, getRunningCompaction_args args) throws org.apache.thrift.TException {
+        getRunningCompaction_result result = new getRunningCompaction_result();
+        try {
+          result.success = iface.getRunningCompaction(args.tinfo, args.credentials);
+        } catch (org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException sec) {
+          result.sec = sec;
+        }
+        return result;
+      }
+    }
+
+    public static class getRunningCompactionId<I extends Iface> extends org.apache.thrift.ProcessFunction<I, getRunningCompactionId_args> {
+      public getRunningCompactionId() {
+        super("getRunningCompactionId");
+      }
+
+      public getRunningCompactionId_args getEmptyArgsInstance() {
+        return new getRunningCompactionId_args();
+      }
+
+      protected boolean isOneway() {
+        return false;
+      }
+
+      @Override
+      protected boolean rethrowUnhandledExceptions() {
+        return false;
+      }
+
+      public getRunningCompactionId_result getResult(I iface, getRunningCompactionId_args args) throws org.apache.thrift.TException {
+        getRunningCompactionId_result result = new getRunningCompactionId_result();
+        try {
+          result.success = iface.getRunningCompactionId(args.tinfo, args.credentials);
+        } catch (org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException sec) {
+          result.sec = sec;
+        }
+        return result;
+      }
+    }
+
+    public static class getActiveCompactions<I extends Iface> extends org.apache.thrift.ProcessFunction<I, getActiveCompactions_args> {
+      public getActiveCompactions() {
+        super("getActiveCompactions");
+      }
+
+      public getActiveCompactions_args getEmptyArgsInstance() {
+        return new getActiveCompactions_args();
+      }
+
+      protected boolean isOneway() {
+        return false;
+      }
+
+      @Override
+      protected boolean rethrowUnhandledExceptions() {
+        return false;
+      }
+
+      public getActiveCompactions_result getResult(I iface, getActiveCompactions_args args) throws org.apache.thrift.TException {
+        getActiveCompactions_result result = new getActiveCompactions_result();
+        try {
+          result.success = iface.getActiveCompactions(args.tinfo, args.credentials);
+        } catch (org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException sec) {
+          result.sec = sec;
+        }
+        return result;
+      }
+    }
+
+  }
+
+  public static class AsyncProcessor<I extends AsyncIface> extends org.apache.thrift.TBaseAsyncProcessor<I> {
+    private static final org.slf4j.Logger _LOGGER = org.slf4j.LoggerFactory.getLogger(AsyncProcessor.class.getName());
+    public AsyncProcessor(I iface) {
+      super(iface, getProcessMap(new java.util.HashMap<java.lang.String, org.apache.thrift.AsyncProcessFunction<I, ? extends org.apache.thrift.TBase, ?>>()));
+    }
+
+    protected AsyncProcessor(I iface, java.util.Map<java.lang.String,  org.apache.thrift.AsyncProcessFunction<I, ? extends  org.apache.thrift.TBase, ?>> processMap) {
+      super(iface, getProcessMap(processMap));
+    }
+
+    private static <I extends AsyncIface> java.util.Map<java.lang.String,  org.apache.thrift.AsyncProcessFunction<I, ? extends  org.apache.thrift.TBase,?>> getProcessMap(java.util.Map<java.lang.String,  org.apache.thrift.AsyncProcessFunction<I, ? extends  org.apache.thrift.TBase, ?>> processMap) {
+      processMap.put("cancel", new cancel());
+      processMap.put("getRunningCompaction", new getRunningCompaction());
+      processMap.put("getRunningCompactionId", new getRunningCompactionId());
+      processMap.put("getActiveCompactions", new getActiveCompactions());
+      return processMap;
+    }
+
+    public static class cancel<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, cancel_args, Void> {
+      public cancel() {
+        super("cancel");
+      }
+
+      public cancel_args getEmptyArgsInstance() {
+        return new cancel_args();
+      }
+
+      public org.apache.thrift.async.AsyncMethodCallback<Void> 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<Void>() { 
+          public void onComplete(Void o) {
+            cancel_result result = new cancel_result();
+            try {
+              fcall.sendResponse(fb, result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
+            } catch (org.apache.thrift.transport.TTransportException e) {
+              _LOGGER.error("TTransportException writing to internal frame buffer", e);
+              fb.close();
+            } catch (java.lang.Exception e) {
+              _LOGGER.error("Exception writing to internal frame buffer", e);
+              onError(e);
+            }
+          }
+          public void onError(java.lang.Exception e) {
+            byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
+            org.apache.thrift.TSerializable msg;
+            cancel_result result = new cancel_result();
+            if (e instanceof UnknownCompactionIdException) {
+              result.e = (UnknownCompactionIdException) e;
+              result.setEIsSet(true);
+              msg = result;
+            } else if (e instanceof org.apache.thrift.transport.TTransportException) {
+              _LOGGER.error("TTransportException inside handler", e);
+              fb.close();
+              return;
+            } else if (e instanceof org.apache.thrift.TApplicationException) {
+              _LOGGER.error("TApplicationException inside handler", e);
+              msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
+              msg = (org.apache.thrift.TApplicationException)e;
+            } else {
+              _LOGGER.error("Exception inside handler", e);
+              msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
+              msg = new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage());
+            }
+            try {
+              fcall.sendResponse(fb,msg,msgType,seqid);
+            } catch (java.lang.Exception ex) {
+              _LOGGER.error("Exception writing to internal frame buffer", ex);
+              fb.close();
+            }
+          }
+        };
+      }
+
+      protected boolean isOneway() {
+        return false;
+      }
+
+      public void start(I iface, cancel_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
+        iface.cancel(args.tinfo, args.credentials, args.externalCompactionId,resultHandler);
+      }
+    }
+
+    public static class getRunningCompaction<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, getRunningCompaction_args, org.apache.accumulo.core.tabletserver.thrift.TExternalCompactionJob> {
+      public getRunningCompaction() {
+        super("getRunningCompaction");
+      }
+
+      public getRunningCompaction_args getEmptyArgsInstance() {
+        return new getRunningCompaction_args();
+      }
+
+      public org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.tabletserver.thrift.TExternalCompactionJob> 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<org.apache.accumulo.core.tabletserver.thrift.TExternalCompactionJob>() { 
+          public void onComplete(org.apache.accumulo.core.tabletserver.thrift.TExternalCompactionJob o) {
+            getRunningCompaction_result result = new getRunningCompaction_result();
+            result.success = o;
+            try {
+              fcall.sendResponse(fb, result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
+            } catch (org.apache.thrift.transport.TTransportException e) {
+              _LOGGER.error("TTransportException writing to internal frame buffer", e);
+              fb.close();
+            } catch (java.lang.Exception e) {
+              _LOGGER.error("Exception writing to internal frame buffer", e);
+              onError(e);
+            }
+          }
+          public void onError(java.lang.Exception e) {
+            byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
+            org.apache.thrift.TSerializable msg;
+            getRunningCompaction_result result = new getRunningCompaction_result();
+            if (e instanceof org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException) {
+              result.sec = (org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException) e;
+              result.setSecIsSet(true);
+              msg = result;
+            } else if (e instanceof org.apache.thrift.transport.TTransportException) {
+              _LOGGER.error("TTransportException inside handler", e);
+              fb.close();
+              return;
+            } else if (e instanceof org.apache.thrift.TApplicationException) {
+              _LOGGER.error("TApplicationException inside handler", e);
+              msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
+              msg = (org.apache.thrift.TApplicationException)e;
+            } else {
+              _LOGGER.error("Exception inside handler", e);
+              msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
+              msg = new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage());
+            }
+            try {
+              fcall.sendResponse(fb,msg,msgType,seqid);
+            } catch (java.lang.Exception ex) {
+              _LOGGER.error("Exception writing to internal frame buffer", ex);
+              fb.close();
+            }
+          }
+        };
+      }
+
+      protected boolean isOneway() {
+        return false;
+      }
+
+      public void start(I iface, getRunningCompaction_args args, org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.tabletserver.thrift.TExternalCompactionJob> resultHandler) throws org.apache.thrift.TException {
+        iface.getRunningCompaction(args.tinfo, args.credentials,resultHandler);
+      }
+    }
+
+    public static class getRunningCompactionId<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, getRunningCompactionId_args, java.lang.String> {
+      public getRunningCompactionId() {
+        super("getRunningCompactionId");
+      }
+
+      public getRunningCompactionId_args getEmptyArgsInstance() {
+        return new getRunningCompactionId_args();
+      }
+
+      public org.apache.thrift.async.AsyncMethodCallback<java.lang.String> 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.String>() { 
+          public void onComplete(java.lang.String o) {
+            getRunningCompactionId_result result = new getRunningCompactionId_result();
+            result.success = o;
+            try {
+              fcall.sendResponse(fb, result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
+            } catch (org.apache.thrift.transport.TTransportException e) {
+              _LOGGER.error("TTransportException writing to internal frame buffer", e);
+              fb.close();
+            } catch (java.lang.Exception e) {
+              _LOGGER.error("Exception writing to internal frame buffer", e);
+              onError(e);
+            }
+          }
+          public void onError(java.lang.Exception e) {
+            byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
+            org.apache.thrift.TSerializable msg;
+            getRunningCompactionId_result result = new getRunningCompactionId_result();
+            if (e instanceof org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException) {
+              result.sec = (org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException) e;
+              result.setSecIsSet(true);
+              msg = result;
+            } else if (e instanceof org.apache.thrift.transport.TTransportException) {
+              _LOGGER.error("TTransportException inside handler", e);
+              fb.close();
+              return;
+            } else if (e instanceof org.apache.thrift.TApplicationException) {
+              _LOGGER.error("TApplicationException inside handler", e);
+              msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
+              msg = (org.apache.thrift.TApplicationException)e;
+            } else {
+              _LOGGER.error("Exception inside handler", e);
+              msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
+              msg = new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage());
+            }
+            try {
+              fcall.sendResponse(fb,msg,msgType,seqid);
+            } catch (java.lang.Exception ex) {
+              _LOGGER.error("Exception writing to internal frame buffer", ex);
+              fb.close();
+            }
+          }
+        };
+      }
+
+      protected boolean isOneway() {
+        return false;
+      }
+
+      public void start(I iface, getRunningCompactionId_args args, org.apache.thrift.async.AsyncMethodCallback<java.lang.String> resultHandler) throws org.apache.thrift.TException {
+        iface.getRunningCompactionId(args.tinfo, args.credentials,resultHandler);
+      }
+    }
+
+    public static class getActiveCompactions<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, getActiveCompactions_args, java.util.List<org.apache.accumulo.core.tabletserver.thrift.ActiveCompaction>> {
+      public getActiveCompactions() {
+        super("getActiveCompactions");
+      }
+
+      public getActiveCompactions_args getEmptyArgsInstance() {
+        return new getActiveCompactions_args();
+      }
+
+      public org.apache.thrift.async.AsyncMethodCallback<java.util.List<org.apache.accumulo.core.tabletserver.thrift.ActiveCompaction>> 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.util.List<org.apache.accumulo.core.tabletserver.thrift.ActiveCompaction>>() { 
+          public void onComplete(java.util.List<org.apache.accumulo.core.tabletserver.thrift.ActiveCompaction> o) {
+            getActiveCompactions_result result = new getActiveCompactions_result();
+            result.success = o;
+            try {
+              fcall.sendResponse(fb, result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
+            } catch (org.apache.thrift.transport.TTransportException e) {
+              _LOGGER.error("TTransportException writing to internal frame buffer", e);
+              fb.close();
+            } catch (java.lang.Exception e) {
+              _LOGGER.error("Exception writing to internal frame buffer", e);
+              onError(e);
+            }
+          }
+          public void onError(java.lang.Exception e) {
+            byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
+            org.apache.thrift.TSerializable msg;
+            getActiveCompactions_result result = new getActiveCompactions_result();
+            if (e instanceof org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException) {
+              result.sec = (org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException) e;
+              result.setSecIsSet(true);
+              msg = result;
+            } else if (e instanceof org.apache.thrift.transport.TTransportException) {
+              _LOGGER.error("TTransportException inside handler", e);
+              fb.close();
+              return;
+            } else if (e instanceof org.apache.thrift.TApplicationException) {
+              _LOGGER.error("TApplicationException inside handler", e);
+              msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
+              msg = (org.apache.thrift.TApplicationException)e;
+            } else {
+              _LOGGER.error("Exception inside handler", e);
+              msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
+              msg = new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage());
+            }
+            try {
+              fcall.sendResponse(fb,msg,msgType,seqid);
+            } catch (java.lang.Exception ex) {
+              _LOGGER.error("Exception writing to internal frame buffer", ex);
+              fb.close();
+            }
+          }
+        };
+      }
+
+      protected boolean isOneway() {
+        return false;
+      }
+
+      public void start(I iface, getActiveCompactions_args args, org.apache.thrift.async.AsyncMethodCallback<java.util.List<org.apache.accumulo.core.tabletserver.thrift.ActiveCompaction>> resultHandler) throws org.apache.thrift.TException {
+        iface.getActiveCompactions(args.tinfo, args.credentials,resultHandler);
+      }
+    }
+
+  }
+
+  public static class cancel_args implements org.apache.thrift.TBase<cancel_args, cancel_args._Fields>, java.io.Serializable, Cloneable, Comparable<cancel_args>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("cancel_args");
+
+    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 EXTERNAL_COMPACTION_ID_FIELD_DESC = new org.apache.thrift.protocol.TField("externalCompactionId", org.apache.thrift.protocol.TType.STRING, (short)3);
+
+    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new cancel_argsStandardSchemeFactory();
+    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new cancel_argsTupleSchemeFactory();
+
+    public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo; // required
+    public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials; // required
+    public @org.apache.thrift.annotation.Nullable java.lang.String externalCompactionId; // 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"),
+      EXTERNAL_COMPACTION_ID((short)3, "externalCompactionId");
+
+      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: // TINFO
+            return TINFO;
+          case 2: // CREDENTIALS
+            return CREDENTIALS;
+          case 3: // EXTERNAL_COMPACTION_ID
+            return EXTERNAL_COMPACTION_ID;
+          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;
+      }
+
+      public short getThriftFieldId() {
+        return _thriftId;
+      }
+
+      public java.lang.String getFieldName() {
+        return _fieldName;
+      }
+    }
+
+    // isset id assignments
+    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.TINFO, new org.apache.thrift.meta_data.FieldMetaData("tinfo", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.trace.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.EXTERNAL_COMPACTION_ID, new org.apache.thrift.meta_data.FieldMetaData("externalCompactionId", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+      metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(cancel_args.class, metaDataMap);
+    }
+
+    public cancel_args() {
+    }
+
+    public cancel_args(
+      org.apache.accumulo.core.trace.thrift.TInfo tinfo,
+      org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials,
+      java.lang.String externalCompactionId)
+    {
+      this();
+      this.tinfo = tinfo;
+      this.credentials = credentials;
+      this.externalCompactionId = externalCompactionId;
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public cancel_args(cancel_args other) {
+      if (other.isSetTinfo()) {
+        this.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo(other.tinfo);
+      }
+      if (other.isSetCredentials()) {
+        this.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(other.credentials);
+      }
+      if (other.isSetExternalCompactionId()) {
+        this.externalCompactionId = other.externalCompactionId;
+      }
+    }
+
+    public cancel_args deepCopy() {
+      return new cancel_args(this);
+    }
+
+    @Override
+    public void clear() {
+      this.tinfo = null;
+      this.credentials = null;
+      this.externalCompactionId = null;
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public org.apache.accumulo.core.trace.thrift.TInfo getTinfo() {
+      return this.tinfo;
+    }
+
+    public cancel_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo) {
+      this.tinfo = tinfo;
+      return this;
+    }
+
+    public void unsetTinfo() {
+      this.tinfo = null;
+    }
+
+    /** Returns true if field tinfo is set (has been assigned a value) and false otherwise */
+    public boolean isSetTinfo() {
+      return this.tinfo != null;
+    }
+
+    public void setTinfoIsSet(boolean value) {
+      if (!value) {
+        this.tinfo = null;
+      }
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public org.apache.accumulo.core.securityImpl.thrift.TCredentials getCredentials() {
+      return this.credentials;
+    }
+
+    public cancel_args setCredentials(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) {
+      this.credentials = credentials;
+      return this;
+    }
+
+    public void unsetCredentials() {
+      this.credentials = null;
+    }
+
+    /** Returns true if field credentials is set (has been assigned a value) and false otherwise */
+    public boolean isSetCredentials() {
+      return this.credentials != null;
+    }
+
+    public void setCredentialsIsSet(boolean value) {
+      if (!value) {
+        this.credentials = null;
+      }
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public java.lang.String getExternalCompactionId() {
+      return this.externalCompactionId;
+    }
+
+    public cancel_args setExternalCompactionId(@org.apache.thrift.annotation.Nullable java.lang.String externalCompactionId) {
+      this.externalCompactionId = externalCompactionId;
+      return this;
+    }
+
+    public void unsetExternalCompactionId() {
+      this.externalCompactionId = null;
+    }
+
+    /** Returns true if field externalCompactionId is set (has been assigned a value) and false otherwise */
+    public boolean isSetExternalCompactionId() {
+      return this.externalCompactionId != null;
+    }
+
+    public void setExternalCompactionIdIsSet(boolean value) {
+      if (!value) {
+        this.externalCompactionId = null;
+      }
+    }
+
+    public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable java.lang.Object value) {
+      switch (field) {
+      case TINFO:
+        if (value == null) {
+          unsetTinfo();
+        } else {
+          setTinfo((org.apache.accumulo.core.trace.thrift.TInfo)value);
+        }
+        break;
+
+      case CREDENTIALS:
+        if (value == null) {
+          unsetCredentials();
+        } else {
+          setCredentials((org.apache.accumulo.core.securityImpl.thrift.TCredentials)value);
+        }
+        break;
+
+      case EXTERNAL_COMPACTION_ID:
+        if (value == null) {
+          unsetExternalCompactionId();
+        } else {
+          setExternalCompactionId((java.lang.String)value);
+        }
+        break;
+
+      }
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public java.lang.Object getFieldValue(_Fields field) {
+      switch (field) {
+      case TINFO:
+        return getTinfo();
+
+      case CREDENTIALS:
+        return getCredentials();
+
+      case EXTERNAL_COMPACTION_ID:
+        return getExternalCompactionId();
+
+      }
+      throw new java.lang.IllegalStateException();
+    }
+
+    /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+    public boolean isSet(_Fields field) {
+      if (field == null) {
+        throw new java.lang.IllegalArgumentException();
+      }
+
+      switch (field) {
+      case TINFO:
+        return isSetTinfo();
+      case CREDENTIALS:
+        return isSetCredentials();
+      case EXTERNAL_COMPACTION_ID:
+        return isSetExternalCompactionId();
+      }
+      throw new java.lang.IllegalStateException();
+    }
+
+    @Override
+    public boolean equals(java.lang.Object that) {
+      if (that == null)
+        return false;
+      if (that instanceof cancel_args)
+        return this.equals((cancel_args)that);
+      return false;
+    }
+
+    public boolean equals(cancel_args that) {
+      if (that == null)
+        return false;
+      if (this == that)
+        return true;
+
+      boolean this_present_tinfo = true && this.isSetTinfo();
+      boolean that_present_tinfo = true && that.isSetTinfo();
+      if (this_present_tinfo || that_present_tinfo) {
+        if (!(this_present_tinfo && that_present_tinfo))
+          return false;
+        if (!this.tinfo.equals(that.tinfo))
+          return false;
+      }
+
+      boolean this_present_credentials = true && this.isSetCredentials();
+      boolean that_present_credentials = true && that.isSetCredentials();
+      if (this_present_credentials || that_present_credentials) {
+        if (!(this_present_credentials && that_present_credentials))
+          return false;
+        if (!this.credentials.equals(that.credentials))
+          return false;
+      }
+
+      boolean this_present_externalCompactionId = true && this.isSetExternalCompactionId();
+      boolean that_present_externalCompactionId = true && that.isSetExternalCompactionId();
+      if (this_present_externalCompactionId || that_present_externalCompactionId) {
+        if (!(this_present_externalCompactionId && that_present_externalCompactionId))
+          return false;
+        if (!this.externalCompactionId.equals(that.externalCompactionId))
+          return false;
+      }
+
+      return true;
+    }
+
+    @Override
+    public int hashCode() {
+      int hashCode = 1;
+
+      hashCode = hashCode * 8191 + ((isSetTinfo()) ? 131071 : 524287);
+      if (isSetTinfo())
+        hashCode = hashCode * 8191 + tinfo.hashCode();
+
+      hashCode = hashCode * 8191 + ((isSetCredentials()) ? 131071 : 524287);
+      if (isSetCredentials())
+        hashCode = hashCode * 8191 + credentials.hashCode();
+
+      hashCode = hashCode * 8191 + ((isSetExternalCompactionId()) ? 131071 : 524287);
+      if (isSetExternalCompactionId())
+        hashCode = hashCode * 8191 + externalCompactionId.hashCode();
+
+      return hashCode;
+    }
+
+    @Override
+    public int compareTo(cancel_args other) {
+      if (!getClass().equals(other.getClass())) {
+        return getClass().getName().compareTo(other.getClass().getName());
+      }
+
+      int lastComparison = 0;
+
+      lastComparison = java.lang.Boolean.valueOf(isSetTinfo()).compareTo(other.isSetTinfo());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetTinfo()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tinfo, other.tinfo);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      lastComparison = java.lang.Boolean.valueOf(isSetCredentials()).compareTo(other.isSetCredentials());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetCredentials()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.credentials, other.credentials);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      lastComparison = java.lang.Boolean.valueOf(isSetExternalCompactionId()).compareTo(other.isSetExternalCompactionId());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetExternalCompactionId()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.externalCompactionId, other.externalCompactionId);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      return 0;
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public _Fields fieldForId(int fieldId) {
+      return _Fields.findByThriftId(fieldId);
+    }
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+      scheme(iprot).read(iprot, this);
+    }
+
+    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("cancel_args(");
+      boolean first = true;
+
+      sb.append("tinfo:");
+      if (this.tinfo == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.tinfo);
+      }
+      first = false;
+      if (!first) sb.append(", ");
+      sb.append("credentials:");
+      if (this.credentials == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.credentials);
+      }
+      first = false;
+      if (!first) sb.append(", ");
+      sb.append("externalCompactionId:");
+      if (this.externalCompactionId == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.externalCompactionId);
+      }
+      first = false;
+      sb.append(")");
+      return sb.toString();
+    }
+
+    public void validate() throws org.apache.thrift.TException {
+      // check for required fields
+      // check for sub-struct validity
+      if (tinfo != null) {
+        tinfo.validate();
+      }
+      if (credentials != null) {
+        credentials.validate();
+      }
+    }
+
+    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 {
+        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 cancel_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public cancel_argsStandardScheme getScheme() {
+        return new cancel_argsStandardScheme();
+      }
+    }
+
+    private static class cancel_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme<cancel_args> {
+
+      public void read(org.apache.thrift.protocol.TProtocol iprot, cancel_args 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: // TINFO
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+                struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+                struct.tinfo.read(iprot);
+                struct.setTinfoIsSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            case 2: // CREDENTIALS
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+                struct.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials();
+                struct.credentials.read(iprot);
+                struct.setCredentialsIsSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            case 3: // EXTERNAL_COMPACTION_ID
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+                struct.externalCompactionId = iprot.readString();
+                struct.setExternalCompactionIdIsSet(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();
+      }
+
+      public void write(org.apache.thrift.protocol.TProtocol oprot, cancel_args struct) throws org.apache.thrift.TException {
+        struct.validate();
+
+        oprot.writeStructBegin(STRUCT_DESC);
+        if (struct.tinfo != null) {
+          oprot.writeFieldBegin(TINFO_FIELD_DESC);
+          struct.tinfo.write(oprot);
+          oprot.writeFieldEnd();
+        }
+        if (struct.credentials != null) {
+          oprot.writeFieldBegin(CREDENTIALS_FIELD_DESC);
+          struct.credentials.write(oprot);
+          oprot.writeFieldEnd();
+        }
+        if (struct.externalCompactionId != null) {
+          oprot.writeFieldBegin(EXTERNAL_COMPACTION_ID_FIELD_DESC);
+          oprot.writeString(struct.externalCompactionId);
+          oprot.writeFieldEnd();
+        }
+        oprot.writeFieldStop();
+        oprot.writeStructEnd();
+      }
+
+    }
+
+    private static class cancel_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public cancel_argsTupleScheme getScheme() {
+        return new cancel_argsTupleScheme();
+      }
+    }
+
+    private static class cancel_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme<cancel_args> {
+
+      @Override
+      public void write(org.apache.thrift.protocol.TProtocol prot, cancel_args 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.isSetTinfo()) {
+          optionals.set(0);
+        }
+        if (struct.isSetCredentials()) {
+          optionals.set(1);
+        }
+        if (struct.isSetExternalCompactionId()) {
+          optionals.set(2);
+        }
+        oprot.writeBitSet(optionals, 3);
+        if (struct.isSetTinfo()) {
+          struct.tinfo.write(oprot);
+        }
+        if (struct.isSetCredentials()) {
+          struct.credentials.write(oprot);
+        }
+        if (struct.isSetExternalCompactionId()) {
+          oprot.writeString(struct.externalCompactionId);
+        }
+      }
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol prot, cancel_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(3);
+        if (incoming.get(0)) {
+          struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+          struct.tinfo.read(iprot);
+          struct.setTinfoIsSet(true);
+        }
+        if (incoming.get(1)) {
+          struct.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials();
+          struct.credentials.read(iprot);
+          struct.setCredentialsIsSet(true);
+        }
+        if (incoming.get(2)) {
+          struct.externalCompactionId = iprot.readString();
+          struct.setExternalCompactionIdIsSet(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();
+    }
+  }
+
+  public static class cancel_result implements org.apache.thrift.TBase<cancel_result, cancel_result._Fields>, java.io.Serializable, Cloneable, Comparable<cancel_result>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("cancel_result");
+
+    private static final org.apache.thrift.protocol.TField E_FIELD_DESC = new org.apache.thrift.protocol.TField("e", org.apache.thrift.protocol.TType.STRUCT, (short)1);
+
+    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new cancel_resultStandardSchemeFactory();
+    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new cancel_resultTupleSchemeFactory();
+
+    public @org.apache.thrift.annotation.Nullable UnknownCompactionIdException e; // 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 {
+      E((short)1, "e");
+
+      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: // E
+            return E;
+          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;
+      }
+
+      public short getThriftFieldId() {
+        return _thriftId;
+      }
+
+      public java.lang.String getFieldName() {
+        return _fieldName;
+      }
+    }
+
+    // isset id assignments
+    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.E, new org.apache.thrift.meta_data.FieldMetaData("e", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, UnknownCompactionIdException.class)));
+      metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(cancel_result.class, metaDataMap);
+    }
+
+    public cancel_result() {
+    }
+
+    public cancel_result(
+      UnknownCompactionIdException e)
+    {
+      this();
+      this.e = e;
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public cancel_result(cancel_result other) {
+      if (other.isSetE()) {
+        this.e = new UnknownCompactionIdException(other.e);
+      }
+    }
+
+    public cancel_result deepCopy() {
+      return new cancel_result(this);
+    }
+
+    @Override
+    public void clear() {
+      this.e = null;
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public UnknownCompactionIdException getE() {
+      return this.e;
+    }
+
+    public cancel_result setE(@org.apache.thrift.annotation.Nullable UnknownCompactionIdException e) {
+      this.e = e;
+      return this;
+    }
+
+    public void unsetE() {
+      this.e = null;
+    }
+
+    /** Returns true if field e is set (has been assigned a value) and false otherwise */
+    public boolean isSetE() {
+      return this.e != null;
+    }
+
+    public void setEIsSet(boolean value) {
+      if (!value) {
+        this.e = null;
+      }
+    }
+
+    public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable java.lang.Object value) {
+      switch (field) {
+      case E:
+        if (value == null) {
+          unsetE();
+        } else {
+          setE((UnknownCompactionIdException)value);
+        }
+        break;
+
+      }
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public java.lang.Object getFieldValue(_Fields field) {
+      switch (field) {
+      case E:
+        return getE();
+
+      }
+      throw new java.lang.IllegalStateException();
+    }
+
+    /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+    public boolean isSet(_Fields field) {
+      if (field == null) {
+        throw new java.lang.IllegalArgumentException();
+      }
+
+      switch (field) {
+      case E:
+        return isSetE();
+      }
+      throw new java.lang.IllegalStateException();
+    }
+
+    @Override
+    public boolean equals(java.lang.Object that) {
+      if (that == null)
+        return false;
+      if (that instanceof cancel_result)
+        return this.equals((cancel_result)that);
+      return false;
+    }
+
+    public boolean equals(cancel_result that) {
+      if (that == null)
+        return false;
+      if (this == that)
+        return true;
+
+      boolean this_present_e = true && this.isSetE();
+      boolean that_present_e = true && that.isSetE();
+      if (this_present_e || that_present_e) {
+        if (!(this_present_e && that_present_e))
+          return false;
+        if (!this.e.equals(that.e))
+          return false;
+      }
+
+      return true;
+    }
+
+    @Override
+    public int hashCode() {
+      int hashCode = 1;
+
+      hashCode = hashCode * 8191 + ((isSetE()) ? 131071 : 524287);
+      if (isSetE())
+        hashCode = hashCode * 8191 + e.hashCode();
+
+      return hashCode;
+    }
+
+    @Override
+    public int compareTo(cancel_result other) {
+      if (!getClass().equals(other.getClass())) {
+        return getClass().getName().compareTo(other.getClass().getName());
+      }
+
+      int lastComparison = 0;
+
+      lastComparison = java.lang.Boolean.valueOf(isSetE()).compareTo(other.isSetE());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetE()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.e, other.e);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      return 0;
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public _Fields fieldForId(int fieldId) {
+      return _Fields.findByThriftId(fieldId);
+    }
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+      scheme(iprot).read(iprot, this);
+    }
+
+    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("cancel_result(");
+      boolean first = true;
+
+      sb.append("e:");
+      if (this.e == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.e);
+      }
+      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 {
+        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 cancel_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public cancel_resultStandardScheme getScheme() {
+        return new cancel_resultStandardScheme();
+      }
+    }
+
+    private static class cancel_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme<cancel_result> {
+
+      public void read(org.apache.thrift.protocol.TProtocol iprot, cancel_result 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: // E
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+                struct.e = new UnknownCompactionIdException();
+                struct.e.read(iprot);
+                struct.setEIsSet(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();
+      }
+
+      public void write(org.apache.thrift.protocol.TProtocol oprot, cancel_result struct) throws org.apache.thrift.TException {
+        struct.validate();
+
+        oprot.writeStructBegin(STRUCT_DESC);
+        if (struct.e != null) {
+          oprot.writeFieldBegin(E_FIELD_DESC);
+          struct.e.write(oprot);
+          oprot.writeFieldEnd();
+        }
+        oprot.writeFieldStop();
+        oprot.writeStructEnd();
+      }
+
+    }
+
+    private static class cancel_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public cancel_resultTupleScheme getScheme() {
+        return new cancel_resultTupleScheme();
+      }
+    }
+
+    private static class cancel_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme<cancel_result> {
+
+      @Override
+      public void write(org.apache.thrift.protocol.TProtocol prot, cancel_result 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.isSetE()) {
+          optionals.set(0);
+        }
+        oprot.writeBitSet(optionals, 1);
+        if (struct.isSetE()) {
+          struct.e.write(oprot);
+        }
+      }
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol prot, cancel_result struct) throws org.apache.thrift.TException {
+        org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
+        java.util.BitSet incoming = iprot.readBitSet(1);
+        if (incoming.get(0)) {
+          struct.e = new UnknownCompactionIdException();
+          struct.e.read(iprot);
+          struct.setEIsSet(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();
+    }
+  }
+
+  public static class getRunningCompaction_args implements org.apache.thrift.TBase<getRunningCompaction_args, getRunningCompaction_args._Fields>, java.io.Serializable, Cloneable, Comparable<getRunningCompaction_args>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getRunningCompaction_args");
+
+    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.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new getRunningCompaction_argsStandardSchemeFactory();
+    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new getRunningCompaction_argsTupleSchemeFactory();
+
+    public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo; // required
+    public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials; // 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");
+
+      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: // TINFO
+            return TINFO;
+          case 2: // CREDENTIALS
+            return CREDENTIALS;
+          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;
+      }
+
+      public short getThriftFieldId() {
+        return _thriftId;
+      }
+
+      public java.lang.String getFieldName() {
+        return _fieldName;
+      }
+    }
+
+    // isset id assignments
+    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.TINFO, new org.apache.thrift.meta_data.FieldMetaData("tinfo", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.trace.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)));
+      metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(getRunningCompaction_args.class, metaDataMap);
+    }
+
+    public getRunningCompaction_args() {
+    }
+
+    public getRunningCompaction_args(
+      org.apache.accumulo.core.trace.thrift.TInfo tinfo,
+      org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials)
+    {
+      this();
+      this.tinfo = tinfo;
+      this.credentials = credentials;
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public getRunningCompaction_args(getRunningCompaction_args other) {
+      if (other.isSetTinfo()) {
+        this.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo(other.tinfo);
+      }
+      if (other.isSetCredentials()) {
+        this.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(other.credentials);
+      }
+    }
+
+    public getRunningCompaction_args deepCopy() {
+      return new getRunningCompaction_args(this);
+    }
+
+    @Override
+    public void clear() {
+      this.tinfo = null;
+      this.credentials = null;
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public org.apache.accumulo.core.trace.thrift.TInfo getTinfo() {
+      return this.tinfo;
+    }
+
+    public getRunningCompaction_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo) {
+      this.tinfo = tinfo;
+      return this;
+    }
+
+    public void unsetTinfo() {
+      this.tinfo = null;
+    }
+
+    /** Returns true if field tinfo is set (has been assigned a value) and false otherwise */
+    public boolean isSetTinfo() {
+      return this.tinfo != null;
+    }
+
+    public void setTinfoIsSet(boolean value) {
+      if (!value) {
+        this.tinfo = null;
+      }
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public org.apache.accumulo.core.securityImpl.thrift.TCredentials getCredentials() {
+      return this.credentials;
+    }
+
+    public getRunningCompaction_args setCredentials(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) {
+      this.credentials = credentials;
+      return this;
+    }
+
+    public void unsetCredentials() {
+      this.credentials = null;
+    }
+
+    /** Returns true if field credentials is set (has been assigned a value) and false otherwise */
+    public boolean isSetCredentials() {
+      return this.credentials != null;
+    }
+
+    public void setCredentialsIsSet(boolean value) {
+      if (!value) {
+        this.credentials = null;
+      }
+    }
+
+    public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable java.lang.Object value) {
+      switch (field) {
+      case TINFO:
+        if (value == null) {
+          unsetTinfo();
+        } else {
+          setTinfo((org.apache.accumulo.core.trace.thrift.TInfo)value);
+        }
+        break;
+
+      case CREDENTIALS:
+        if (value == null) {
+          unsetCredentials();
+        } else {
+          setCredentials((org.apache.accumulo.core.securityImpl.thrift.TCredentials)value);
+        }
+        break;
+
+      }
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public java.lang.Object getFieldValue(_Fields field) {
+      switch (field) {
+      case TINFO:
+        return getTinfo();
+
+      case CREDENTIALS:
+        return getCredentials();
+
+      }
+      throw new java.lang.IllegalStateException();
+    }
+
+    /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+    public boolean isSet(_Fields field) {
+      if (field == null) {
+        throw new java.lang.IllegalArgumentException();
+      }
+
+      switch (field) {
+      case TINFO:
+        return isSetTinfo();
+      case CREDENTIALS:
+        return isSetCredentials();
+      }
+      throw new java.lang.IllegalStateException();
+    }
+
+    @Override
+    public boolean equals(java.lang.Object that) {
+      if (that == null)
+        return false;
+      if (that instanceof getRunningCompaction_args)
+        return this.equals((getRunningCompaction_args)that);
+      return false;
+    }
+
+    public boolean equals(getRunningCompaction_args that) {
+      if (that == null)
+        return false;
+      if (this == that)
+        return true;
+
+      boolean this_present_tinfo = true && this.isSetTinfo();
+      boolean that_present_tinfo = true && that.isSetTinfo();
+      if (this_present_tinfo || that_present_tinfo) {
+        if (!(this_present_tinfo && that_present_tinfo))
+          return false;
+        if (!this.tinfo.equals(that.tinfo))
+          return false;
+      }
+
+      boolean this_present_credentials = true && this.isSetCredentials();
+      boolean that_present_credentials = true && that.isSetCredentials();
+      if (this_present_credentials || that_present_credentials) {
+        if (!(this_present_credentials && that_present_credentials))
+          return false;
+        if (!this.credentials.equals(that.credentials))
+          return false;
+      }
+
+      return true;
+    }
+
+    @Override
+    public int hashCode() {
+      int hashCode = 1;
+
+      hashCode = hashCode * 8191 + ((isSetTinfo()) ? 131071 : 524287);
+      if (isSetTinfo())
+        hashCode = hashCode * 8191 + tinfo.hashCode();
+
+      hashCode = hashCode * 8191 + ((isSetCredentials()) ? 131071 : 524287);
+      if (isSetCredentials())
+        hashCode = hashCode * 8191 + credentials.hashCode();
+
+      return hashCode;
+    }
+
+    @Override
+    public int compareTo(getRunningCompaction_args other) {
+      if (!getClass().equals(other.getClass())) {
+        return getClass().getName().compareTo(other.getClass().getName());
+      }
+
+      int lastComparison = 0;
+
+      lastComparison = java.lang.Boolean.valueOf(isSetTinfo()).compareTo(other.isSetTinfo());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetTinfo()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tinfo, other.tinfo);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      lastComparison = java.lang.Boolean.valueOf(isSetCredentials()).compareTo(other.isSetCredentials());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetCredentials()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.credentials, other.credentials);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      return 0;
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public _Fields fieldForId(int fieldId) {
+      return _Fields.findByThriftId(fieldId);
+    }
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+      scheme(iprot).read(iprot, this);
+    }
+
+    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("getRunningCompaction_args(");
+      boolean first = true;
+
+      sb.append("tinfo:");
+      if (this.tinfo == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.tinfo);
+      }
+      first = false;
+      if (!first) sb.append(", ");
+      sb.append("credentials:");
+      if (this.credentials == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.credentials);
+      }
+      first = false;
+      sb.append(")");
+      return sb.toString();
+    }
+
+    public void validate() throws org.apache.thrift.TException {
+      // check for required fields
+      // check for sub-struct validity
+      if (tinfo != null) {
+        tinfo.validate();
+      }
+      if (credentials != null) {
+        credentials.validate();
+      }
+    }
+
+    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 {
+        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 getRunningCompaction_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public getRunningCompaction_argsStandardScheme getScheme() {
+        return new getRunningCompaction_argsStandardScheme();
+      }
+    }
+
+    private static class getRunningCompaction_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme<getRunningCompaction_args> {
+
+      public void read(org.apache.thrift.protocol.TProtocol iprot, getRunningCompaction_args 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: // TINFO
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+                struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+                struct.tinfo.read(iprot);
+                struct.setTinfoIsSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            case 2: // CREDENTIALS
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+                struct.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials();
+                struct.credentials.read(iprot);
+                struct.setCredentialsIsSet(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();
+      }
+
+      public void write(org.apache.thrift.protocol.TProtocol oprot, getRunningCompaction_args struct) throws org.apache.thrift.TException {
+        struct.validate();
+
+        oprot.writeStructBegin(STRUCT_DESC);
+        if (struct.tinfo != null) {
+          oprot.writeFieldBegin(TINFO_FIELD_DESC);
+          struct.tinfo.write(oprot);
+          oprot.writeFieldEnd();
+        }
+        if (struct.credentials != null) {
+          oprot.writeFieldBegin(CREDENTIALS_FIELD_DESC);
+          struct.credentials.write(oprot);
+          oprot.writeFieldEnd();
+        }
+        oprot.writeFieldStop();
+        oprot.writeStructEnd();
+      }
+
+    }
+
+    private static class getRunningCompaction_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public getRunningCompaction_argsTupleScheme getScheme() {
+        return new getRunningCompaction_argsTupleScheme();
+      }
+    }
+
+    private static class getRunningCompaction_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme<getRunningCompaction_args> {
+
+      @Override
+      public void write(org.apache.thrift.protocol.TProtocol prot, getRunningCompaction_args 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.isSetTinfo()) {
+          optionals.set(0);
+        }
+        if (struct.isSetCredentials()) {
+          optionals.set(1);
+        }
+        oprot.writeBitSet(optionals, 2);
+        if (struct.isSetTinfo()) {
+          struct.tinfo.write(oprot);
+        }
+        if (struct.isSetCredentials()) {
+          struct.credentials.write(oprot);
+        }
+      }
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol prot, getRunningCompaction_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);
+        if (incoming.get(0)) {
+          struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+          struct.tinfo.read(iprot);
+          struct.setTinfoIsSet(true);
+        }
+        if (incoming.get(1)) {
+          struct.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials();
+          struct.credentials.read(iprot);
+          struct.setCredentialsIsSet(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();
+    }
+  }
+
+  public static class getRunningCompaction_result implements org.apache.thrift.TBase<getRunningCompaction_result, getRunningCompaction_result._Fields>, java.io.Serializable, Cloneable, Comparable<getRunningCompaction_result>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getRunningCompaction_result");
+
+    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.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new getRunningCompaction_resultStandardSchemeFactory();
+    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new getRunningCompaction_resultTupleSchemeFactory();
+
+    public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.tabletserver.thrift.TExternalCompactionJob success; // required
+    public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException sec; // 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 {
+      SUCCESS((short)0, "success"),
+      SEC((short)1, "sec");
+
+      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 0: // SUCCESS
+            return SUCCESS;
+          case 1: // SEC
+            return SEC;
+          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;
+      }
+
+      public short getThriftFieldId() {
+        return _thriftId;
+      }
+
+      public java.lang.String getFieldName() {
+        return _fieldName;
+      }
+    }
+
+    // isset id assignments
+    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.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.tabletserver.thrift.TExternalCompactionJob.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)));
+      metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(getRunningCompaction_result.class, metaDataMap);
+    }
+
+    public getRunningCompaction_result() {
+    }
+
+    public getRunningCompaction_result(
+      org.apache.accumulo.core.tabletserver.thrift.TExternalCompactionJob success,
+      org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException sec)
+    {
+      this();
+      this.success = success;
+      this.sec = sec;
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public getRunningCompaction_result(getRunningCompaction_result other) {
+      if (other.isSetSuccess()) {
+        this.success = new org.apache.accumulo.core.tabletserver.thrift.TExternalCompactionJob(other.success);
+      }
+      if (other.isSetSec()) {
+        this.sec = new org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException(other.sec);
+      }
+    }
+
+    public getRunningCompaction_result deepCopy() {
+      return new getRunningCompaction_result(this);
+    }
+
+    @Override
+    public void clear() {
+      this.success = null;
+      this.sec = null;
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public org.apache.accumulo.core.tabletserver.thrift.TExternalCompactionJob getSuccess() {
+      return this.success;
+    }
+
+    public getRunningCompaction_result setSuccess(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.tabletserver.thrift.TExternalCompactionJob success) {
+      this.success = success;
+      return this;
+    }
+
+    public void unsetSuccess() {
+      this.success = null;
+    }
+
+    /** Returns true if field success is set (has been assigned a value) and false otherwise */
+    public boolean isSetSuccess() {
+      return this.success != null;
+    }
+
+    public void setSuccessIsSet(boolean value) {
+      if (!value) {
+        this.success = null;
+      }
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException getSec() {
+      return this.sec;
+    }
+
+    public getRunningCompaction_result setSec(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException sec) {
+      this.sec = sec;
+      return this;
+    }
+
+    public void unsetSec() {
+      this.sec = null;
+    }
+
+    /** Returns true if field sec is set (has been assigned a value) and false otherwise */
+    public boolean isSetSec() {
+      return this.sec != null;
+    }
+
+    public void setSecIsSet(boolean value) {
+      if (!value) {
+        this.sec = null;
+      }
+    }
+
+    public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable java.lang.Object value) {
+      switch (field) {
+      case SUCCESS:
+        if (value == null) {
+          unsetSuccess();
+        } else {
+          setSuccess((org.apache.accumulo.core.tabletserver.thrift.TExternalCompactionJob)value);
+        }
+        break;
+
+      case SEC:
+        if (value == null) {
+          unsetSec();
+        } else {
+          setSec((org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException)value);
+        }
+        break;
+
+      }
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public java.lang.Object getFieldValue(_Fields field) {
+      switch (field) {
+      case SUCCESS:
+        return getSuccess();
+
+      case SEC:
+        return getSec();
+
+      }
+      throw new java.lang.IllegalStateException();
+    }
+
+    /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+    public boolean isSet(_Fields field) {
+      if (field == null) {
+        throw new java.lang.IllegalArgumentException();
+      }
+
+      switch (field) {
+      case SUCCESS:
+        return isSetSuccess();
+      case SEC:
+        return isSetSec();
+      }
+      throw new java.lang.IllegalStateException();
+    }
+
+    @Override
+    public boolean equals(java.lang.Object that) {
+      if (that == null)
+        return false;
+      if (that instanceof getRunningCompaction_result)
+        return this.equals((getRunningCompaction_result)that);
+      return false;
+    }
+
+    public boolean equals(getRunningCompaction_result that) {
+      if (that == null)
+        return false;
+      if (this == that)
+        return 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.equals(that.success))
+          return false;
+      }
+
+      boolean this_present_sec = true && this.isSetSec();
+      boolean that_present_sec = true && that.isSetSec();
+      if (this_present_sec || that_present_sec) {
+        if (!(this_present_sec && that_present_sec))
+          return false;
+        if (!this.sec.equals(that.sec))
+          return false;
+      }
+
+      return true;
+    }
+
+    @Override
+    public int hashCode() {
+      int hashCode = 1;
+
+      hashCode = hashCode * 8191 + ((isSetSuccess()) ? 131071 : 524287);
+      if (isSetSuccess())
+        hashCode = hashCode * 8191 + success.hashCode();
+
+      hashCode = hashCode * 8191 + ((isSetSec()) ? 131071 : 524287);
+      if (isSetSec())
+        hashCode = hashCode * 8191 + sec.hashCode();
+
+      return hashCode;
+    }
+
+    @Override
+    public int compareTo(getRunningCompaction_result other) {
+      if (!getClass().equals(other.getClass())) {
+        return getClass().getName().compareTo(other.getClass().getName());
+      }
+
+      int lastComparison = 0;
+
+      lastComparison = java.lang.Boolean.valueOf(isSetSuccess()).compareTo(other.isSetSuccess());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetSuccess()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, other.success);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      lastComparison = java.lang.Boolean.valueOf(isSetSec()).compareTo(other.isSetSec());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetSec()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.sec, other.sec);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      return 0;
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public _Fields fieldForId(int fieldId) {
+      return _Fields.findByThriftId(fieldId);
+    }
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+      scheme(iprot).read(iprot, this);
+    }
+
+    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("getRunningCompaction_result(");
+      boolean first = true;
+
+      sb.append("success:");
+      if (this.success == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.success);
+      }
+      first = false;
+      if (!first) sb.append(", ");
+      sb.append("sec:");
+      if (this.sec == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.sec);
+      }
+      first = false;
+      sb.append(")");
+      return sb.toString();
+    }
+
+    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 {
+      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 {
+        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 getRunningCompaction_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public getRunningCompaction_resultStandardScheme getScheme() {
+        return new getRunningCompaction_resultStandardScheme();
+      }
+    }
+
+    private static class getRunningCompaction_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme<getRunningCompaction_result> {
+
+      public void read(org.apache.thrift.protocol.TProtocol iprot, getRunningCompaction_result 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 0: // SUCCESS
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+                struct.success = new org.apache.accumulo.core.tabletserver.thrift.TExternalCompactionJob();
+                struct.success.read(iprot);
+                struct.setSuccessIsSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            case 1: // SEC
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+                struct.sec = new org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException();
+                struct.sec.read(iprot);
+                struct.setSecIsSet(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();
+      }
+
+      public void write(org.apache.thrift.protocol.TProtocol oprot, getRunningCompaction_result struct) throws org.apache.thrift.TException {
+        struct.validate();
+
+        oprot.writeStructBegin(STRUCT_DESC);
+        if (struct.success != null) {
+          oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
+          struct.success.write(oprot);
+          oprot.writeFieldEnd();
+        }
+        if (struct.sec != null) {
+          oprot.writeFieldBegin(SEC_FIELD_DESC);
+          struct.sec.write(oprot);
+          oprot.writeFieldEnd();
+        }
+        oprot.writeFieldStop();
+        oprot.writeStructEnd();
+      }
+
+    }
+
+    private static class getRunningCompaction_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public getRunningCompaction_resultTupleScheme getScheme() {
+        return new getRunningCompaction_resultTupleScheme();
+      }
+    }
+
+    private static class getRunningCompaction_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme<getRunningCompaction_result> {
+
+      @Override
+      public void write(org.apache.thrift.protocol.TProtocol prot, getRunningCompaction_result 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.isSetSuccess()) {
+          optionals.set(0);
+        }
+        if (struct.isSetSec()) {
+          optionals.set(1);
+        }
+        oprot.writeBitSet(optionals, 2);
+        if (struct.isSetSuccess()) {
+          struct.success.write(oprot);
+        }
+        if (struct.isSetSec()) {
+          struct.sec.write(oprot);
+        }
... 24334 lines suppressed ...