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/03/02 16:50:40 UTC

[accumulo] branch 1451-external-compactions-feature updated: re #1451: updated thrift rpc from latest design document

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

dlmarion pushed a commit to branch 1451-external-compactions-feature
in repository https://gitbox.apache.org/repos/asf/accumulo.git


The following commit(s) were added to refs/heads/1451-external-compactions-feature by this push:
     new 368abed  re #1451: updated thrift rpc from latest design document
368abed is described below

commit 368abed810cfd0e8ce29c831067dea6e3ae50b31
Author: Dave Marion <dl...@apache.org>
AuthorDate: Tue Mar 2 16:50:08 2021 +0000

    re #1451: updated thrift rpc from latest design document
---
 .../compaction/thrift/CompactionCoordinator.java   | 2145 +++++------
 .../core/compaction/thrift/CompactionStats.java    |  582 +++
 .../accumulo/core/compaction/thrift/Compactor.java |   32 +-
 .../accumulo/core/compaction/thrift/Status.java    |  108 +-
 .../core/dataImpl/thrift/CompactionStats.java      |  582 +++
 .../core/tabletserver/thrift/CompactionJob.java    | 1620 ++++++++
 .../thrift/CompactionQueueSummary.java             |  591 +++
 .../tabletserver/thrift/TabletClientService.java   | 3915 +++++++++++++++++---
 core/src/main/thrift/compaction-coordinator.thrift |   67 +-
 core/src/main/thrift/data.thrift                   |    7 +
 core/src/main/thrift/tabletserver.thrift           |   34 +
 11 files changed, 7709 insertions(+), 1974 deletions(-)

diff --git a/core/src/main/java/org/apache/accumulo/core/compaction/thrift/CompactionCoordinator.java b/core/src/main/java/org/apache/accumulo/core/compaction/thrift/CompactionCoordinator.java
index b5af1e2..4f4fb31 100644
--- a/core/src/main/java/org/apache/accumulo/core/compaction/thrift/CompactionCoordinator.java
+++ b/core/src/main/java/org/apache/accumulo/core/compaction/thrift/CompactionCoordinator.java
@@ -29,29 +29,29 @@ public class CompactionCoordinator {
 
   public interface Iface {
 
-    public void AddOrUpdateCompaction(org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent, java.lang.String queueName, long priority, Action action) throws org.apache.thrift.TException;
+    public void cancelCompaction(org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent, java.lang.String queueName, long priority) throws org.apache.thrift.TException;
 
-    public Status getCompactionStatus(java.lang.String tableId, java.lang.String endRow, java.lang.String queueName) throws org.apache.thrift.TException;
+    public java.util.List<Status> getCompactionStatus(org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent, java.lang.String queueName, long priority) throws org.apache.thrift.TException;
 
-    public java.util.List<Status> getCompactionStatusHistory(java.lang.String tableId, java.lang.String endRow, java.lang.String queueName) throws org.apache.thrift.TException;
+    public org.apache.accumulo.core.tabletserver.thrift.CompactionJob getCompactionJob(java.lang.String queueName, java.lang.String compactor) throws org.apache.thrift.TException;
 
-    public CompactionJob getCompactionJob(java.lang.String queueName, long compactorHostAndPort) throws org.apache.thrift.TException;
+    public void compactionCompleted(org.apache.accumulo.core.tabletserver.thrift.CompactionJob job, org.apache.accumulo.core.dataImpl.thrift.CompactionStats stats) throws org.apache.thrift.TException;
 
-    public void updateCompactionState(CompactionJob compaction, CompactionState state, java.lang.String message, long timestamp) throws org.apache.thrift.TException;
+    public void updateCompactionStatus(org.apache.accumulo.core.tabletserver.thrift.CompactionJob compaction, CompactionState state, java.lang.String message, long timestamp) throws org.apache.thrift.TException;
 
   }
 
   public interface AsyncIface {
 
-    public void AddOrUpdateCompaction(org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent, java.lang.String queueName, long priority, Action action, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
+    public void cancelCompaction(org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent, java.lang.String queueName, long priority, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
 
-    public void getCompactionStatus(java.lang.String tableId, java.lang.String endRow, java.lang.String queueName, org.apache.thrift.async.AsyncMethodCallback<Status> resultHandler) throws org.apache.thrift.TException;
+    public void getCompactionStatus(org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent, java.lang.String queueName, long priority, org.apache.thrift.async.AsyncMethodCallback<java.util.List<Status>> resultHandler) throws org.apache.thrift.TException;
 
-    public void getCompactionStatusHistory(java.lang.String tableId, java.lang.String endRow, java.lang.String queueName, org.apache.thrift.async.AsyncMethodCallback<java.util.List<Status>> resultHandler) throws org.apache.thrift.TException;
+    public void getCompactionJob(java.lang.String queueName, java.lang.String compactor, org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.tabletserver.thrift.CompactionJob> resultHandler) throws org.apache.thrift.TException;
 
-    public void getCompactionJob(java.lang.String queueName, long compactorHostAndPort, org.apache.thrift.async.AsyncMethodCallback<CompactionJob> resultHandler) throws org.apache.thrift.TException;
+    public void compactionCompleted(org.apache.accumulo.core.tabletserver.thrift.CompactionJob job, org.apache.accumulo.core.dataImpl.thrift.CompactionStats stats, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
 
-    public void updateCompactionState(CompactionJob compaction, CompactionState state, java.lang.String message, long timestamp, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
+    public void updateCompactionStatus(org.apache.accumulo.core.tabletserver.thrift.CompactionJob compaction, CompactionState state, java.lang.String message, long timestamp, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
 
   }
 
@@ -75,45 +75,44 @@ public class CompactionCoordinator {
       super(iprot, oprot);
     }
 
-    public void AddOrUpdateCompaction(org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent, java.lang.String queueName, long priority, Action action) throws org.apache.thrift.TException
+    public void cancelCompaction(org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent, java.lang.String queueName, long priority) throws org.apache.thrift.TException
     {
-      send_AddOrUpdateCompaction(extent, queueName, priority, action);
-      recv_AddOrUpdateCompaction();
+      send_cancelCompaction(extent, queueName, priority);
+      recv_cancelCompaction();
     }
 
-    public void send_AddOrUpdateCompaction(org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent, java.lang.String queueName, long priority, Action action) throws org.apache.thrift.TException
+    public void send_cancelCompaction(org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent, java.lang.String queueName, long priority) throws org.apache.thrift.TException
     {
-      AddOrUpdateCompaction_args args = new AddOrUpdateCompaction_args();
+      cancelCompaction_args args = new cancelCompaction_args();
       args.setExtent(extent);
       args.setQueueName(queueName);
       args.setPriority(priority);
-      args.setAction(action);
-      sendBase("AddOrUpdateCompaction", args);
+      sendBase("cancelCompaction", args);
     }
 
-    public void recv_AddOrUpdateCompaction() throws org.apache.thrift.TException
+    public void recv_cancelCompaction() throws org.apache.thrift.TException
     {
-      AddOrUpdateCompaction_result result = new AddOrUpdateCompaction_result();
-      receiveBase(result, "AddOrUpdateCompaction");
+      cancelCompaction_result result = new cancelCompaction_result();
+      receiveBase(result, "cancelCompaction");
       return;
     }
 
-    public Status getCompactionStatus(java.lang.String tableId, java.lang.String endRow, java.lang.String queueName) throws org.apache.thrift.TException
+    public java.util.List<Status> getCompactionStatus(org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent, java.lang.String queueName, long priority) throws org.apache.thrift.TException
     {
-      send_getCompactionStatus(tableId, endRow, queueName);
+      send_getCompactionStatus(extent, queueName, priority);
       return recv_getCompactionStatus();
     }
 
-    public void send_getCompactionStatus(java.lang.String tableId, java.lang.String endRow, java.lang.String queueName) throws org.apache.thrift.TException
+    public void send_getCompactionStatus(org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent, java.lang.String queueName, long priority) throws org.apache.thrift.TException
     {
       getCompactionStatus_args args = new getCompactionStatus_args();
-      args.setTableId(tableId);
-      args.setEndRow(endRow);
+      args.setExtent(extent);
       args.setQueueName(queueName);
+      args.setPriority(priority);
       sendBase("getCompactionStatus", args);
     }
 
-    public Status recv_getCompactionStatus() throws org.apache.thrift.TException
+    public java.util.List<Status> recv_getCompactionStatus() throws org.apache.thrift.TException
     {
       getCompactionStatus_result result = new getCompactionStatus_result();
       receiveBase(result, "getCompactionStatus");
@@ -123,75 +122,71 @@ public class CompactionCoordinator {
       throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "getCompactionStatus failed: unknown result");
     }
 
-    public java.util.List<Status> getCompactionStatusHistory(java.lang.String tableId, java.lang.String endRow, java.lang.String queueName) throws org.apache.thrift.TException
+    public org.apache.accumulo.core.tabletserver.thrift.CompactionJob getCompactionJob(java.lang.String queueName, java.lang.String compactor) throws org.apache.thrift.TException
     {
-      send_getCompactionStatusHistory(tableId, endRow, queueName);
-      return recv_getCompactionStatusHistory();
+      send_getCompactionJob(queueName, compactor);
+      return recv_getCompactionJob();
     }
 
-    public void send_getCompactionStatusHistory(java.lang.String tableId, java.lang.String endRow, java.lang.String queueName) throws org.apache.thrift.TException
+    public void send_getCompactionJob(java.lang.String queueName, java.lang.String compactor) throws org.apache.thrift.TException
     {
-      getCompactionStatusHistory_args args = new getCompactionStatusHistory_args();
-      args.setTableId(tableId);
-      args.setEndRow(endRow);
+      getCompactionJob_args args = new getCompactionJob_args();
       args.setQueueName(queueName);
-      sendBase("getCompactionStatusHistory", args);
+      args.setCompactor(compactor);
+      sendBase("getCompactionJob", args);
     }
 
-    public java.util.List<Status> recv_getCompactionStatusHistory() throws org.apache.thrift.TException
+    public org.apache.accumulo.core.tabletserver.thrift.CompactionJob recv_getCompactionJob() throws org.apache.thrift.TException
     {
-      getCompactionStatusHistory_result result = new getCompactionStatusHistory_result();
-      receiveBase(result, "getCompactionStatusHistory");
+      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, "getCompactionStatusHistory failed: unknown result");
+      throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "getCompactionJob failed: unknown result");
     }
 
-    public CompactionJob getCompactionJob(java.lang.String queueName, long compactorHostAndPort) throws org.apache.thrift.TException
+    public void compactionCompleted(org.apache.accumulo.core.tabletserver.thrift.CompactionJob job, org.apache.accumulo.core.dataImpl.thrift.CompactionStats stats) throws org.apache.thrift.TException
     {
-      send_getCompactionJob(queueName, compactorHostAndPort);
-      return recv_getCompactionJob();
+      send_compactionCompleted(job, stats);
+      recv_compactionCompleted();
     }
 
-    public void send_getCompactionJob(java.lang.String queueName, long compactorHostAndPort) throws org.apache.thrift.TException
+    public void send_compactionCompleted(org.apache.accumulo.core.tabletserver.thrift.CompactionJob job, org.apache.accumulo.core.dataImpl.thrift.CompactionStats stats) throws org.apache.thrift.TException
     {
-      getCompactionJob_args args = new getCompactionJob_args();
-      args.setQueueName(queueName);
-      args.setCompactorHostAndPort(compactorHostAndPort);
-      sendBase("getCompactionJob", args);
+      compactionCompleted_args args = new compactionCompleted_args();
+      args.setJob(job);
+      args.setStats(stats);
+      sendBase("compactionCompleted", args);
     }
 
-    public CompactionJob recv_getCompactionJob() throws org.apache.thrift.TException
+    public void recv_compactionCompleted() 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");
+      compactionCompleted_result result = new compactionCompleted_result();
+      receiveBase(result, "compactionCompleted");
+      return;
     }
 
-    public void updateCompactionState(CompactionJob compaction, CompactionState state, java.lang.String message, long timestamp) throws org.apache.thrift.TException
+    public void updateCompactionStatus(org.apache.accumulo.core.tabletserver.thrift.CompactionJob compaction, CompactionState state, java.lang.String message, long timestamp) throws org.apache.thrift.TException
     {
-      send_updateCompactionState(compaction, state, message, timestamp);
-      recv_updateCompactionState();
+      send_updateCompactionStatus(compaction, state, message, timestamp);
+      recv_updateCompactionStatus();
     }
 
-    public void send_updateCompactionState(CompactionJob compaction, CompactionState state, java.lang.String message, long timestamp) throws org.apache.thrift.TException
+    public void send_updateCompactionStatus(org.apache.accumulo.core.tabletserver.thrift.CompactionJob compaction, CompactionState state, java.lang.String message, long timestamp) throws org.apache.thrift.TException
     {
-      updateCompactionState_args args = new updateCompactionState_args();
+      updateCompactionStatus_args args = new updateCompactionStatus_args();
       args.setCompaction(compaction);
       args.setState(state);
       args.setMessage(message);
       args.setTimestamp(timestamp);
-      sendBase("updateCompactionState", args);
+      sendBase("updateCompactionStatus", args);
     }
 
-    public void recv_updateCompactionState() throws org.apache.thrift.TException
+    public void recv_updateCompactionStatus() throws org.apache.thrift.TException
     {
-      updateCompactionState_result result = new updateCompactionState_result();
-      receiveBase(result, "updateCompactionState");
+      updateCompactionStatus_result result = new updateCompactionStatus_result();
+      receiveBase(result, "updateCompactionStatus");
       return;
     }
 
@@ -213,33 +208,30 @@ public class CompactionCoordinator {
       super(protocolFactory, clientManager, transport);
     }
 
-    public void AddOrUpdateCompaction(org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent, java.lang.String queueName, long priority, Action action, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
+    public void cancelCompaction(org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent, java.lang.String queueName, long priority, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
       checkReady();
-      AddOrUpdateCompaction_call method_call = new AddOrUpdateCompaction_call(extent, queueName, priority, action, resultHandler, this, ___protocolFactory, ___transport);
+      cancelCompaction_call method_call = new cancelCompaction_call(extent, queueName, priority, resultHandler, this, ___protocolFactory, ___transport);
       this.___currentMethod = method_call;
       ___manager.call(method_call);
     }
 
-    public static class AddOrUpdateCompaction_call extends org.apache.thrift.async.TAsyncMethodCall<Void> {
+    public static class cancelCompaction_call extends org.apache.thrift.async.TAsyncMethodCall<Void> {
       private org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent;
       private java.lang.String queueName;
       private long priority;
-      private Action action;
-      public AddOrUpdateCompaction_call(org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent, java.lang.String queueName, long priority, Action action, 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 {
+      public cancelCompaction_call(org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent, java.lang.String queueName, long priority, 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.extent = extent;
         this.queueName = queueName;
         this.priority = priority;
-        this.action = action;
       }
 
       public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
-        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("AddOrUpdateCompaction", org.apache.thrift.protocol.TMessageType.CALL, 0));
-        AddOrUpdateCompaction_args args = new AddOrUpdateCompaction_args();
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("cancelCompaction", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        cancelCompaction_args args = new cancelCompaction_args();
         args.setExtent(extent);
         args.setQueueName(queueName);
         args.setPriority(priority);
-        args.setAction(action);
         args.write(prot);
         prot.writeMessageEnd();
       }
@@ -254,35 +246,35 @@ public class CompactionCoordinator {
       }
     }
 
-    public void getCompactionStatus(java.lang.String tableId, java.lang.String endRow, java.lang.String queueName, org.apache.thrift.async.AsyncMethodCallback<Status> resultHandler) throws org.apache.thrift.TException {
+    public void getCompactionStatus(org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent, java.lang.String queueName, long priority, org.apache.thrift.async.AsyncMethodCallback<java.util.List<Status>> resultHandler) throws org.apache.thrift.TException {
       checkReady();
-      getCompactionStatus_call method_call = new getCompactionStatus_call(tableId, endRow, queueName, resultHandler, this, ___protocolFactory, ___transport);
+      getCompactionStatus_call method_call = new getCompactionStatus_call(extent, queueName, priority, resultHandler, this, ___protocolFactory, ___transport);
       this.___currentMethod = method_call;
       ___manager.call(method_call);
     }
 
-    public static class getCompactionStatus_call extends org.apache.thrift.async.TAsyncMethodCall<Status> {
-      private java.lang.String tableId;
-      private java.lang.String endRow;
+    public static class getCompactionStatus_call extends org.apache.thrift.async.TAsyncMethodCall<java.util.List<Status>> {
+      private org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent;
       private java.lang.String queueName;
-      public getCompactionStatus_call(java.lang.String tableId, java.lang.String endRow, java.lang.String queueName, org.apache.thrift.async.AsyncMethodCallback<Status> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+      private long priority;
+      public getCompactionStatus_call(org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent, java.lang.String queueName, long priority, org.apache.thrift.async.AsyncMethodCallback<java.util.List<Status>> 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.tableId = tableId;
-        this.endRow = endRow;
+        this.extent = extent;
         this.queueName = queueName;
+        this.priority = priority;
       }
 
       public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
         prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("getCompactionStatus", org.apache.thrift.protocol.TMessageType.CALL, 0));
         getCompactionStatus_args args = new getCompactionStatus_args();
-        args.setTableId(tableId);
-        args.setEndRow(endRow);
+        args.setExtent(extent);
         args.setQueueName(queueName);
+        args.setPriority(priority);
         args.write(prot);
         prot.writeMessageEnd();
       }
 
-      public Status getResult() throws org.apache.thrift.TException {
+      public java.util.List<Status> 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!");
         }
@@ -292,92 +284,89 @@ public class CompactionCoordinator {
       }
     }
 
-    public void getCompactionStatusHistory(java.lang.String tableId, java.lang.String endRow, java.lang.String queueName, org.apache.thrift.async.AsyncMethodCallback<java.util.List<Status>> resultHandler) throws org.apache.thrift.TException {
+    public void getCompactionJob(java.lang.String queueName, java.lang.String compactor, org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.tabletserver.thrift.CompactionJob> resultHandler) throws org.apache.thrift.TException {
       checkReady();
-      getCompactionStatusHistory_call method_call = new getCompactionStatusHistory_call(tableId, endRow, queueName, resultHandler, this, ___protocolFactory, ___transport);
+      getCompactionJob_call method_call = new getCompactionJob_call(queueName, compactor, resultHandler, this, ___protocolFactory, ___transport);
       this.___currentMethod = method_call;
       ___manager.call(method_call);
     }
 
-    public static class getCompactionStatusHistory_call extends org.apache.thrift.async.TAsyncMethodCall<java.util.List<Status>> {
-      private java.lang.String tableId;
-      private java.lang.String endRow;
+    public static class getCompactionJob_call extends org.apache.thrift.async.TAsyncMethodCall<org.apache.accumulo.core.tabletserver.thrift.CompactionJob> {
       private java.lang.String queueName;
-      public getCompactionStatusHistory_call(java.lang.String tableId, java.lang.String endRow, java.lang.String queueName, org.apache.thrift.async.AsyncMethodCallback<java.util.List<Status>> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+      private java.lang.String compactor;
+      public getCompactionJob_call(java.lang.String queueName, java.lang.String compactor, org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.tabletserver.thrift.CompactionJob> 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.tableId = tableId;
-        this.endRow = endRow;
         this.queueName = queueName;
+        this.compactor = compactor;
       }
 
       public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
-        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("getCompactionStatusHistory", org.apache.thrift.protocol.TMessageType.CALL, 0));
-        getCompactionStatusHistory_args args = new getCompactionStatusHistory_args();
-        args.setTableId(tableId);
-        args.setEndRow(endRow);
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("getCompactionJob", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        getCompactionJob_args args = new getCompactionJob_args();
         args.setQueueName(queueName);
+        args.setCompactor(compactor);
         args.write(prot);
         prot.writeMessageEnd();
       }
 
-      public java.util.List<Status> getResult() throws org.apache.thrift.TException {
+      public org.apache.accumulo.core.tabletserver.thrift.CompactionJob 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_getCompactionStatusHistory();
+        return (new Client(prot)).recv_getCompactionJob();
       }
     }
 
-    public void getCompactionJob(java.lang.String queueName, long compactorHostAndPort, org.apache.thrift.async.AsyncMethodCallback<CompactionJob> resultHandler) throws org.apache.thrift.TException {
+    public void compactionCompleted(org.apache.accumulo.core.tabletserver.thrift.CompactionJob job, org.apache.accumulo.core.dataImpl.thrift.CompactionStats stats, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
       checkReady();
-      getCompactionJob_call method_call = new getCompactionJob_call(queueName, compactorHostAndPort, resultHandler, this, ___protocolFactory, ___transport);
+      compactionCompleted_call method_call = new compactionCompleted_call(job, stats, resultHandler, this, ___protocolFactory, ___transport);
       this.___currentMethod = method_call;
       ___manager.call(method_call);
     }
 
-    public static class getCompactionJob_call extends org.apache.thrift.async.TAsyncMethodCall<CompactionJob> {
-      private java.lang.String queueName;
-      private long compactorHostAndPort;
-      public getCompactionJob_call(java.lang.String queueName, long compactorHostAndPort, org.apache.thrift.async.AsyncMethodCallback<CompactionJob> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+    public static class compactionCompleted_call extends org.apache.thrift.async.TAsyncMethodCall<Void> {
+      private org.apache.accumulo.core.tabletserver.thrift.CompactionJob job;
+      private org.apache.accumulo.core.dataImpl.thrift.CompactionStats stats;
+      public compactionCompleted_call(org.apache.accumulo.core.tabletserver.thrift.CompactionJob job, org.apache.accumulo.core.dataImpl.thrift.CompactionStats stats, 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.queueName = queueName;
-        this.compactorHostAndPort = compactorHostAndPort;
+        this.job = job;
+        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("getCompactionJob", org.apache.thrift.protocol.TMessageType.CALL, 0));
-        getCompactionJob_args args = new getCompactionJob_args();
-        args.setQueueName(queueName);
-        args.setCompactorHostAndPort(compactorHostAndPort);
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("compactionCompleted", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        compactionCompleted_args args = new compactionCompleted_args();
+        args.setJob(job);
+        args.setStats(stats);
         args.write(prot);
         prot.writeMessageEnd();
       }
 
-      public CompactionJob getResult() throws org.apache.thrift.TException {
+      public Void 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();
+        return null;
       }
     }
 
-    public void updateCompactionState(CompactionJob compaction, CompactionState state, java.lang.String message, long timestamp, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
+    public void updateCompactionStatus(org.apache.accumulo.core.tabletserver.thrift.CompactionJob compaction, CompactionState state, java.lang.String message, long timestamp, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
       checkReady();
-      updateCompactionState_call method_call = new updateCompactionState_call(compaction, state, message, timestamp, resultHandler, this, ___protocolFactory, ___transport);
+      updateCompactionStatus_call method_call = new updateCompactionStatus_call(compaction, state, message, timestamp, resultHandler, this, ___protocolFactory, ___transport);
       this.___currentMethod = method_call;
       ___manager.call(method_call);
     }
 
-    public static class updateCompactionState_call extends org.apache.thrift.async.TAsyncMethodCall<Void> {
-      private CompactionJob compaction;
+    public static class updateCompactionStatus_call extends org.apache.thrift.async.TAsyncMethodCall<Void> {
+      private org.apache.accumulo.core.tabletserver.thrift.CompactionJob compaction;
       private CompactionState state;
       private java.lang.String message;
       private long timestamp;
-      public updateCompactionState_call(CompactionJob compaction, CompactionState 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 transport) throws org.apache.thrift.TException {
+      public updateCompactionStatus_call(org.apache.accumulo.core.tabletserver.thrift.CompactionJob compaction, CompactionState 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 transport) throws org.apache.thrift.TException {
         super(client, protocolFactory, transport, resultHandler, false);
         this.compaction = compaction;
         this.state = state;
@@ -386,8 +375,8 @@ public class CompactionCoordinator {
       }
 
       public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
-        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("updateCompactionState", org.apache.thrift.protocol.TMessageType.CALL, 0));
-        updateCompactionState_args args = new updateCompactionState_args();
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("updateCompactionStatus", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        updateCompactionStatus_args args = new updateCompactionStatus_args();
         args.setCompaction(compaction);
         args.setState(state);
         args.setMessage(message);
@@ -419,21 +408,21 @@ public class CompactionCoordinator {
     }
 
     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("AddOrUpdateCompaction", new AddOrUpdateCompaction());
+      processMap.put("cancelCompaction", new cancelCompaction());
       processMap.put("getCompactionStatus", new getCompactionStatus());
-      processMap.put("getCompactionStatusHistory", new getCompactionStatusHistory());
       processMap.put("getCompactionJob", new getCompactionJob());
-      processMap.put("updateCompactionState", new updateCompactionState());
+      processMap.put("compactionCompleted", new compactionCompleted());
+      processMap.put("updateCompactionStatus", new updateCompactionStatus());
       return processMap;
     }
 
-    public static class AddOrUpdateCompaction<I extends Iface> extends org.apache.thrift.ProcessFunction<I, AddOrUpdateCompaction_args> {
-      public AddOrUpdateCompaction() {
-        super("AddOrUpdateCompaction");
+    public static class cancelCompaction<I extends Iface> extends org.apache.thrift.ProcessFunction<I, cancelCompaction_args> {
+      public cancelCompaction() {
+        super("cancelCompaction");
       }
 
-      public AddOrUpdateCompaction_args getEmptyArgsInstance() {
-        return new AddOrUpdateCompaction_args();
+      public cancelCompaction_args getEmptyArgsInstance() {
+        return new cancelCompaction_args();
       }
 
       protected boolean isOneway() {
@@ -445,9 +434,9 @@ public class CompactionCoordinator {
         return false;
       }
 
-      public AddOrUpdateCompaction_result getResult(I iface, AddOrUpdateCompaction_args args) throws org.apache.thrift.TException {
-        AddOrUpdateCompaction_result result = new AddOrUpdateCompaction_result();
-        iface.AddOrUpdateCompaction(args.extent, args.queueName, args.priority, args.action);
+      public cancelCompaction_result getResult(I iface, cancelCompaction_args args) throws org.apache.thrift.TException {
+        cancelCompaction_result result = new cancelCompaction_result();
+        iface.cancelCompaction(args.extent, args.queueName, args.priority);
         return result;
       }
     }
@@ -472,18 +461,18 @@ public class CompactionCoordinator {
 
       public getCompactionStatus_result getResult(I iface, getCompactionStatus_args args) throws org.apache.thrift.TException {
         getCompactionStatus_result result = new getCompactionStatus_result();
-        result.success = iface.getCompactionStatus(args.tableId, args.endRow, args.queueName);
+        result.success = iface.getCompactionStatus(args.extent, args.queueName, args.priority);
         return result;
       }
     }
 
-    public static class getCompactionStatusHistory<I extends Iface> extends org.apache.thrift.ProcessFunction<I, getCompactionStatusHistory_args> {
-      public getCompactionStatusHistory() {
-        super("getCompactionStatusHistory");
+    public static class getCompactionJob<I extends Iface> extends org.apache.thrift.ProcessFunction<I, getCompactionJob_args> {
+      public getCompactionJob() {
+        super("getCompactionJob");
       }
 
-      public getCompactionStatusHistory_args getEmptyArgsInstance() {
-        return new getCompactionStatusHistory_args();
+      public getCompactionJob_args getEmptyArgsInstance() {
+        return new getCompactionJob_args();
       }
 
       protected boolean isOneway() {
@@ -495,20 +484,20 @@ public class CompactionCoordinator {
         return false;
       }
 
-      public getCompactionStatusHistory_result getResult(I iface, getCompactionStatusHistory_args args) throws org.apache.thrift.TException {
-        getCompactionStatusHistory_result result = new getCompactionStatusHistory_result();
-        result.success = iface.getCompactionStatusHistory(args.tableId, args.endRow, args.queueName);
+      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.queueName, args.compactor);
         return result;
       }
     }
 
-    public static class getCompactionJob<I extends Iface> extends org.apache.thrift.ProcessFunction<I, getCompactionJob_args> {
-      public getCompactionJob() {
-        super("getCompactionJob");
+    public static class compactionCompleted<I extends Iface> extends org.apache.thrift.ProcessFunction<I, compactionCompleted_args> {
+      public compactionCompleted() {
+        super("compactionCompleted");
       }
 
-      public getCompactionJob_args getEmptyArgsInstance() {
-        return new getCompactionJob_args();
+      public compactionCompleted_args getEmptyArgsInstance() {
+        return new compactionCompleted_args();
       }
 
       protected boolean isOneway() {
@@ -520,20 +509,20 @@ public class CompactionCoordinator {
         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.queueName, args.compactorHostAndPort);
+      public compactionCompleted_result getResult(I iface, compactionCompleted_args args) throws org.apache.thrift.TException {
+        compactionCompleted_result result = new compactionCompleted_result();
+        iface.compactionCompleted(args.job, args.stats);
         return result;
       }
     }
 
-    public static class updateCompactionState<I extends Iface> extends org.apache.thrift.ProcessFunction<I, updateCompactionState_args> {
-      public updateCompactionState() {
-        super("updateCompactionState");
+    public static class updateCompactionStatus<I extends Iface> extends org.apache.thrift.ProcessFunction<I, updateCompactionStatus_args> {
+      public updateCompactionStatus() {
+        super("updateCompactionStatus");
       }
 
-      public updateCompactionState_args getEmptyArgsInstance() {
-        return new updateCompactionState_args();
+      public updateCompactionStatus_args getEmptyArgsInstance() {
+        return new updateCompactionStatus_args();
       }
 
       protected boolean isOneway() {
@@ -545,9 +534,9 @@ public class CompactionCoordinator {
         return false;
       }
 
-      public updateCompactionState_result getResult(I iface, updateCompactionState_args args) throws org.apache.thrift.TException {
-        updateCompactionState_result result = new updateCompactionState_result();
-        iface.updateCompactionState(args.compaction, args.state, args.message, args.timestamp);
+      public updateCompactionStatus_result getResult(I iface, updateCompactionStatus_args args) throws org.apache.thrift.TException {
+        updateCompactionStatus_result result = new updateCompactionStatus_result();
+        iface.updateCompactionStatus(args.compaction, args.state, args.message, args.timestamp);
         return result;
       }
     }
@@ -565,28 +554,28 @@ public class CompactionCoordinator {
     }
 
     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("AddOrUpdateCompaction", new AddOrUpdateCompaction());
+      processMap.put("cancelCompaction", new cancelCompaction());
       processMap.put("getCompactionStatus", new getCompactionStatus());
-      processMap.put("getCompactionStatusHistory", new getCompactionStatusHistory());
       processMap.put("getCompactionJob", new getCompactionJob());
-      processMap.put("updateCompactionState", new updateCompactionState());
+      processMap.put("compactionCompleted", new compactionCompleted());
+      processMap.put("updateCompactionStatus", new updateCompactionStatus());
       return processMap;
     }
 
-    public static class AddOrUpdateCompaction<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, AddOrUpdateCompaction_args, Void> {
-      public AddOrUpdateCompaction() {
-        super("AddOrUpdateCompaction");
+    public static class cancelCompaction<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, cancelCompaction_args, Void> {
+      public cancelCompaction() {
+        super("cancelCompaction");
       }
 
-      public AddOrUpdateCompaction_args getEmptyArgsInstance() {
-        return new AddOrUpdateCompaction_args();
+      public cancelCompaction_args getEmptyArgsInstance() {
+        return new cancelCompaction_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) {
-            AddOrUpdateCompaction_result result = new AddOrUpdateCompaction_result();
+            cancelCompaction_result result = new cancelCompaction_result();
             try {
               fcall.sendResponse(fb, result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
             } catch (org.apache.thrift.transport.TTransportException e) {
@@ -600,7 +589,7 @@ public class CompactionCoordinator {
           public void onError(java.lang.Exception e) {
             byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
             org.apache.thrift.TSerializable msg;
-            AddOrUpdateCompaction_result result = new AddOrUpdateCompaction_result();
+            cancelCompaction_result result = new cancelCompaction_result();
             if (e instanceof org.apache.thrift.transport.TTransportException) {
               _LOGGER.error("TTransportException inside handler", e);
               fb.close();
@@ -628,12 +617,12 @@ public class CompactionCoordinator {
         return false;
       }
 
-      public void start(I iface, AddOrUpdateCompaction_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
-        iface.AddOrUpdateCompaction(args.extent, args.queueName, args.priority, args.action,resultHandler);
+      public void start(I iface, cancelCompaction_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
+        iface.cancelCompaction(args.extent, args.queueName, args.priority,resultHandler);
       }
     }
 
-    public static class getCompactionStatus<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, getCompactionStatus_args, Status> {
+    public static class getCompactionStatus<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, getCompactionStatus_args, java.util.List<Status>> {
       public getCompactionStatus() {
         super("getCompactionStatus");
       }
@@ -642,10 +631,10 @@ public class CompactionCoordinator {
         return new getCompactionStatus_args();
       }
 
-      public org.apache.thrift.async.AsyncMethodCallback<Status> getResultHandler(final org.apache.thrift.server.AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) {
+      public org.apache.thrift.async.AsyncMethodCallback<java.util.List<Status>> 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<Status>() { 
-          public void onComplete(Status o) {
+        return new org.apache.thrift.async.AsyncMethodCallback<java.util.List<Status>>() { 
+          public void onComplete(java.util.List<Status> o) {
             getCompactionStatus_result result = new getCompactionStatus_result();
             result.success = o;
             try {
@@ -689,25 +678,25 @@ public class CompactionCoordinator {
         return false;
       }
 
-      public void start(I iface, getCompactionStatus_args args, org.apache.thrift.async.AsyncMethodCallback<Status> resultHandler) throws org.apache.thrift.TException {
-        iface.getCompactionStatus(args.tableId, args.endRow, args.queueName,resultHandler);
+      public void start(I iface, getCompactionStatus_args args, org.apache.thrift.async.AsyncMethodCallback<java.util.List<Status>> resultHandler) throws org.apache.thrift.TException {
+        iface.getCompactionStatus(args.extent, args.queueName, args.priority,resultHandler);
       }
     }
 
-    public static class getCompactionStatusHistory<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, getCompactionStatusHistory_args, java.util.List<Status>> {
-      public getCompactionStatusHistory() {
-        super("getCompactionStatusHistory");
+    public static class getCompactionJob<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, getCompactionJob_args, org.apache.accumulo.core.tabletserver.thrift.CompactionJob> {
+      public getCompactionJob() {
+        super("getCompactionJob");
       }
 
-      public getCompactionStatusHistory_args getEmptyArgsInstance() {
-        return new getCompactionStatusHistory_args();
+      public getCompactionJob_args getEmptyArgsInstance() {
+        return new getCompactionJob_args();
       }
 
-      public org.apache.thrift.async.AsyncMethodCallback<java.util.List<Status>> getResultHandler(final org.apache.thrift.server.AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) {
+      public org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.tabletserver.thrift.CompactionJob> 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<Status>>() { 
-          public void onComplete(java.util.List<Status> o) {
-            getCompactionStatusHistory_result result = new getCompactionStatusHistory_result();
+        return new org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.tabletserver.thrift.CompactionJob>() { 
+          public void onComplete(org.apache.accumulo.core.tabletserver.thrift.CompactionJob o) {
+            getCompactionJob_result result = new getCompactionJob_result();
             result.success = o;
             try {
               fcall.sendResponse(fb, result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
@@ -722,7 +711,7 @@ public class CompactionCoordinator {
           public void onError(java.lang.Exception e) {
             byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
             org.apache.thrift.TSerializable msg;
-            getCompactionStatusHistory_result result = new getCompactionStatusHistory_result();
+            getCompactionJob_result result = new getCompactionJob_result();
             if (e instanceof org.apache.thrift.transport.TTransportException) {
               _LOGGER.error("TTransportException inside handler", e);
               fb.close();
@@ -750,26 +739,25 @@ public class CompactionCoordinator {
         return false;
       }
 
-      public void start(I iface, getCompactionStatusHistory_args args, org.apache.thrift.async.AsyncMethodCallback<java.util.List<Status>> resultHandler) throws org.apache.thrift.TException {
-        iface.getCompactionStatusHistory(args.tableId, args.endRow, args.queueName,resultHandler);
+      public void start(I iface, getCompactionJob_args args, org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.tabletserver.thrift.CompactionJob> resultHandler) throws org.apache.thrift.TException {
+        iface.getCompactionJob(args.queueName, args.compactor,resultHandler);
       }
     }
 
-    public static class getCompactionJob<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, getCompactionJob_args, CompactionJob> {
-      public getCompactionJob() {
-        super("getCompactionJob");
+    public static class compactionCompleted<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, compactionCompleted_args, Void> {
+      public compactionCompleted() {
+        super("compactionCompleted");
       }
 
-      public getCompactionJob_args getEmptyArgsInstance() {
-        return new getCompactionJob_args();
+      public compactionCompleted_args getEmptyArgsInstance() {
+        return new compactionCompleted_args();
       }
 
-      public org.apache.thrift.async.AsyncMethodCallback<CompactionJob> getResultHandler(final org.apache.thrift.server.AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) {
+      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<CompactionJob>() { 
-          public void onComplete(CompactionJob o) {
-            getCompactionJob_result result = new getCompactionJob_result();
-            result.success = o;
+        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) {
@@ -783,7 +771,7 @@ public class CompactionCoordinator {
           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();
+            compactionCompleted_result result = new compactionCompleted_result();
             if (e instanceof org.apache.thrift.transport.TTransportException) {
               _LOGGER.error("TTransportException inside handler", e);
               fb.close();
@@ -811,25 +799,25 @@ public class CompactionCoordinator {
         return false;
       }
 
-      public void start(I iface, getCompactionJob_args args, org.apache.thrift.async.AsyncMethodCallback<CompactionJob> resultHandler) throws org.apache.thrift.TException {
-        iface.getCompactionJob(args.queueName, args.compactorHostAndPort,resultHandler);
+      public void start(I iface, compactionCompleted_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
+        iface.compactionCompleted(args.job, args.stats,resultHandler);
       }
     }
 
-    public static class updateCompactionState<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, updateCompactionState_args, Void> {
-      public updateCompactionState() {
-        super("updateCompactionState");
+    public static class updateCompactionStatus<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, updateCompactionStatus_args, Void> {
+      public updateCompactionStatus() {
+        super("updateCompactionStatus");
       }
 
-      public updateCompactionState_args getEmptyArgsInstance() {
-        return new updateCompactionState_args();
+      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) {
-            updateCompactionState_result result = new updateCompactionState_result();
+            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) {
@@ -843,7 +831,7 @@ public class CompactionCoordinator {
           public void onError(java.lang.Exception e) {
             byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
             org.apache.thrift.TSerializable msg;
-            updateCompactionState_result result = new updateCompactionState_result();
+            updateCompactionStatus_result result = new updateCompactionStatus_result();
             if (e instanceof org.apache.thrift.transport.TTransportException) {
               _LOGGER.error("TTransportException inside handler", e);
               fb.close();
@@ -871,43 +859,32 @@ public class CompactionCoordinator {
         return false;
       }
 
-      public void start(I iface, updateCompactionState_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
-        iface.updateCompactionState(args.compaction, args.state, args.message, args.timestamp,resultHandler);
+      public void start(I iface, updateCompactionStatus_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
+        iface.updateCompactionStatus(args.compaction, args.state, args.message, args.timestamp,resultHandler);
       }
     }
 
   }
 
-  public static class AddOrUpdateCompaction_args implements org.apache.thrift.TBase<AddOrUpdateCompaction_args, AddOrUpdateCompaction_args._Fields>, java.io.Serializable, Cloneable, Comparable<AddOrUpdateCompaction_args>   {
-    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("AddOrUpdateCompaction_args");
+  public static class cancelCompaction_args implements org.apache.thrift.TBase<cancelCompaction_args, cancelCompaction_args._Fields>, java.io.Serializable, Cloneable, Comparable<cancelCompaction_args>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("cancelCompaction_args");
 
     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)1);
     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)2);
     private static final org.apache.thrift.protocol.TField PRIORITY_FIELD_DESC = new org.apache.thrift.protocol.TField("priority", org.apache.thrift.protocol.TType.I64, (short)3);
-    private static final org.apache.thrift.protocol.TField ACTION_FIELD_DESC = new org.apache.thrift.protocol.TField("action", org.apache.thrift.protocol.TType.I32, (short)4);
 
-    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new AddOrUpdateCompaction_argsStandardSchemeFactory();
-    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new AddOrUpdateCompaction_argsTupleSchemeFactory();
+    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new cancelCompaction_argsStandardSchemeFactory();
+    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new cancelCompaction_argsTupleSchemeFactory();
 
     public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent; // required
     public @org.apache.thrift.annotation.Nullable java.lang.String queueName; // required
     public long priority; // required
-    /**
-     * 
-     * @see Action
-     */
-    public @org.apache.thrift.annotation.Nullable Action action; // 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 {
       EXTENT((short)1, "extent"),
       QUEUE_NAME((short)2, "queueName"),
-      PRIORITY((short)3, "priority"),
-      /**
-       * 
-       * @see Action
-       */
-      ACTION((short)4, "action");
+      PRIORITY((short)3, "priority");
 
       private static final java.util.Map<java.lang.String, _Fields> byName = new java.util.HashMap<java.lang.String, _Fields>();
 
@@ -929,8 +906,6 @@ public class CompactionCoordinator {
             return QUEUE_NAME;
           case 3: // PRIORITY
             return PRIORITY;
-          case 4: // ACTION
-            return ACTION;
           default:
             return null;
         }
@@ -983,33 +958,29 @@ public class CompactionCoordinator {
           new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
       tmpMap.put(_Fields.PRIORITY, new org.apache.thrift.meta_data.FieldMetaData("priority", org.apache.thrift.TFieldRequirementType.DEFAULT, 
           new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64)));
-      tmpMap.put(_Fields.ACTION, new org.apache.thrift.meta_data.FieldMetaData("action", org.apache.thrift.TFieldRequirementType.DEFAULT, 
-          new org.apache.thrift.meta_data.EnumMetaData(org.apache.thrift.protocol.TType.ENUM, Action.class)));
       metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
-      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(AddOrUpdateCompaction_args.class, metaDataMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(cancelCompaction_args.class, metaDataMap);
     }
 
-    public AddOrUpdateCompaction_args() {
+    public cancelCompaction_args() {
     }
 
-    public AddOrUpdateCompaction_args(
+    public cancelCompaction_args(
       org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent,
       java.lang.String queueName,
-      long priority,
-      Action action)
+      long priority)
     {
       this();
       this.extent = extent;
       this.queueName = queueName;
       this.priority = priority;
       setPriorityIsSet(true);
-      this.action = action;
     }
 
     /**
      * Performs a deep copy on <i>other</i>.
      */
-    public AddOrUpdateCompaction_args(AddOrUpdateCompaction_args other) {
+    public cancelCompaction_args(cancelCompaction_args other) {
       __isset_bitfield = other.__isset_bitfield;
       if (other.isSetExtent()) {
         this.extent = new org.apache.accumulo.core.dataImpl.thrift.TKeyExtent(other.extent);
@@ -1018,13 +989,10 @@ public class CompactionCoordinator {
         this.queueName = other.queueName;
       }
       this.priority = other.priority;
-      if (other.isSetAction()) {
-        this.action = other.action;
-      }
     }
 
-    public AddOrUpdateCompaction_args deepCopy() {
-      return new AddOrUpdateCompaction_args(this);
+    public cancelCompaction_args deepCopy() {
+      return new cancelCompaction_args(this);
     }
 
     @Override
@@ -1033,7 +1001,6 @@ public class CompactionCoordinator {
       this.queueName = null;
       setPriorityIsSet(false);
       this.priority = 0;
-      this.action = null;
     }
 
     @org.apache.thrift.annotation.Nullable
@@ -1041,7 +1008,7 @@ public class CompactionCoordinator {
       return this.extent;
     }
 
-    public AddOrUpdateCompaction_args setExtent(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent) {
+    public cancelCompaction_args setExtent(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent) {
       this.extent = extent;
       return this;
     }
@@ -1066,7 +1033,7 @@ public class CompactionCoordinator {
       return this.queueName;
     }
 
-    public AddOrUpdateCompaction_args setQueueName(@org.apache.thrift.annotation.Nullable java.lang.String queueName) {
+    public cancelCompaction_args setQueueName(@org.apache.thrift.annotation.Nullable java.lang.String queueName) {
       this.queueName = queueName;
       return this;
     }
@@ -1090,7 +1057,7 @@ public class CompactionCoordinator {
       return this.priority;
     }
 
-    public AddOrUpdateCompaction_args setPriority(long priority) {
+    public cancelCompaction_args setPriority(long priority) {
       this.priority = priority;
       setPriorityIsSet(true);
       return this;
@@ -1109,39 +1076,6 @@ public class CompactionCoordinator {
       __isset_bitfield = org.apache.thrift.EncodingUtils.setBit(__isset_bitfield, __PRIORITY_ISSET_ID, value);
     }
 
-    /**
-     * 
-     * @see Action
-     */
-    @org.apache.thrift.annotation.Nullable
-    public Action getAction() {
-      return this.action;
-    }
-
-    /**
-     * 
-     * @see Action
-     */
-    public AddOrUpdateCompaction_args setAction(@org.apache.thrift.annotation.Nullable Action action) {
-      this.action = action;
-      return this;
-    }
-
-    public void unsetAction() {
-      this.action = null;
-    }
-
-    /** Returns true if field action is set (has been assigned a value) and false otherwise */
-    public boolean isSetAction() {
-      return this.action != null;
-    }
-
-    public void setActionIsSet(boolean value) {
-      if (!value) {
-        this.action = null;
-      }
-    }
-
     public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable java.lang.Object value) {
       switch (field) {
       case EXTENT:
@@ -1168,14 +1102,6 @@ public class CompactionCoordinator {
         }
         break;
 
-      case ACTION:
-        if (value == null) {
-          unsetAction();
-        } else {
-          setAction((Action)value);
-        }
-        break;
-
       }
     }
 
@@ -1191,9 +1117,6 @@ public class CompactionCoordinator {
       case PRIORITY:
         return getPriority();
 
-      case ACTION:
-        return getAction();
-
       }
       throw new java.lang.IllegalStateException();
     }
@@ -1211,8 +1134,6 @@ public class CompactionCoordinator {
         return isSetQueueName();
       case PRIORITY:
         return isSetPriority();
-      case ACTION:
-        return isSetAction();
       }
       throw new java.lang.IllegalStateException();
     }
@@ -1221,12 +1142,12 @@ public class CompactionCoordinator {
     public boolean equals(java.lang.Object that) {
       if (that == null)
         return false;
-      if (that instanceof AddOrUpdateCompaction_args)
-        return this.equals((AddOrUpdateCompaction_args)that);
+      if (that instanceof cancelCompaction_args)
+        return this.equals((cancelCompaction_args)that);
       return false;
     }
 
-    public boolean equals(AddOrUpdateCompaction_args that) {
+    public boolean equals(cancelCompaction_args that) {
       if (that == null)
         return false;
       if (this == that)
@@ -1259,15 +1180,6 @@ public class CompactionCoordinator {
           return false;
       }
 
-      boolean this_present_action = true && this.isSetAction();
-      boolean that_present_action = true && that.isSetAction();
-      if (this_present_action || that_present_action) {
-        if (!(this_present_action && that_present_action))
-          return false;
-        if (!this.action.equals(that.action))
-          return false;
-      }
-
       return true;
     }
 
@@ -1285,15 +1197,11 @@ public class CompactionCoordinator {
 
       hashCode = hashCode * 8191 + org.apache.thrift.TBaseHelper.hashCode(priority);
 
-      hashCode = hashCode * 8191 + ((isSetAction()) ? 131071 : 524287);
-      if (isSetAction())
-        hashCode = hashCode * 8191 + action.getValue();
-
       return hashCode;
     }
 
     @Override
-    public int compareTo(AddOrUpdateCompaction_args other) {
+    public int compareTo(cancelCompaction_args other) {
       if (!getClass().equals(other.getClass())) {
         return getClass().getName().compareTo(other.getClass().getName());
       }
@@ -1330,16 +1238,6 @@ public class CompactionCoordinator {
           return lastComparison;
         }
       }
-      lastComparison = java.lang.Boolean.valueOf(isSetAction()).compareTo(other.isSetAction());
-      if (lastComparison != 0) {
-        return lastComparison;
-      }
-      if (isSetAction()) {
-        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.action, other.action);
-        if (lastComparison != 0) {
-          return lastComparison;
-        }
-      }
       return 0;
     }
 
@@ -1358,7 +1256,7 @@ public class CompactionCoordinator {
 
     @Override
     public java.lang.String toString() {
-      java.lang.StringBuilder sb = new java.lang.StringBuilder("AddOrUpdateCompaction_args(");
+      java.lang.StringBuilder sb = new java.lang.StringBuilder("cancelCompaction_args(");
       boolean first = true;
 
       sb.append("extent:");
@@ -1380,14 +1278,6 @@ public class CompactionCoordinator {
       sb.append("priority:");
       sb.append(this.priority);
       first = false;
-      if (!first) sb.append(", ");
-      sb.append("action:");
-      if (this.action == null) {
-        sb.append("null");
-      } else {
-        sb.append(this.action);
-      }
-      first = false;
       sb.append(")");
       return sb.toString();
     }
@@ -1418,15 +1308,15 @@ public class CompactionCoordinator {
       }
     }
 
-    private static class AddOrUpdateCompaction_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      public AddOrUpdateCompaction_argsStandardScheme getScheme() {
-        return new AddOrUpdateCompaction_argsStandardScheme();
+    private static class cancelCompaction_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public cancelCompaction_argsStandardScheme getScheme() {
+        return new cancelCompaction_argsStandardScheme();
       }
     }
 
-    private static class AddOrUpdateCompaction_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme<AddOrUpdateCompaction_args> {
+    private static class cancelCompaction_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme<cancelCompaction_args> {
 
-      public void read(org.apache.thrift.protocol.TProtocol iprot, AddOrUpdateCompaction_args struct) throws org.apache.thrift.TException {
+      public void read(org.apache.thrift.protocol.TProtocol iprot, cancelCompaction_args struct) throws org.apache.thrift.TException {
         org.apache.thrift.protocol.TField schemeField;
         iprot.readStructBegin();
         while (true)
@@ -1461,14 +1351,6 @@ public class CompactionCoordinator {
                 org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
               }
               break;
-            case 4: // ACTION
-              if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
-                struct.action = org.apache.accumulo.core.compaction.thrift.Action.findByValue(iprot.readI32());
-                struct.setActionIsSet(true);
-              } else { 
-                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
-              }
-              break;
             default:
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
           }
@@ -1480,7 +1362,7 @@ public class CompactionCoordinator {
         struct.validate();
       }
 
-      public void write(org.apache.thrift.protocol.TProtocol oprot, AddOrUpdateCompaction_args struct) throws org.apache.thrift.TException {
+      public void write(org.apache.thrift.protocol.TProtocol oprot, cancelCompaction_args struct) throws org.apache.thrift.TException {
         struct.validate();
 
         oprot.writeStructBegin(STRUCT_DESC);
@@ -1497,27 +1379,22 @@ public class CompactionCoordinator {
         oprot.writeFieldBegin(PRIORITY_FIELD_DESC);
         oprot.writeI64(struct.priority);
         oprot.writeFieldEnd();
-        if (struct.action != null) {
-          oprot.writeFieldBegin(ACTION_FIELD_DESC);
-          oprot.writeI32(struct.action.getValue());
-          oprot.writeFieldEnd();
-        }
         oprot.writeFieldStop();
         oprot.writeStructEnd();
       }
 
     }
 
-    private static class AddOrUpdateCompaction_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      public AddOrUpdateCompaction_argsTupleScheme getScheme() {
-        return new AddOrUpdateCompaction_argsTupleScheme();
+    private static class cancelCompaction_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public cancelCompaction_argsTupleScheme getScheme() {
+        return new cancelCompaction_argsTupleScheme();
       }
     }
 
-    private static class AddOrUpdateCompaction_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme<AddOrUpdateCompaction_args> {
+    private static class cancelCompaction_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme<cancelCompaction_args> {
 
       @Override
-      public void write(org.apache.thrift.protocol.TProtocol prot, AddOrUpdateCompaction_args struct) throws org.apache.thrift.TException {
+      public void write(org.apache.thrift.protocol.TProtocol prot, cancelCompaction_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.isSetExtent()) {
@@ -1529,10 +1406,7 @@ public class CompactionCoordinator {
         if (struct.isSetPriority()) {
           optionals.set(2);
         }
-        if (struct.isSetAction()) {
-          optionals.set(3);
-        }
-        oprot.writeBitSet(optionals, 4);
+        oprot.writeBitSet(optionals, 3);
         if (struct.isSetExtent()) {
           struct.extent.write(oprot);
         }
@@ -1542,15 +1416,12 @@ public class CompactionCoordinator {
         if (struct.isSetPriority()) {
           oprot.writeI64(struct.priority);
         }
-        if (struct.isSetAction()) {
-          oprot.writeI32(struct.action.getValue());
-        }
       }
 
       @Override
-      public void read(org.apache.thrift.protocol.TProtocol prot, AddOrUpdateCompaction_args struct) throws org.apache.thrift.TException {
+      public void read(org.apache.thrift.protocol.TProtocol prot, cancelCompaction_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);
+        java.util.BitSet incoming = iprot.readBitSet(3);
         if (incoming.get(0)) {
           struct.extent = new org.apache.accumulo.core.dataImpl.thrift.TKeyExtent();
           struct.extent.read(iprot);
@@ -1564,10 +1435,6 @@ public class CompactionCoordinator {
           struct.priority = iprot.readI64();
           struct.setPriorityIsSet(true);
         }
-        if (incoming.get(3)) {
-          struct.action = org.apache.accumulo.core.compaction.thrift.Action.findByValue(iprot.readI32());
-          struct.setActionIsSet(true);
-        }
       }
     }
 
@@ -1576,12 +1443,12 @@ public class CompactionCoordinator {
     }
   }
 
-  public static class AddOrUpdateCompaction_result implements org.apache.thrift.TBase<AddOrUpdateCompaction_result, AddOrUpdateCompaction_result._Fields>, java.io.Serializable, Cloneable, Comparable<AddOrUpdateCompaction_result>   {
-    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("AddOrUpdateCompaction_result");
+  public static class cancelCompaction_result implements org.apache.thrift.TBase<cancelCompaction_result, cancelCompaction_result._Fields>, java.io.Serializable, Cloneable, Comparable<cancelCompaction_result>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("cancelCompaction_result");
 
 
-    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new AddOrUpdateCompaction_resultStandardSchemeFactory();
-    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new AddOrUpdateCompaction_resultTupleSchemeFactory();
+    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new cancelCompaction_resultStandardSchemeFactory();
+    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new cancelCompaction_resultTupleSchemeFactory();
 
 
     /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
@@ -1645,20 +1512,20 @@ public class CompactionCoordinator {
     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);
       metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
-      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(AddOrUpdateCompaction_result.class, metaDataMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(cancelCompaction_result.class, metaDataMap);
     }
 
-    public AddOrUpdateCompaction_result() {
+    public cancelCompaction_result() {
     }
 
     /**
      * Performs a deep copy on <i>other</i>.
      */
-    public AddOrUpdateCompaction_result(AddOrUpdateCompaction_result other) {
+    public cancelCompaction_result(cancelCompaction_result other) {
     }
 
-    public AddOrUpdateCompaction_result deepCopy() {
-      return new AddOrUpdateCompaction_result(this);
+    public cancelCompaction_result deepCopy() {
+      return new cancelCompaction_result(this);
     }
 
     @Override
@@ -1692,12 +1559,12 @@ public class CompactionCoordinator {
     public boolean equals(java.lang.Object that) {
       if (that == null)
         return false;
-      if (that instanceof AddOrUpdateCompaction_result)
-        return this.equals((AddOrUpdateCompaction_result)that);
+      if (that instanceof cancelCompaction_result)
+        return this.equals((cancelCompaction_result)that);
       return false;
     }
 
-    public boolean equals(AddOrUpdateCompaction_result that) {
+    public boolean equals(cancelCompaction_result that) {
       if (that == null)
         return false;
       if (this == that)
@@ -1714,7 +1581,7 @@ public class CompactionCoordinator {
     }
 
     @Override
-    public int compareTo(AddOrUpdateCompaction_result other) {
+    public int compareTo(cancelCompaction_result other) {
       if (!getClass().equals(other.getClass())) {
         return getClass().getName().compareTo(other.getClass().getName());
       }
@@ -1739,7 +1606,7 @@ public class CompactionCoordinator {
 
     @Override
     public java.lang.String toString() {
-      java.lang.StringBuilder sb = new java.lang.StringBuilder("AddOrUpdateCompaction_result(");
+      java.lang.StringBuilder sb = new java.lang.StringBuilder("cancelCompaction_result(");
       boolean first = true;
 
       sb.append(")");
@@ -1767,15 +1634,15 @@ public class CompactionCoordinator {
       }
     }
 
-    private static class AddOrUpdateCompaction_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      public AddOrUpdateCompaction_resultStandardScheme getScheme() {
-        return new AddOrUpdateCompaction_resultStandardScheme();
+    private static class cancelCompaction_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public cancelCompaction_resultStandardScheme getScheme() {
+        return new cancelCompaction_resultStandardScheme();
       }
     }
 
-    private static class AddOrUpdateCompaction_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme<AddOrUpdateCompaction_result> {
+    private static class cancelCompaction_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme<cancelCompaction_result> {
 
-      public void read(org.apache.thrift.protocol.TProtocol iprot, AddOrUpdateCompaction_result struct) throws org.apache.thrift.TException {
+      public void read(org.apache.thrift.protocol.TProtocol iprot, cancelCompaction_result struct) throws org.apache.thrift.TException {
         org.apache.thrift.protocol.TField schemeField;
         iprot.readStructBegin();
         while (true)
@@ -1796,7 +1663,7 @@ public class CompactionCoordinator {
         struct.validate();
       }
 
-      public void write(org.apache.thrift.protocol.TProtocol oprot, AddOrUpdateCompaction_result struct) throws org.apache.thrift.TException {
+      public void write(org.apache.thrift.protocol.TProtocol oprot, cancelCompaction_result struct) throws org.apache.thrift.TException {
         struct.validate();
 
         oprot.writeStructBegin(STRUCT_DESC);
@@ -1806,21 +1673,21 @@ public class CompactionCoordinator {
 
     }
 
-    private static class AddOrUpdateCompaction_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      public AddOrUpdateCompaction_resultTupleScheme getScheme() {
-        return new AddOrUpdateCompaction_resultTupleScheme();
+    private static class cancelCompaction_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public cancelCompaction_resultTupleScheme getScheme() {
+        return new cancelCompaction_resultTupleScheme();
       }
     }
 
-    private static class AddOrUpdateCompaction_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme<AddOrUpdateCompaction_result> {
+    private static class cancelCompaction_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme<cancelCompaction_result> {
 
       @Override
-      public void write(org.apache.thrift.protocol.TProtocol prot, AddOrUpdateCompaction_result struct) throws org.apache.thrift.TException {
+      public void write(org.apache.thrift.protocol.TProtocol prot, cancelCompaction_result struct) throws org.apache.thrift.TException {
         org.apache.thrift.protocol.TTupleProtocol oprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
       }
 
       @Override
-      public void read(org.apache.thrift.protocol.TProtocol prot, AddOrUpdateCompaction_result struct) throws org.apache.thrift.TException {
+      public void read(org.apache.thrift.protocol.TProtocol prot, cancelCompaction_result struct) throws org.apache.thrift.TException {
         org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
       }
     }
@@ -1833,22 +1700,22 @@ public class CompactionCoordinator {
   public static class getCompactionStatus_args implements org.apache.thrift.TBase<getCompactionStatus_args, getCompactionStatus_args._Fields>, java.io.Serializable, Cloneable, Comparable<getCompactionStatus_args>   {
     private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getCompactionStatus_args");
 
-    private static final org.apache.thrift.protocol.TField TABLE_ID_FIELD_DESC = new org.apache.thrift.protocol.TField("tableId", org.apache.thrift.protocol.TType.STRING, (short)1);
-    private static final org.apache.thrift.protocol.TField END_ROW_FIELD_DESC = new org.apache.thrift.protocol.TField("endRow", org.apache.thrift.protocol.TType.STRING, (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 EXTENT_FIELD_DESC = new org.apache.thrift.protocol.TField("extent", org.apache.thrift.protocol.TType.STRUCT, (short)1);
+    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)2);
+    private static final org.apache.thrift.protocol.TField PRIORITY_FIELD_DESC = new org.apache.thrift.protocol.TField("priority", org.apache.thrift.protocol.TType.I64, (short)3);
 
     private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new getCompactionStatus_argsStandardSchemeFactory();
     private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new getCompactionStatus_argsTupleSchemeFactory();
 
-    public @org.apache.thrift.annotation.Nullable java.lang.String tableId; // required
-    public @org.apache.thrift.annotation.Nullable java.lang.String endRow; // required
+    public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent; // required
     public @org.apache.thrift.annotation.Nullable java.lang.String queueName; // required
+    public long priority; // 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 {
-      TABLE_ID((short)1, "tableId"),
-      END_ROW((short)2, "endRow"),
-      QUEUE_NAME((short)3, "queueName");
+      EXTENT((short)1, "extent"),
+      QUEUE_NAME((short)2, "queueName"),
+      PRIORITY((short)3, "priority");
 
       private static final java.util.Map<java.lang.String, _Fields> byName = new java.util.HashMap<java.lang.String, _Fields>();
 
@@ -1864,12 +1731,12 @@ public class CompactionCoordinator {
       @org.apache.thrift.annotation.Nullable
       public static _Fields findByThriftId(int fieldId) {
         switch(fieldId) {
-          case 1: // TABLE_ID
-            return TABLE_ID;
-          case 2: // END_ROW
-            return END_ROW;
-          case 3: // QUEUE_NAME
+          case 1: // EXTENT
+            return EXTENT;
+          case 2: // QUEUE_NAME
             return QUEUE_NAME;
+          case 3: // PRIORITY
+            return PRIORITY;
           default:
             return null;
         }
@@ -1911,15 +1778,17 @@ public class CompactionCoordinator {
     }
 
     // isset id assignments
+    private static final int __PRIORITY_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.TABLE_ID, new org.apache.thrift.meta_data.FieldMetaData("tableId", org.apache.thrift.TFieldRequirementType.DEFAULT, 
-          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
-      tmpMap.put(_Fields.END_ROW, new org.apache.thrift.meta_data.FieldMetaData("endRow", 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.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.PRIORITY, new org.apache.thrift.meta_data.FieldMetaData("priority", 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(getCompactionStatus_args.class, metaDataMap);
     }
@@ -1928,29 +1797,29 @@ public class CompactionCoordinator {
     }
 
     public getCompactionStatus_args(
-      java.lang.String tableId,
-      java.lang.String endRow,
-      java.lang.String queueName)
+      org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent,
+      java.lang.String queueName,
+      long priority)
     {
       this();
-      this.tableId = tableId;
-      this.endRow = endRow;
+      this.extent = extent;
       this.queueName = queueName;
+      this.priority = priority;
+      setPriorityIsSet(true);
     }
 
     /**
      * Performs a deep copy on <i>other</i>.
      */
     public getCompactionStatus_args(getCompactionStatus_args other) {
-      if (other.isSetTableId()) {
-        this.tableId = other.tableId;
-      }
-      if (other.isSetEndRow()) {
-        this.endRow = other.endRow;
+      __isset_bitfield = other.__isset_bitfield;
+      if (other.isSetExtent()) {
+        this.extent = new org.apache.accumulo.core.dataImpl.thrift.TKeyExtent(other.extent);
       }
       if (other.isSetQueueName()) {
         this.queueName = other.queueName;
       }
+      this.priority = other.priority;
     }
 
     public getCompactionStatus_args deepCopy() {
@@ -1959,58 +1828,34 @@ public class CompactionCoordinator {
 
     @Override
     public void clear() {
-      this.tableId = null;
-      this.endRow = null;
+      this.extent = null;
       this.queueName = null;
+      setPriorityIsSet(false);
+      this.priority = 0;
     }
 
     @org.apache.thrift.annotation.Nullable
-    public java.lang.String getTableId() {
-      return this.tableId;
-    }
-
-    public getCompactionStatus_args setTableId(@org.apache.thrift.annotation.Nullable java.lang.String tableId) {
-      this.tableId = tableId;
-      return this;
-    }
-
-    public void unsetTableId() {
-      this.tableId = null;
-    }
-
-    /** Returns true if field tableId is set (has been assigned a value) and false otherwise */
-    public boolean isSetTableId() {
-      return this.tableId != null;
-    }
-
-    public void setTableIdIsSet(boolean value) {
-      if (!value) {
-        this.tableId = null;
-      }
-    }
-
-    @org.apache.thrift.annotation.Nullable
-    public java.lang.String getEndRow() {
-      return this.endRow;
+    public org.apache.accumulo.core.dataImpl.thrift.TKeyExtent getExtent() {
+      return this.extent;
     }
 
-    public getCompactionStatus_args setEndRow(@org.apache.thrift.annotation.Nullable java.lang.String endRow) {
-      this.endRow = endRow;
+    public getCompactionStatus_args setExtent(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent) {
+      this.extent = extent;
       return this;
     }
 
-    public void unsetEndRow() {
-      this.endRow = null;
+    public void unsetExtent() {
+      this.extent = null;
     }
 
-    /** Returns true if field endRow is set (has been assigned a value) and false otherwise */
-    public boolean isSetEndRow() {
-      return this.endRow != 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 setEndRowIsSet(boolean value) {
+    public void setExtentIsSet(boolean value) {
       if (!value) {
-        this.endRow = null;
+        this.extent = null;
       }
     }
 
@@ -2039,29 +1884,52 @@ public class CompactionCoordinator {
       }
     }
 
+    public long getPriority() {
+      return this.priority;
+    }
+
+    public getCompactionStatus_args setPriority(long priority) {
+      this.priority = priority;
+      setPriorityIsSet(true);
+      return this;
+    }
+
+    public void unsetPriority() {
+      __isset_bitfield = org.apache.thrift.EncodingUtils.clearBit(__isset_bitfield, __PRIORITY_ISSET_ID);
+    }
+
+    /** Returns true if field priority is set (has been assigned a value) and false otherwise */
+    public boolean isSetPriority() {
+      return org.apache.thrift.EncodingUtils.testBit(__isset_bitfield, __PRIORITY_ISSET_ID);
+    }
+
+    public void setPriorityIsSet(boolean value) {
+      __isset_bitfield = org.apache.thrift.EncodingUtils.setBit(__isset_bitfield, __PRIORITY_ISSET_ID, value);
+    }
+
     public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable java.lang.Object value) {
       switch (field) {
-      case TABLE_ID:
+      case EXTENT:
         if (value == null) {
-          unsetTableId();
+          unsetExtent();
         } else {
-          setTableId((java.lang.String)value);
+          setExtent((org.apache.accumulo.core.dataImpl.thrift.TKeyExtent)value);
         }
         break;
 
-      case END_ROW:
+      case QUEUE_NAME:
         if (value == null) {
-          unsetEndRow();
+          unsetQueueName();
         } else {
-          setEndRow((java.lang.String)value);
+          setQueueName((java.lang.String)value);
         }
         break;
 
-      case QUEUE_NAME:
+      case PRIORITY:
         if (value == null) {
-          unsetQueueName();
+          unsetPriority();
         } else {
-          setQueueName((java.lang.String)value);
+          setPriority((java.lang.Long)value);
         }
         break;
 
@@ -2071,15 +1939,15 @@ public class CompactionCoordinator {
     @org.apache.thrift.annotation.Nullable
     public java.lang.Object getFieldValue(_Fields field) {
       switch (field) {
-      case TABLE_ID:
-        return getTableId();
-
-      case END_ROW:
-        return getEndRow();
+      case EXTENT:
+        return getExtent();
 
       case QUEUE_NAME:
         return getQueueName();
 
+      case PRIORITY:
+        return getPriority();
+
       }
       throw new java.lang.IllegalStateException();
     }
@@ -2091,12 +1959,12 @@ public class CompactionCoordinator {
       }
 
       switch (field) {
-      case TABLE_ID:
-        return isSetTableId();
-      case END_ROW:
-        return isSetEndRow();
+      case EXTENT:
+        return isSetExtent();
       case QUEUE_NAME:
         return isSetQueueName();
+      case PRIORITY:
+        return isSetPriority();
       }
       throw new java.lang.IllegalStateException();
     }
@@ -2116,21 +1984,12 @@ public class CompactionCoordinator {
       if (this == that)
         return true;
 
-      boolean this_present_tableId = true && this.isSetTableId();
-      boolean that_present_tableId = true && that.isSetTableId();
-      if (this_present_tableId || that_present_tableId) {
-        if (!(this_present_tableId && that_present_tableId))
-          return false;
-        if (!this.tableId.equals(that.tableId))
-          return false;
-      }
-
-      boolean this_present_endRow = true && this.isSetEndRow();
-      boolean that_present_endRow = true && that.isSetEndRow();
-      if (this_present_endRow || that_present_endRow) {
-        if (!(this_present_endRow && that_present_endRow))
+      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.endRow.equals(that.endRow))
+        if (!this.extent.equals(that.extent))
           return false;
       }
 
@@ -2143,25 +2002,32 @@ public class CompactionCoordinator {
           return false;
       }
 
-      return true;
-    }
-
-    @Override
-    public int hashCode() {
-      int hashCode = 1;
+      boolean this_present_priority = true;
+      boolean that_present_priority = true;
+      if (this_present_priority || that_present_priority) {
+        if (!(this_present_priority && that_present_priority))
+          return false;
+        if (this.priority != that.priority)
+          return false;
+      }
 
-      hashCode = hashCode * 8191 + ((isSetTableId()) ? 131071 : 524287);
-      if (isSetTableId())
-        hashCode = hashCode * 8191 + tableId.hashCode();
+      return true;
+    }
+
+    @Override
+    public int hashCode() {
+      int hashCode = 1;
 
-      hashCode = hashCode * 8191 + ((isSetEndRow()) ? 131071 : 524287);
-      if (isSetEndRow())
-        hashCode = hashCode * 8191 + endRow.hashCode();
+      hashCode = hashCode * 8191 + ((isSetExtent()) ? 131071 : 524287);
+      if (isSetExtent())
+        hashCode = hashCode * 8191 + extent.hashCode();
 
       hashCode = hashCode * 8191 + ((isSetQueueName()) ? 131071 : 524287);
       if (isSetQueueName())
         hashCode = hashCode * 8191 + queueName.hashCode();
 
+      hashCode = hashCode * 8191 + org.apache.thrift.TBaseHelper.hashCode(priority);
+
       return hashCode;
     }
 
@@ -2173,32 +2039,32 @@ public class CompactionCoordinator {
 
       int lastComparison = 0;
 
-      lastComparison = java.lang.Boolean.valueOf(isSetTableId()).compareTo(other.isSetTableId());
+      lastComparison = java.lang.Boolean.valueOf(isSetExtent()).compareTo(other.isSetExtent());
       if (lastComparison != 0) {
         return lastComparison;
       }
-      if (isSetTableId()) {
-        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tableId, other.tableId);
+      if (isSetExtent()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.extent, other.extent);
         if (lastComparison != 0) {
           return lastComparison;
         }
       }
-      lastComparison = java.lang.Boolean.valueOf(isSetEndRow()).compareTo(other.isSetEndRow());
+      lastComparison = java.lang.Boolean.valueOf(isSetQueueName()).compareTo(other.isSetQueueName());
       if (lastComparison != 0) {
         return lastComparison;
       }
-      if (isSetEndRow()) {
-        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.endRow, other.endRow);
+      if (isSetQueueName()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.queueName, other.queueName);
         if (lastComparison != 0) {
           return lastComparison;
         }
       }
-      lastComparison = java.lang.Boolean.valueOf(isSetQueueName()).compareTo(other.isSetQueueName());
+      lastComparison = java.lang.Boolean.valueOf(isSetPriority()).compareTo(other.isSetPriority());
       if (lastComparison != 0) {
         return lastComparison;
       }
-      if (isSetQueueName()) {
-        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.queueName, other.queueName);
+      if (isSetPriority()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.priority, other.priority);
         if (lastComparison != 0) {
           return lastComparison;
         }
@@ -2224,19 +2090,11 @@ public class CompactionCoordinator {
       java.lang.StringBuilder sb = new java.lang.StringBuilder("getCompactionStatus_args(");
       boolean first = true;
 
-      sb.append("tableId:");
-      if (this.tableId == null) {
-        sb.append("null");
-      } else {
-        sb.append(this.tableId);
-      }
-      first = false;
-      if (!first) sb.append(", ");
-      sb.append("endRow:");
-      if (this.endRow == null) {
+      sb.append("extent:");
+      if (this.extent == null) {
         sb.append("null");
       } else {
-        sb.append(this.endRow);
+        sb.append(this.extent);
       }
       first = false;
       if (!first) sb.append(", ");
@@ -2247,6 +2105,10 @@ public class CompactionCoordinator {
         sb.append(this.queueName);
       }
       first = false;
+      if (!first) sb.append(", ");
+      sb.append("priority:");
+      sb.append(this.priority);
+      first = false;
       sb.append(")");
       return sb.toString();
     }
@@ -2254,6 +2116,9 @@ public class CompactionCoordinator {
     public void validate() throws org.apache.thrift.TException {
       // check for required fields
       // check for sub-struct validity
+      if (extent != null) {
+        extent.validate();
+      }
     }
 
     private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
@@ -2266,6 +2131,8 @@ public class CompactionCoordinator {
 
     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);
@@ -2290,26 +2157,27 @@ public class CompactionCoordinator {
             break;
           }
           switch (schemeField.id) {
-            case 1: // TABLE_ID
-              if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
-                struct.tableId = iprot.readString();
-                struct.setTableIdIsSet(true);
+            case 1: // 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 2: // END_ROW
+            case 2: // QUEUE_NAME
               if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
-                struct.endRow = iprot.readString();
-                struct.setEndRowIsSet(true);
+                struct.queueName = iprot.readString();
+                struct.setQueueNameIsSet(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);
+            case 3: // PRIORITY
+              if (schemeField.type == org.apache.thrift.protocol.TType.I64) {
+                struct.priority = iprot.readI64();
+                struct.setPriorityIsSet(true);
               } else { 
                 org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
               }
@@ -2329,14 +2197,9 @@ public class CompactionCoordinator {
         struct.validate();
 
         oprot.writeStructBegin(STRUCT_DESC);
-        if (struct.tableId != null) {
-          oprot.writeFieldBegin(TABLE_ID_FIELD_DESC);
-          oprot.writeString(struct.tableId);
-          oprot.writeFieldEnd();
-        }
-        if (struct.endRow != null) {
-          oprot.writeFieldBegin(END_ROW_FIELD_DESC);
-          oprot.writeString(struct.endRow);
+        if (struct.extent != null) {
+          oprot.writeFieldBegin(EXTENT_FIELD_DESC);
+          struct.extent.write(oprot);
           oprot.writeFieldEnd();
         }
         if (struct.queueName != null) {
@@ -2344,6 +2207,9 @@ public class CompactionCoordinator {
           oprot.writeString(struct.queueName);
           oprot.writeFieldEnd();
         }
+        oprot.writeFieldBegin(PRIORITY_FIELD_DESC);
+        oprot.writeI64(struct.priority);
+        oprot.writeFieldEnd();
         oprot.writeFieldStop();
         oprot.writeStructEnd();
       }
@@ -2362,25 +2228,25 @@ public class CompactionCoordinator {
       public void write(org.apache.thrift.protocol.TProtocol prot, getCompactionStatus_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.isSetTableId()) {
+        if (struct.isSetExtent()) {
           optionals.set(0);
         }
-        if (struct.isSetEndRow()) {
+        if (struct.isSetQueueName()) {
           optionals.set(1);
         }
-        if (struct.isSetQueueName()) {
+        if (struct.isSetPriority()) {
           optionals.set(2);
         }
         oprot.writeBitSet(optionals, 3);
-        if (struct.isSetTableId()) {
-          oprot.writeString(struct.tableId);
-        }
-        if (struct.isSetEndRow()) {
-          oprot.writeString(struct.endRow);
+        if (struct.isSetExtent()) {
+          struct.extent.write(oprot);
         }
         if (struct.isSetQueueName()) {
           oprot.writeString(struct.queueName);
         }
+        if (struct.isSetPriority()) {
+          oprot.writeI64(struct.priority);
+        }
       }
 
       @Override
@@ -2388,17 +2254,18 @@ public class CompactionCoordinator {
         org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
         java.util.BitSet incoming = iprot.readBitSet(3);
         if (incoming.get(0)) {
-          struct.tableId = iprot.readString();
-          struct.setTableIdIsSet(true);
+          struct.extent = new org.apache.accumulo.core.dataImpl.thrift.TKeyExtent();
+          struct.extent.read(iprot);
+          struct.setExtentIsSet(true);
         }
         if (incoming.get(1)) {
-          struct.endRow = iprot.readString();
-          struct.setEndRowIsSet(true);
-        }
-        if (incoming.get(2)) {
           struct.queueName = iprot.readString();
           struct.setQueueNameIsSet(true);
         }
+        if (incoming.get(2)) {
+          struct.priority = iprot.readI64();
+          struct.setPriorityIsSet(true);
+        }
       }
     }
 
@@ -2410,12 +2277,12 @@ public class CompactionCoordinator {
   public static class getCompactionStatus_result implements org.apache.thrift.TBase<getCompactionStatus_result, getCompactionStatus_result._Fields>, java.io.Serializable, Cloneable, Comparable<getCompactionStatus_result>   {
     private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getCompactionStatus_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 SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.LIST, (short)0);
 
     private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new getCompactionStatus_resultStandardSchemeFactory();
     private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new getCompactionStatus_resultTupleSchemeFactory();
 
-    public @org.apache.thrift.annotation.Nullable Status success; // required
+    public @org.apache.thrift.annotation.Nullable java.util.List<Status> 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 {
@@ -2482,7 +2349,8 @@ public class CompactionCoordinator {
     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, Status.class)));
+          new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, 
+              new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, Status.class))));
       metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
       org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(getCompactionStatus_result.class, metaDataMap);
     }
@@ -2491,7 +2359,7 @@ public class CompactionCoordinator {
     }
 
     public getCompactionStatus_result(
-      Status success)
+      java.util.List<Status> success)
     {
       this();
       this.success = success;
@@ -2502,7 +2370,11 @@ public class CompactionCoordinator {
      */
     public getCompactionStatus_result(getCompactionStatus_result other) {
       if (other.isSetSuccess()) {
-        this.success = new Status(other.success);
+        java.util.List<Status> __this__success = new java.util.ArrayList<Status>(other.success.size());
+        for (Status other_element : other.success) {
+          __this__success.add(new Status(other_element));
+        }
+        this.success = __this__success;
       }
     }
 
@@ -2515,12 +2387,28 @@ public class CompactionCoordinator {
       this.success = null;
     }
 
+    public int getSuccessSize() {
+      return (this.success == null) ? 0 : this.success.size();
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public java.util.Iterator<Status> getSuccessIterator() {
+      return (this.success == null) ? null : this.success.iterator();
+    }
+
+    public void addToSuccess(Status elem) {
+      if (this.success == null) {
+        this.success = new java.util.ArrayList<Status>();
+      }
+      this.success.add(elem);
+    }
+
     @org.apache.thrift.annotation.Nullable
-    public Status getSuccess() {
+    public java.util.List<Status> getSuccess() {
       return this.success;
     }
 
-    public getCompactionStatus_result setSuccess(@org.apache.thrift.annotation.Nullable Status success) {
+    public getCompactionStatus_result setSuccess(@org.apache.thrift.annotation.Nullable java.util.List<Status> success) {
       this.success = success;
       return this;
     }
@@ -2546,7 +2434,7 @@ public class CompactionCoordinator {
         if (value == null) {
           unsetSuccess();
         } else {
-          setSuccess((Status)value);
+          setSuccess((java.util.List<Status>)value);
         }
         break;
 
@@ -2667,9 +2555,6 @@ public class CompactionCoordinator {
     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 {
@@ -2707,9 +2592,19 @@ public class CompactionCoordinator {
           }
           switch (schemeField.id) {
             case 0: // SUCCESS
-              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
-                struct.success = new Status();
-                struct.success.read(iprot);
+              if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
+                {
+                  org.apache.thrift.protocol.TList _list0 = iprot.readListBegin();
+                  struct.success = new java.util.ArrayList<Status>(_list0.size);
+                  @org.apache.thrift.annotation.Nullable Status _elem1;
+                  for (int _i2 = 0; _i2 < _list0.size; ++_i2)
+                  {
+                    _elem1 = new Status();
+                    _elem1.read(iprot);
+                    struct.success.add(_elem1);
+                  }
+                  iprot.readListEnd();
+                }
                 struct.setSuccessIsSet(true);
               } else { 
                 org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
@@ -2732,7 +2627,14 @@ public class CompactionCoordinator {
         oprot.writeStructBegin(STRUCT_DESC);
         if (struct.success != null) {
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
-          struct.success.write(oprot);
+          {
+            oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size()));
+            for (Status _iter3 : struct.success)
+            {
+              _iter3.write(oprot);
+            }
+            oprot.writeListEnd();
+          }
           oprot.writeFieldEnd();
         }
         oprot.writeFieldStop();
@@ -2758,7 +2660,13 @@ public class CompactionCoordinator {
         }
         oprot.writeBitSet(optionals, 1);
         if (struct.isSetSuccess()) {
-          struct.success.write(oprot);
+          {
+            oprot.writeI32(struct.success.size());
+            for (Status _iter4 : struct.success)
+            {
+              _iter4.write(oprot);
+            }
+          }
         }
       }
 
@@ -2767,8 +2675,17 @@ public class CompactionCoordinator {
         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 Status();
-          struct.success.read(iprot);
+          {
+            org.apache.thrift.protocol.TList _list5 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.success = new java.util.ArrayList<Status>(_list5.size);
+            @org.apache.thrift.annotation.Nullable Status _elem6;
+            for (int _i7 = 0; _i7 < _list5.size; ++_i7)
+            {
+              _elem6 = new Status();
+              _elem6.read(iprot);
+              struct.success.add(_elem6);
+            }
+          }
           struct.setSuccessIsSet(true);
         }
       }
@@ -2779,25 +2696,22 @@ public class CompactionCoordinator {
     }
   }
 
-  public static class getCompactionStatusHistory_args implements org.apache.thrift.TBase<getCompactionStatusHistory_args, getCompactionStatusHistory_args._Fields>, java.io.Serializable, Cloneable, Comparable<getCompactionStatusHistory_args>   {
-    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getCompactionStatusHistory_args");
+  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 TABLE_ID_FIELD_DESC = new org.apache.thrift.protocol.TField("tableId", org.apache.thrift.protocol.TType.STRING, (short)1);
-    private static final org.apache.thrift.protocol.TField END_ROW_FIELD_DESC = new org.apache.thrift.protocol.TField("endRow", org.apache.thrift.protocol.TType.STRING, (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 QUEUE_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("queueName", org.apache.thrift.protocol.TType.STRING, (short)1);
+    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)2);
 
-    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new getCompactionStatusHistory_argsStandardSchemeFactory();
-    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new getCompactionStatusHistory_argsTupleSchemeFactory();
+    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 java.lang.String tableId; // required
-    public @org.apache.thrift.annotation.Nullable java.lang.String endRow; // required
     public @org.apache.thrift.annotation.Nullable java.lang.String queueName; // required
+    public @org.apache.thrift.annotation.Nullable java.lang.String compactor; // 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 {
-      TABLE_ID((short)1, "tableId"),
-      END_ROW((short)2, "endRow"),
-      QUEUE_NAME((short)3, "queueName");
+      QUEUE_NAME((short)1, "queueName"),
+      COMPACTOR((short)2, "compactor");
 
       private static final java.util.Map<java.lang.String, _Fields> byName = new java.util.HashMap<java.lang.String, _Fields>();
 
@@ -2813,12 +2727,10 @@ public class CompactionCoordinator {
       @org.apache.thrift.annotation.Nullable
       public static _Fields findByThriftId(int fieldId) {
         switch(fieldId) {
-          case 1: // TABLE_ID
-            return TABLE_ID;
-          case 2: // END_ROW
-            return END_ROW;
-          case 3: // QUEUE_NAME
+          case 1: // QUEUE_NAME
             return QUEUE_NAME;
+          case 2: // COMPACTOR
+            return COMPACTOR;
           default:
             return null;
         }
@@ -2863,154 +2775,113 @@ public class CompactionCoordinator {
     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.TABLE_ID, new org.apache.thrift.meta_data.FieldMetaData("tableId", org.apache.thrift.TFieldRequirementType.DEFAULT, 
-          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
-      tmpMap.put(_Fields.END_ROW, new org.apache.thrift.meta_data.FieldMetaData("endRow", org.apache.thrift.TFieldRequirementType.DEFAULT, 
-          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
       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)));
       metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
-      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(getCompactionStatusHistory_args.class, metaDataMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(getCompactionJob_args.class, metaDataMap);
     }
 
-    public getCompactionStatusHistory_args() {
+    public getCompactionJob_args() {
     }
 
-    public getCompactionStatusHistory_args(
-      java.lang.String tableId,
-      java.lang.String endRow,
-      java.lang.String queueName)
+    public getCompactionJob_args(
+      java.lang.String queueName,
+      java.lang.String compactor)
     {
       this();
-      this.tableId = tableId;
-      this.endRow = endRow;
       this.queueName = queueName;
+      this.compactor = compactor;
     }
 
     /**
      * Performs a deep copy on <i>other</i>.
      */
-    public getCompactionStatusHistory_args(getCompactionStatusHistory_args other) {
-      if (other.isSetTableId()) {
-        this.tableId = other.tableId;
-      }
-      if (other.isSetEndRow()) {
-        this.endRow = other.endRow;
-      }
+    public getCompactionJob_args(getCompactionJob_args other) {
       if (other.isSetQueueName()) {
         this.queueName = other.queueName;
       }
+      if (other.isSetCompactor()) {
+        this.compactor = other.compactor;
+      }
     }
 
-    public getCompactionStatusHistory_args deepCopy() {
-      return new getCompactionStatusHistory_args(this);
+    public getCompactionJob_args deepCopy() {
+      return new getCompactionJob_args(this);
     }
 
     @Override
     public void clear() {
-      this.tableId = null;
-      this.endRow = null;
       this.queueName = null;
+      this.compactor = null;
     }
 
     @org.apache.thrift.annotation.Nullable
-    public java.lang.String getTableId() {
-      return this.tableId;
-    }
-
-    public getCompactionStatusHistory_args setTableId(@org.apache.thrift.annotation.Nullable java.lang.String tableId) {
-      this.tableId = tableId;
-      return this;
-    }
-
-    public void unsetTableId() {
-      this.tableId = null;
-    }
-
-    /** Returns true if field tableId is set (has been assigned a value) and false otherwise */
-    public boolean isSetTableId() {
-      return this.tableId != null;
-    }
-
-    public void setTableIdIsSet(boolean value) {
-      if (!value) {
-        this.tableId = null;
-      }
-    }
-
-    @org.apache.thrift.annotation.Nullable
-    public java.lang.String getEndRow() {
-      return this.endRow;
+    public java.lang.String getQueueName() {
+      return this.queueName;
     }
 
-    public getCompactionStatusHistory_args setEndRow(@org.apache.thrift.annotation.Nullable java.lang.String endRow) {
-      this.endRow = endRow;
+    public getCompactionJob_args setQueueName(@org.apache.thrift.annotation.Nullable java.lang.String queueName) {
+      this.queueName = queueName;
       return this;
     }
 
-    public void unsetEndRow() {
-      this.endRow = null;
+    public void unsetQueueName() {
+      this.queueName = null;
     }
 
-    /** Returns true if field endRow is set (has been assigned a value) and false otherwise */
-    public boolean isSetEndRow() {
-      return this.endRow != 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 setEndRowIsSet(boolean value) {
+    public void setQueueNameIsSet(boolean value) {
       if (!value) {
-        this.endRow = null;
+        this.queueName = null;
       }
     }
 
     @org.apache.thrift.annotation.Nullable
-    public java.lang.String getQueueName() {
-      return this.queueName;
+    public java.lang.String getCompactor() {
+      return this.compactor;
     }
 
-    public getCompactionStatusHistory_args setQueueName(@org.apache.thrift.annotation.Nullable java.lang.String queueName) {
-      this.queueName = queueName;
+    public getCompactionJob_args setCompactor(@org.apache.thrift.annotation.Nullable java.lang.String compactor) {
+      this.compactor = compactor;
       return this;
     }
 
-    public void unsetQueueName() {
-      this.queueName = null;
+    public void unsetCompactor() {
+      this.compactor = null;
     }
 
-    /** Returns true if field queueName is set (has been assigned a value) and false otherwise */
-    public boolean isSetQueueName() {
-      return this.queueName != 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 setQueueNameIsSet(boolean value) {
+    public void setCompactorIsSet(boolean value) {
       if (!value) {
-        this.queueName = null;
+        this.compactor = null;
       }
     }
 
     public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable java.lang.Object value) {
       switch (field) {
-      case TABLE_ID:
-        if (value == null) {
-          unsetTableId();
-        } else {
-          setTableId((java.lang.String)value);
-        }
-        break;
-
-      case END_ROW:
+      case QUEUE_NAME:
         if (value == null) {
-          unsetEndRow();
+          unsetQueueName();
         } else {
-          setEndRow((java.lang.String)value);
+          setQueueName((java.lang.String)value);
         }
         break;
 
-      case QUEUE_NAME:
+      case COMPACTOR:
         if (value == null) {
-          unsetQueueName();
+          unsetCompactor();
         } else {
-          setQueueName((java.lang.String)value);
+          setCompactor((java.lang.String)value);
         }
         break;
 
@@ -3020,15 +2891,12 @@ public class CompactionCoordinator {
     @org.apache.thrift.annotation.Nullable
     public java.lang.Object getFieldValue(_Fields field) {
       switch (field) {
-      case TABLE_ID:
-        return getTableId();
-
-      case END_ROW:
-        return getEndRow();
-
       case QUEUE_NAME:
         return getQueueName();
 
+      case COMPACTOR:
+        return getCompactor();
+
       }
       throw new java.lang.IllegalStateException();
     }
@@ -3040,12 +2908,10 @@ public class CompactionCoordinator {
       }
 
       switch (field) {
-      case TABLE_ID:
-        return isSetTableId();
-      case END_ROW:
-        return isSetEndRow();
       case QUEUE_NAME:
         return isSetQueueName();
+      case COMPACTOR:
+        return isSetCompactor();
       }
       throw new java.lang.IllegalStateException();
     }
@@ -3054,35 +2920,17 @@ public class CompactionCoordinator {
     public boolean equals(java.lang.Object that) {
       if (that == null)
         return false;
-      if (that instanceof getCompactionStatusHistory_args)
-        return this.equals((getCompactionStatusHistory_args)that);
+      if (that instanceof getCompactionJob_args)
+        return this.equals((getCompactionJob_args)that);
       return false;
     }
 
-    public boolean equals(getCompactionStatusHistory_args that) {
+    public boolean equals(getCompactionJob_args that) {
       if (that == null)
         return false;
       if (this == that)
         return true;
 
-      boolean this_present_tableId = true && this.isSetTableId();
-      boolean that_present_tableId = true && that.isSetTableId();
-      if (this_present_tableId || that_present_tableId) {
-        if (!(this_present_tableId && that_present_tableId))
-          return false;
-        if (!this.tableId.equals(that.tableId))
-          return false;
-      }
-
-      boolean this_present_endRow = true && this.isSetEndRow();
-      boolean that_present_endRow = true && that.isSetEndRow();
-      if (this_present_endRow || that_present_endRow) {
-        if (!(this_present_endRow && that_present_endRow))
-          return false;
-        if (!this.endRow.equals(that.endRow))
-          return false;
-      }
-
       boolean this_present_queueName = true && this.isSetQueueName();
       boolean that_present_queueName = true && that.isSetQueueName();
       if (this_present_queueName || that_present_queueName) {
@@ -3092,6 +2940,15 @@ public class CompactionCoordinator {
           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;
+      }
+
       return true;
     }
 
@@ -3099,55 +2956,41 @@ public class CompactionCoordinator {
     public int hashCode() {
       int hashCode = 1;
 
-      hashCode = hashCode * 8191 + ((isSetTableId()) ? 131071 : 524287);
-      if (isSetTableId())
-        hashCode = hashCode * 8191 + tableId.hashCode();
-
-      hashCode = hashCode * 8191 + ((isSetEndRow()) ? 131071 : 524287);
-      if (isSetEndRow())
-        hashCode = hashCode * 8191 + endRow.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();
+
       return hashCode;
     }
 
     @Override
-    public int compareTo(getCompactionStatusHistory_args other) {
+    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(isSetTableId()).compareTo(other.isSetTableId());
-      if (lastComparison != 0) {
-        return lastComparison;
-      }
-      if (isSetTableId()) {
-        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tableId, other.tableId);
-        if (lastComparison != 0) {
-          return lastComparison;
-        }
-      }
-      lastComparison = java.lang.Boolean.valueOf(isSetEndRow()).compareTo(other.isSetEndRow());
+      lastComparison = java.lang.Boolean.valueOf(isSetQueueName()).compareTo(other.isSetQueueName());
       if (lastComparison != 0) {
         return lastComparison;
       }
-      if (isSetEndRow()) {
-        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.endRow, other.endRow);
+      if (isSetQueueName()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.queueName, other.queueName);
         if (lastComparison != 0) {
           return lastComparison;
         }
       }
-      lastComparison = java.lang.Boolean.valueOf(isSetQueueName()).compareTo(other.isSetQueueName());
+      lastComparison = java.lang.Boolean.valueOf(isSetCompactor()).compareTo(other.isSetCompactor());
       if (lastComparison != 0) {
         return lastComparison;
       }
-      if (isSetQueueName()) {
-        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.queueName, other.queueName);
+      if (isSetCompactor()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.compactor, other.compactor);
         if (lastComparison != 0) {
           return lastComparison;
         }
@@ -3170,30 +3013,22 @@ public class CompactionCoordinator {
 
     @Override
     public java.lang.String toString() {
-      java.lang.StringBuilder sb = new java.lang.StringBuilder("getCompactionStatusHistory_args(");
+      java.lang.StringBuilder sb = new java.lang.StringBuilder("getCompactionJob_args(");
       boolean first = true;
 
-      sb.append("tableId:");
-      if (this.tableId == null) {
+      sb.append("queueName:");
+      if (this.queueName == null) {
         sb.append("null");
       } else {
-        sb.append(this.tableId);
+        sb.append(this.queueName);
       }
       first = false;
       if (!first) sb.append(", ");
-      sb.append("endRow:");
-      if (this.endRow == null) {
+      sb.append("compactor:");
+      if (this.compactor == null) {
         sb.append("null");
       } else {
-        sb.append(this.endRow);
-      }
-      first = false;
-      if (!first) sb.append(", ");
-      sb.append("queueName:");
-      if (this.queueName == null) {
-        sb.append("null");
-      } else {
-        sb.append(this.queueName);
+        sb.append(this.compactor);
       }
       first = false;
       sb.append(")");
@@ -3221,15 +3056,15 @@ public class CompactionCoordinator {
       }
     }
 
-    private static class getCompactionStatusHistory_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      public getCompactionStatusHistory_argsStandardScheme getScheme() {
-        return new getCompactionStatusHistory_argsStandardScheme();
+    private static class getCompactionJob_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public getCompactionJob_argsStandardScheme getScheme() {
+        return new getCompactionJob_argsStandardScheme();
       }
     }
 
-    private static class getCompactionStatusHistory_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme<getCompactionStatusHistory_args> {
+    private static class getCompactionJob_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme<getCompactionJob_args> {
 
-      public void read(org.apache.thrift.protocol.TProtocol iprot, getCompactionStatusHistory_args struct) throws org.apache.thrift.TException {
+      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)
@@ -3239,26 +3074,18 @@ public class CompactionCoordinator {
             break;
           }
           switch (schemeField.id) {
-            case 1: // TABLE_ID
-              if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
-                struct.tableId = iprot.readString();
-                struct.setTableIdIsSet(true);
-              } else { 
-                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
-              }
-              break;
-            case 2: // END_ROW
+            case 1: // QUEUE_NAME
               if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
-                struct.endRow = iprot.readString();
-                struct.setEndRowIsSet(true);
+                struct.queueName = iprot.readString();
+                struct.setQueueNameIsSet(true);
               } else { 
                 org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
               }
               break;
-            case 3: // QUEUE_NAME
+            case 2: // COMPACTOR
               if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
-                struct.queueName = iprot.readString();
-                struct.setQueueNameIsSet(true);
+                struct.compactor = iprot.readString();
+                struct.setCompactorIsSet(true);
               } else { 
                 org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
               }
@@ -3274,80 +3101,65 @@ public class CompactionCoordinator {
         struct.validate();
       }
 
-      public void write(org.apache.thrift.protocol.TProtocol oprot, getCompactionStatusHistory_args struct) throws org.apache.thrift.TException {
+      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.tableId != null) {
-          oprot.writeFieldBegin(TABLE_ID_FIELD_DESC);
-          oprot.writeString(struct.tableId);
-          oprot.writeFieldEnd();
-        }
-        if (struct.endRow != null) {
-          oprot.writeFieldBegin(END_ROW_FIELD_DESC);
-          oprot.writeString(struct.endRow);
-          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();
+        }
         oprot.writeFieldStop();
         oprot.writeStructEnd();
       }
 
     }
 
-    private static class getCompactionStatusHistory_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      public getCompactionStatusHistory_argsTupleScheme getScheme() {
-        return new getCompactionStatusHistory_argsTupleScheme();
+    private static class getCompactionJob_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public getCompactionJob_argsTupleScheme getScheme() {
+        return new getCompactionJob_argsTupleScheme();
       }
     }
 
-    private static class getCompactionStatusHistory_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme<getCompactionStatusHistory_args> {
+    private static class getCompactionJob_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme<getCompactionJob_args> {
 
       @Override
-      public void write(org.apache.thrift.protocol.TProtocol prot, getCompactionStatusHistory_args struct) throws org.apache.thrift.TException {
+      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.isSetTableId()) {
+        if (struct.isSetQueueName()) {
           optionals.set(0);
         }
-        if (struct.isSetEndRow()) {
+        if (struct.isSetCompactor()) {
           optionals.set(1);
         }
-        if (struct.isSetQueueName()) {
-          optionals.set(2);
-        }
-        oprot.writeBitSet(optionals, 3);
-        if (struct.isSetTableId()) {
-          oprot.writeString(struct.tableId);
-        }
-        if (struct.isSetEndRow()) {
-          oprot.writeString(struct.endRow);
-        }
+        oprot.writeBitSet(optionals, 2);
         if (struct.isSetQueueName()) {
           oprot.writeString(struct.queueName);
         }
+        if (struct.isSetCompactor()) {
+          oprot.writeString(struct.compactor);
+        }
       }
 
       @Override
-      public void read(org.apache.thrift.protocol.TProtocol prot, getCompactionStatusHistory_args struct) throws org.apache.thrift.TException {
+      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(3);
+        java.util.BitSet incoming = iprot.readBitSet(2);
         if (incoming.get(0)) {
-          struct.tableId = iprot.readString();
-          struct.setTableIdIsSet(true);
-        }
-        if (incoming.get(1)) {
-          struct.endRow = iprot.readString();
-          struct.setEndRowIsSet(true);
-        }
-        if (incoming.get(2)) {
           struct.queueName = iprot.readString();
           struct.setQueueNameIsSet(true);
         }
+        if (incoming.get(1)) {
+          struct.compactor = iprot.readString();
+          struct.setCompactorIsSet(true);
+        }
       }
     }
 
@@ -3356,15 +3168,15 @@ public class CompactionCoordinator {
     }
   }
 
-  public static class getCompactionStatusHistory_result implements org.apache.thrift.TBase<getCompactionStatusHistory_result, getCompactionStatusHistory_result._Fields>, java.io.Serializable, Cloneable, Comparable<getCompactionStatusHistory_result>   {
-    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getCompactionStatusHistory_result");
+  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.LIST, (short)0);
+    private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.STRUCT, (short)0);
 
-    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new getCompactionStatusHistory_resultStandardSchemeFactory();
-    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new getCompactionStatusHistory_resultTupleSchemeFactory();
+    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 java.util.List<Status> success; // required
+    public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.tabletserver.thrift.CompactionJob 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 {
@@ -3431,17 +3243,16 @@ public class CompactionCoordinator {
     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.ListMetaData(org.apache.thrift.protocol.TType.LIST, 
-              new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, Status.class))));
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.tabletserver.thrift.CompactionJob.class)));
       metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
-      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(getCompactionStatusHistory_result.class, metaDataMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(getCompactionJob_result.class, metaDataMap);
     }
 
-    public getCompactionStatusHistory_result() {
+    public getCompactionJob_result() {
     }
 
-    public getCompactionStatusHistory_result(
-      java.util.List<Status> success)
+    public getCompactionJob_result(
+      org.apache.accumulo.core.tabletserver.thrift.CompactionJob success)
     {
       this();
       this.success = success;
@@ -3450,18 +3261,14 @@ public class CompactionCoordinator {
     /**
      * Performs a deep copy on <i>other</i>.
      */
-    public getCompactionStatusHistory_result(getCompactionStatusHistory_result other) {
+    public getCompactionJob_result(getCompactionJob_result other) {
       if (other.isSetSuccess()) {
-        java.util.List<Status> __this__success = new java.util.ArrayList<Status>(other.success.size());
-        for (Status other_element : other.success) {
-          __this__success.add(new Status(other_element));
-        }
-        this.success = __this__success;
+        this.success = new org.apache.accumulo.core.tabletserver.thrift.CompactionJob(other.success);
       }
     }
 
-    public getCompactionStatusHistory_result deepCopy() {
-      return new getCompactionStatusHistory_result(this);
+    public getCompactionJob_result deepCopy() {
+      return new getCompactionJob_result(this);
     }
 
     @Override
@@ -3469,28 +3276,12 @@ public class CompactionCoordinator {
       this.success = null;
     }
 
-    public int getSuccessSize() {
-      return (this.success == null) ? 0 : this.success.size();
-    }
-
     @org.apache.thrift.annotation.Nullable
-    public java.util.Iterator<Status> getSuccessIterator() {
-      return (this.success == null) ? null : this.success.iterator();
-    }
-
-    public void addToSuccess(Status elem) {
-      if (this.success == null) {
-        this.success = new java.util.ArrayList<Status>();
-      }
-      this.success.add(elem);
-    }
-
-    @org.apache.thrift.annotation.Nullable
-    public java.util.List<Status> getSuccess() {
+    public org.apache.accumulo.core.tabletserver.thrift.CompactionJob getSuccess() {
       return this.success;
     }
 
-    public getCompactionStatusHistory_result setSuccess(@org.apache.thrift.annotation.Nullable java.util.List<Status> success) {
+    public getCompactionJob_result setSuccess(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.tabletserver.thrift.CompactionJob success) {
       this.success = success;
       return this;
     }
@@ -3516,7 +3307,7 @@ public class CompactionCoordinator {
         if (value == null) {
           unsetSuccess();
         } else {
-          setSuccess((java.util.List<Status>)value);
+          setSuccess((org.apache.accumulo.core.tabletserver.thrift.CompactionJob)value);
         }
         break;
 
@@ -3550,12 +3341,12 @@ public class CompactionCoordinator {
     public boolean equals(java.lang.Object that) {
       if (that == null)
         return false;
-      if (that instanceof getCompactionStatusHistory_result)
-        return this.equals((getCompactionStatusHistory_result)that);
+      if (that instanceof getCompactionJob_result)
+        return this.equals((getCompactionJob_result)that);
       return false;
     }
 
-    public boolean equals(getCompactionStatusHistory_result that) {
+    public boolean equals(getCompactionJob_result that) {
       if (that == null)
         return false;
       if (this == that)
@@ -3585,7 +3376,7 @@ public class CompactionCoordinator {
     }
 
     @Override
-    public int compareTo(getCompactionStatusHistory_result other) {
+    public int compareTo(getCompactionJob_result other) {
       if (!getClass().equals(other.getClass())) {
         return getClass().getName().compareTo(other.getClass().getName());
       }
@@ -3620,7 +3411,7 @@ public class CompactionCoordinator {
 
     @Override
     public java.lang.String toString() {
-      java.lang.StringBuilder sb = new java.lang.StringBuilder("getCompactionStatusHistory_result(");
+      java.lang.StringBuilder sb = new java.lang.StringBuilder("getCompactionJob_result(");
       boolean first = true;
 
       sb.append("success:");
@@ -3637,6 +3428,9 @@ public class CompactionCoordinator {
     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 {
@@ -3655,15 +3449,15 @@ public class CompactionCoordinator {
       }
     }
 
-    private static class getCompactionStatusHistory_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      public getCompactionStatusHistory_resultStandardScheme getScheme() {
-        return new getCompactionStatusHistory_resultStandardScheme();
+    private static class getCompactionJob_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public getCompactionJob_resultStandardScheme getScheme() {
+        return new getCompactionJob_resultStandardScheme();
       }
     }
 
-    private static class getCompactionStatusHistory_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme<getCompactionStatusHistory_result> {
+    private static class getCompactionJob_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme<getCompactionJob_result> {
 
-      public void read(org.apache.thrift.protocol.TProtocol iprot, getCompactionStatusHistory_result struct) throws org.apache.thrift.TException {
+      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)
@@ -3674,19 +3468,9 @@ public class CompactionCoordinator {
           }
           switch (schemeField.id) {
             case 0: // SUCCESS
-              if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
-                {
-                  org.apache.thrift.protocol.TList _list8 = iprot.readListBegin();
-                  struct.success = new java.util.ArrayList<Status>(_list8.size);
-                  @org.apache.thrift.annotation.Nullable Status _elem9;
-                  for (int _i10 = 0; _i10 < _list8.size; ++_i10)
-                  {
-                    _elem9 = new Status();
-                    _elem9.read(iprot);
-                    struct.success.add(_elem9);
-                  }
-                  iprot.readListEnd();
-                }
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+                struct.success = new org.apache.accumulo.core.tabletserver.thrift.CompactionJob();
+                struct.success.read(iprot);
                 struct.setSuccessIsSet(true);
               } else { 
                 org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
@@ -3703,20 +3487,13 @@ public class CompactionCoordinator {
         struct.validate();
       }
 
-      public void write(org.apache.thrift.protocol.TProtocol oprot, getCompactionStatusHistory_result struct) throws org.apache.thrift.TException {
+      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);
-          {
-            oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size()));
-            for (Status _iter11 : struct.success)
-            {
-              _iter11.write(oprot);
-            }
-            oprot.writeListEnd();
-          }
+          struct.success.write(oprot);
           oprot.writeFieldEnd();
         }
         oprot.writeFieldStop();
@@ -3725,16 +3502,16 @@ public class CompactionCoordinator {
 
     }
 
-    private static class getCompactionStatusHistory_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      public getCompactionStatusHistory_resultTupleScheme getScheme() {
-        return new getCompactionStatusHistory_resultTupleScheme();
+    private static class getCompactionJob_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public getCompactionJob_resultTupleScheme getScheme() {
+        return new getCompactionJob_resultTupleScheme();
       }
     }
 
-    private static class getCompactionStatusHistory_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme<getCompactionStatusHistory_result> {
+    private static class getCompactionJob_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme<getCompactionJob_result> {
 
       @Override
-      public void write(org.apache.thrift.protocol.TProtocol prot, getCompactionStatusHistory_result struct) throws org.apache.thrift.TException {
+      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()) {
@@ -3742,32 +3519,17 @@ public class CompactionCoordinator {
         }
         oprot.writeBitSet(optionals, 1);
         if (struct.isSetSuccess()) {
-          {
-            oprot.writeI32(struct.success.size());
-            for (Status _iter12 : struct.success)
-            {
-              _iter12.write(oprot);
-            }
-          }
+          struct.success.write(oprot);
         }
       }
 
       @Override
-      public void read(org.apache.thrift.protocol.TProtocol prot, getCompactionStatusHistory_result struct) throws org.apache.thrift.TException {
+      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)) {
-          {
-            org.apache.thrift.protocol.TList _list13 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.success = new java.util.ArrayList<Status>(_list13.size);
-            @org.apache.thrift.annotation.Nullable Status _elem14;
-            for (int _i15 = 0; _i15 < _list13.size; ++_i15)
-            {
-              _elem14 = new Status();
-              _elem14.read(iprot);
-              struct.success.add(_elem14);
-            }
-          }
+          struct.success = new org.apache.accumulo.core.tabletserver.thrift.CompactionJob();
+          struct.success.read(iprot);
           struct.setSuccessIsSet(true);
         }
       }
@@ -3778,22 +3540,22 @@ public class CompactionCoordinator {
     }
   }
 
-  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");
+  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 QUEUE_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("queueName", org.apache.thrift.protocol.TType.STRING, (short)1);
-    private static final org.apache.thrift.protocol.TField COMPACTOR_HOST_AND_PORT_FIELD_DESC = new org.apache.thrift.protocol.TField("compactorHostAndPort", org.apache.thrift.protocol.TType.I64, (short)2);
+    private static final org.apache.thrift.protocol.TField JOB_FIELD_DESC = new org.apache.thrift.protocol.TField("job", org.apache.thrift.protocol.TType.STRUCT, (short)1);
+    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)2);
 
-    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();
+    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 java.lang.String queueName; // required
-    public long compactorHostAndPort; // required
+    public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.tabletserver.thrift.CompactionJob job; // required
+    public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.CompactionStats 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 {
-      QUEUE_NAME((short)1, "queueName"),
-      COMPACTOR_HOST_AND_PORT((short)2, "compactorHostAndPort");
+      JOB((short)1, "job"),
+      STATS((short)2, "stats");
 
       private static final java.util.Map<java.lang.String, _Fields> byName = new java.util.HashMap<java.lang.String, _Fields>();
 
@@ -3809,10 +3571,10 @@ public class CompactionCoordinator {
       @org.apache.thrift.annotation.Nullable
       public static _Fields findByThriftId(int fieldId) {
         switch(fieldId) {
-          case 1: // QUEUE_NAME
-            return QUEUE_NAME;
-          case 2: // COMPACTOR_HOST_AND_PORT
-            return COMPACTOR_HOST_AND_PORT;
+          case 1: // JOB
+            return JOB;
+          case 2: // STATS
+            return STATS;
           default:
             return null;
         }
@@ -3854,117 +3616,116 @@ public class CompactionCoordinator {
     }
 
     // isset id assignments
-    private static final int __COMPACTORHOSTANDPORT_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.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_HOST_AND_PORT, new org.apache.thrift.meta_data.FieldMetaData("compactorHostAndPort", org.apache.thrift.TFieldRequirementType.DEFAULT, 
-          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64)));
+      tmpMap.put(_Fields.JOB, new org.apache.thrift.meta_data.FieldMetaData("job", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.tabletserver.thrift.CompactionJob.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.dataImpl.thrift.CompactionStats.class)));
       metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
-      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(getCompactionJob_args.class, metaDataMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(compactionCompleted_args.class, metaDataMap);
     }
 
-    public getCompactionJob_args() {
+    public compactionCompleted_args() {
     }
 
-    public getCompactionJob_args(
-      java.lang.String queueName,
-      long compactorHostAndPort)
+    public compactionCompleted_args(
+      org.apache.accumulo.core.tabletserver.thrift.CompactionJob job,
+      org.apache.accumulo.core.dataImpl.thrift.CompactionStats stats)
     {
       this();
-      this.queueName = queueName;
-      this.compactorHostAndPort = compactorHostAndPort;
-      setCompactorHostAndPortIsSet(true);
+      this.job = job;
+      this.stats = stats;
     }
 
     /**
      * Performs a deep copy on <i>other</i>.
      */
-    public getCompactionJob_args(getCompactionJob_args other) {
-      __isset_bitfield = other.__isset_bitfield;
-      if (other.isSetQueueName()) {
-        this.queueName = other.queueName;
+    public compactionCompleted_args(compactionCompleted_args other) {
+      if (other.isSetJob()) {
+        this.job = new org.apache.accumulo.core.tabletserver.thrift.CompactionJob(other.job);
+      }
+      if (other.isSetStats()) {
+        this.stats = new org.apache.accumulo.core.dataImpl.thrift.CompactionStats(other.stats);
       }
-      this.compactorHostAndPort = other.compactorHostAndPort;
     }
 
-    public getCompactionJob_args deepCopy() {
-      return new getCompactionJob_args(this);
+    public compactionCompleted_args deepCopy() {
+      return new compactionCompleted_args(this);
     }
 
     @Override
     public void clear() {
-      this.queueName = null;
-      setCompactorHostAndPortIsSet(false);
-      this.compactorHostAndPort = 0;
+      this.job = null;
+      this.stats = null;
     }
 
     @org.apache.thrift.annotation.Nullable
-    public java.lang.String getQueueName() {
-      return this.queueName;
+    public org.apache.accumulo.core.tabletserver.thrift.CompactionJob getJob() {
+      return this.job;
     }
 
-    public getCompactionJob_args setQueueName(@org.apache.thrift.annotation.Nullable java.lang.String queueName) {
-      this.queueName = queueName;
+    public compactionCompleted_args setJob(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.tabletserver.thrift.CompactionJob job) {
+      this.job = job;
       return this;
     }
 
-    public void unsetQueueName() {
-      this.queueName = null;
+    public void unsetJob() {
+      this.job = null;
     }
 
-    /** Returns true if field queueName is set (has been assigned a value) and false otherwise */
-    public boolean isSetQueueName() {
-      return this.queueName != null;
+    /** Returns true if field job is set (has been assigned a value) and false otherwise */
+    public boolean isSetJob() {
+      return this.job != null;
     }
 
-    public void setQueueNameIsSet(boolean value) {
+    public void setJobIsSet(boolean value) {
       if (!value) {
-        this.queueName = null;
+        this.job = null;
       }
     }
 
-    public long getCompactorHostAndPort() {
-      return this.compactorHostAndPort;
+    @org.apache.thrift.annotation.Nullable
+    public org.apache.accumulo.core.dataImpl.thrift.CompactionStats getStats() {
+      return this.stats;
     }
 
-    public getCompactionJob_args setCompactorHostAndPort(long compactorHostAndPort) {
-      this.compactorHostAndPort = compactorHostAndPort;
-      setCompactorHostAndPortIsSet(true);
+    public compactionCompleted_args setStats(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.CompactionStats stats) {
+      this.stats = stats;
       return this;
     }
 
-    public void unsetCompactorHostAndPort() {
-      __isset_bitfield = org.apache.thrift.EncodingUtils.clearBit(__isset_bitfield, __COMPACTORHOSTANDPORT_ISSET_ID);
+    public void unsetStats() {
+      this.stats = null;
     }
 
-    /** Returns true if field compactorHostAndPort is set (has been assigned a value) and false otherwise */
-    public boolean isSetCompactorHostAndPort() {
-      return org.apache.thrift.EncodingUtils.testBit(__isset_bitfield, __COMPACTORHOSTANDPORT_ISSET_ID);
+    /** Returns true if field stats is set (has been assigned a value) and false otherwise */
+    public boolean isSetStats() {
+      return this.stats != null;
     }
 
-    public void setCompactorHostAndPortIsSet(boolean value) {
-      __isset_bitfield = org.apache.thrift.EncodingUtils.setBit(__isset_bitfield, __COMPACTORHOSTANDPORT_ISSET_ID, value);
+    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 QUEUE_NAME:
+      case JOB:
         if (value == null) {
-          unsetQueueName();
+          unsetJob();
         } else {
-          setQueueName((java.lang.String)value);
+          setJob((org.apache.accumulo.core.tabletserver.thrift.CompactionJob)value);
         }
         break;
 
-      case COMPACTOR_HOST_AND_PORT:
+      case STATS:
         if (value == null) {
-          unsetCompactorHostAndPort();
+          unsetStats();
         } else {
-          setCompactorHostAndPort((java.lang.Long)value);
+          setStats((org.apache.accumulo.core.dataImpl.thrift.CompactionStats)value);
         }
         break;
 
@@ -3974,11 +3735,11 @@ public class CompactionCoordinator {
     @org.apache.thrift.annotation.Nullable
     public java.lang.Object getFieldValue(_Fields field) {
       switch (field) {
-      case QUEUE_NAME:
-        return getQueueName();
+      case JOB:
+        return getJob();
 
-      case COMPACTOR_HOST_AND_PORT:
-        return getCompactorHostAndPort();
+      case STATS:
+        return getStats();
 
       }
       throw new java.lang.IllegalStateException();
@@ -3991,10 +3752,10 @@ public class CompactionCoordinator {
       }
 
       switch (field) {
-      case QUEUE_NAME:
-        return isSetQueueName();
-      case COMPACTOR_HOST_AND_PORT:
-        return isSetCompactorHostAndPort();
+      case JOB:
+        return isSetJob();
+      case STATS:
+        return isSetStats();
       }
       throw new java.lang.IllegalStateException();
     }
@@ -4003,32 +3764,32 @@ public class CompactionCoordinator {
     public boolean equals(java.lang.Object that) {
       if (that == null)
         return false;
-      if (that instanceof getCompactionJob_args)
-        return this.equals((getCompactionJob_args)that);
+      if (that instanceof compactionCompleted_args)
+        return this.equals((compactionCompleted_args)that);
       return false;
     }
 
-    public boolean equals(getCompactionJob_args that) {
+    public boolean equals(compactionCompleted_args that) {
       if (that == null)
         return false;
       if (this == that)
         return true;
 
-      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))
+      boolean this_present_job = true && this.isSetJob();
+      boolean that_present_job = true && that.isSetJob();
+      if (this_present_job || that_present_job) {
+        if (!(this_present_job && that_present_job))
           return false;
-        if (!this.queueName.equals(that.queueName))
+        if (!this.job.equals(that.job))
           return false;
       }
 
-      boolean this_present_compactorHostAndPort = true;
-      boolean that_present_compactorHostAndPort = true;
-      if (this_present_compactorHostAndPort || that_present_compactorHostAndPort) {
-        if (!(this_present_compactorHostAndPort && that_present_compactorHostAndPort))
+      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.compactorHostAndPort != that.compactorHostAndPort)
+        if (!this.stats.equals(that.stats))
           return false;
       }
 
@@ -4039,39 +3800,41 @@ public class CompactionCoordinator {
     public int hashCode() {
       int hashCode = 1;
 
-      hashCode = hashCode * 8191 + ((isSetQueueName()) ? 131071 : 524287);
-      if (isSetQueueName())
-        hashCode = hashCode * 8191 + queueName.hashCode();
+      hashCode = hashCode * 8191 + ((isSetJob()) ? 131071 : 524287);
+      if (isSetJob())
+        hashCode = hashCode * 8191 + job.hashCode();
 
-      hashCode = hashCode * 8191 + org.apache.thrift.TBaseHelper.hashCode(compactorHostAndPort);
+      hashCode = hashCode * 8191 + ((isSetStats()) ? 131071 : 524287);
+      if (isSetStats())
+        hashCode = hashCode * 8191 + stats.hashCode();
 
       return hashCode;
     }
 
     @Override
-    public int compareTo(getCompactionJob_args other) {
+    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(isSetQueueName()).compareTo(other.isSetQueueName());
+      lastComparison = java.lang.Boolean.valueOf(isSetJob()).compareTo(other.isSetJob());
       if (lastComparison != 0) {
         return lastComparison;
       }
-      if (isSetQueueName()) {
-        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.queueName, other.queueName);
+      if (isSetJob()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.job, other.job);
         if (lastComparison != 0) {
           return lastComparison;
         }
       }
-      lastComparison = java.lang.Boolean.valueOf(isSetCompactorHostAndPort()).compareTo(other.isSetCompactorHostAndPort());
+      lastComparison = java.lang.Boolean.valueOf(isSetStats()).compareTo(other.isSetStats());
       if (lastComparison != 0) {
         return lastComparison;
       }
-      if (isSetCompactorHostAndPort()) {
-        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.compactorHostAndPort, other.compactorHostAndPort);
+      if (isSetStats()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.stats, other.stats);
         if (lastComparison != 0) {
           return lastComparison;
         }
@@ -4094,19 +3857,23 @@ public class CompactionCoordinator {
 
     @Override
     public java.lang.String toString() {
-      java.lang.StringBuilder sb = new java.lang.StringBuilder("getCompactionJob_args(");
+      java.lang.StringBuilder sb = new java.lang.StringBuilder("compactionCompleted_args(");
       boolean first = true;
 
-      sb.append("queueName:");
-      if (this.queueName == null) {
+      sb.append("job:");
+      if (this.job == null) {
         sb.append("null");
       } else {
-        sb.append(this.queueName);
+        sb.append(this.job);
       }
       first = false;
       if (!first) sb.append(", ");
-      sb.append("compactorHostAndPort:");
-      sb.append(this.compactorHostAndPort);
+      sb.append("stats:");
+      if (this.stats == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.stats);
+      }
       first = false;
       sb.append(")");
       return sb.toString();
@@ -4115,6 +3882,12 @@ public class CompactionCoordinator {
     public void validate() throws org.apache.thrift.TException {
       // check for required fields
       // check for sub-struct validity
+      if (job != null) {
+        job.validate();
+      }
+      if (stats != null) {
+        stats.validate();
+      }
     }
 
     private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
@@ -4127,23 +3900,21 @@ public class CompactionCoordinator {
 
     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 getCompactionJob_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      public getCompactionJob_argsStandardScheme getScheme() {
-        return new getCompactionJob_argsStandardScheme();
+    private static class compactionCompleted_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public compactionCompleted_argsStandardScheme getScheme() {
+        return new compactionCompleted_argsStandardScheme();
       }
     }
 
-    private static class getCompactionJob_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme<getCompactionJob_args> {
+    private static class compactionCompleted_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme<compactionCompleted_args> {
 
-      public void read(org.apache.thrift.protocol.TProtocol iprot, getCompactionJob_args struct) throws org.apache.thrift.TException {
+      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)
@@ -4153,18 +3924,20 @@ public class CompactionCoordinator {
             break;
           }
           switch (schemeField.id) {
-            case 1: // QUEUE_NAME
-              if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
-                struct.queueName = iprot.readString();
-                struct.setQueueNameIsSet(true);
+            case 1: // JOB
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+                struct.job = new org.apache.accumulo.core.tabletserver.thrift.CompactionJob();
+                struct.job.read(iprot);
+                struct.setJobIsSet(true);
               } else { 
                 org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
               }
               break;
-            case 2: // COMPACTOR_HOST_AND_PORT
-              if (schemeField.type == org.apache.thrift.protocol.TType.I64) {
-                struct.compactorHostAndPort = iprot.readI64();
-                struct.setCompactorHostAndPortIsSet(true);
+            case 2: // STATS
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+                struct.stats = new org.apache.accumulo.core.dataImpl.thrift.CompactionStats();
+                struct.stats.read(iprot);
+                struct.setStatsIsSet(true);
               } else { 
                 org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
               }
@@ -4180,62 +3953,66 @@ public class CompactionCoordinator {
         struct.validate();
       }
 
-      public void write(org.apache.thrift.protocol.TProtocol oprot, getCompactionJob_args struct) throws org.apache.thrift.TException {
+      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.queueName != null) {
-          oprot.writeFieldBegin(QUEUE_NAME_FIELD_DESC);
-          oprot.writeString(struct.queueName);
+        if (struct.job != null) {
+          oprot.writeFieldBegin(JOB_FIELD_DESC);
+          struct.job.write(oprot);
+          oprot.writeFieldEnd();
+        }
+        if (struct.stats != null) {
+          oprot.writeFieldBegin(STATS_FIELD_DESC);
+          struct.stats.write(oprot);
           oprot.writeFieldEnd();
         }
-        oprot.writeFieldBegin(COMPACTOR_HOST_AND_PORT_FIELD_DESC);
-        oprot.writeI64(struct.compactorHostAndPort);
-        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 compactionCompleted_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public compactionCompleted_argsTupleScheme getScheme() {
+        return new compactionCompleted_argsTupleScheme();
       }
     }
 
-    private static class getCompactionJob_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme<getCompactionJob_args> {
+    private static class compactionCompleted_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme<compactionCompleted_args> {
 
       @Override
-      public void write(org.apache.thrift.protocol.TProtocol prot, getCompactionJob_args struct) throws org.apache.thrift.TException {
+      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.isSetQueueName()) {
+        if (struct.isSetJob()) {
           optionals.set(0);
         }
-        if (struct.isSetCompactorHostAndPort()) {
+        if (struct.isSetStats()) {
           optionals.set(1);
         }
         oprot.writeBitSet(optionals, 2);
-        if (struct.isSetQueueName()) {
-          oprot.writeString(struct.queueName);
+        if (struct.isSetJob()) {
+          struct.job.write(oprot);
         }
-        if (struct.isSetCompactorHostAndPort()) {
-          oprot.writeI64(struct.compactorHostAndPort);
+        if (struct.isSetStats()) {
+          struct.stats.write(oprot);
         }
       }
 
       @Override
-      public void read(org.apache.thrift.protocol.TProtocol prot, getCompactionJob_args struct) throws org.apache.thrift.TException {
+      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(2);
         if (incoming.get(0)) {
-          struct.queueName = iprot.readString();
-          struct.setQueueNameIsSet(true);
+          struct.job = new org.apache.accumulo.core.tabletserver.thrift.CompactionJob();
+          struct.job.read(iprot);
+          struct.setJobIsSet(true);
         }
         if (incoming.get(1)) {
-          struct.compactorHostAndPort = iprot.readI64();
-          struct.setCompactorHostAndPortIsSet(true);
+          struct.stats = new org.apache.accumulo.core.dataImpl.thrift.CompactionStats();
+          struct.stats.read(iprot);
+          struct.setStatsIsSet(true);
         }
       }
     }
@@ -4245,19 +4022,17 @@ public class CompactionCoordinator {
     }
   }
 
-  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");
+  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 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();
+    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 CompactionJob 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>();
 
@@ -4273,8 +4048,6 @@ public class CompactionCoordinator {
       @org.apache.thrift.annotation.Nullable
       public static _Fields findByThriftId(int fieldId) {
         switch(fieldId) {
-          case 0: // SUCCESS
-            return SUCCESS;
           default:
             return null;
         }
@@ -4314,89 +4087,38 @@ public class CompactionCoordinator {
         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, CompactionJob.class)));
       metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
-      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(getCompactionJob_result.class, metaDataMap);
-    }
-
-    public getCompactionJob_result() {
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(compactionCompleted_result.class, metaDataMap);
     }
 
-    public getCompactionJob_result(
-      CompactionJob success)
-    {
-      this();
-      this.success = success;
+    public compactionCompleted_result() {
     }
 
     /**
      * Performs a deep copy on <i>other</i>.
      */
-    public getCompactionJob_result(getCompactionJob_result other) {
-      if (other.isSetSuccess()) {
-        this.success = new CompactionJob(other.success);
-      }
+    public compactionCompleted_result(compactionCompleted_result other) {
     }
 
-    public getCompactionJob_result deepCopy() {
-      return new getCompactionJob_result(this);
+    public compactionCompleted_result deepCopy() {
+      return new compactionCompleted_result(this);
     }
 
     @Override
     public void clear() {
-      this.success = null;
-    }
-
-    @org.apache.thrift.annotation.Nullable
-    public CompactionJob getSuccess() {
-      return this.success;
-    }
-
-    public getCompactionJob_result setSuccess(@org.apache.thrift.annotation.Nullable CompactionJob 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((CompactionJob)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();
     }
@@ -4408,8 +4130,6 @@ public class CompactionCoordinator {
       }
 
       switch (field) {
-      case SUCCESS:
-        return isSetSuccess();
       }
       throw new java.lang.IllegalStateException();
     }
@@ -4418,26 +4138,17 @@ public class CompactionCoordinator {
     public boolean equals(java.lang.Object that) {
       if (that == null)
         return false;
-      if (that instanceof getCompactionJob_result)
-        return this.equals((getCompactionJob_result)that);
+      if (that instanceof compactionCompleted_result)
+        return this.equals((compactionCompleted_result)that);
       return false;
     }
 
-    public boolean equals(getCompactionJob_result that) {
+    public boolean equals(compactionCompleted_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;
     }
 
@@ -4445,31 +4156,17 @@ public class CompactionCoordinator {
     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) {
+    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(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;
     }
 
@@ -4488,16 +4185,9 @@ public class CompactionCoordinator {
 
     @Override
     public java.lang.String toString() {
-      java.lang.StringBuilder sb = new java.lang.StringBuilder("getCompactionJob_result(");
+      java.lang.StringBuilder sb = new java.lang.StringBuilder("compactionCompleted_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();
     }
@@ -4505,9 +4195,6 @@ public class CompactionCoordinator {
     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 {
@@ -4526,15 +4213,15 @@ public class CompactionCoordinator {
       }
     }
 
-    private static class getCompactionJob_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      public getCompactionJob_resultStandardScheme getScheme() {
-        return new getCompactionJob_resultStandardScheme();
+    private static class compactionCompleted_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public compactionCompleted_resultStandardScheme getScheme() {
+        return new compactionCompleted_resultStandardScheme();
       }
     }
 
-    private static class getCompactionJob_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme<getCompactionJob_result> {
+    private static class compactionCompleted_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme<compactionCompleted_result> {
 
-      public void read(org.apache.thrift.protocol.TProtocol iprot, getCompactionJob_result struct) throws org.apache.thrift.TException {
+      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)
@@ -4544,15 +4231,6 @@ public class CompactionCoordinator {
             break;
           }
           switch (schemeField.id) {
-            case 0: // SUCCESS
-              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
-                struct.success = new CompactionJob();
-                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);
           }
@@ -4564,51 +4242,32 @@ public class CompactionCoordinator {
         struct.validate();
       }
 
-      public void write(org.apache.thrift.protocol.TProtocol oprot, getCompactionJob_result struct) throws org.apache.thrift.TException {
+      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.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 compactionCompleted_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public compactionCompleted_resultTupleScheme getScheme() {
+        return new compactionCompleted_resultTupleScheme();
       }
     }
 
-    private static class getCompactionJob_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme<getCompactionJob_result> {
+    private static class compactionCompleted_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme<compactionCompleted_result> {
 
       @Override
-      public void write(org.apache.thrift.protocol.TProtocol prot, getCompactionJob_result struct) throws org.apache.thrift.TException {
+      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.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 {
+      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.success = new CompactionJob();
-          struct.success.read(iprot);
-          struct.setSuccessIsSet(true);
-        }
       }
     }
 
@@ -4617,18 +4276,18 @@ public class CompactionCoordinator {
     }
   }
 
-  public static class updateCompactionState_args implements org.apache.thrift.TBase<updateCompactionState_args, updateCompactionState_args._Fields>, java.io.Serializable, Cloneable, Comparable<updateCompactionState_args>   {
-    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("updateCompactionState_args");
+  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 COMPACTION_FIELD_DESC = new org.apache.thrift.protocol.TField("compaction", org.apache.thrift.protocol.TType.STRUCT, (short)1);
     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)2);
     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)3);
     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)4);
 
-    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new updateCompactionState_argsStandardSchemeFactory();
-    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new updateCompactionState_argsTupleSchemeFactory();
+    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 CompactionJob compaction; // required
+    public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.tabletserver.thrift.CompactionJob compaction; // required
     /**
      * 
      * @see CompactionState
@@ -4717,7 +4376,7 @@ public class CompactionCoordinator {
     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.COMPACTION, new org.apache.thrift.meta_data.FieldMetaData("compaction", org.apache.thrift.TFieldRequirementType.DEFAULT, 
-          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, CompactionJob.class)));
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.tabletserver.thrift.CompactionJob.class)));
       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, CompactionState.class)));
       tmpMap.put(_Fields.MESSAGE, new org.apache.thrift.meta_data.FieldMetaData("message", org.apache.thrift.TFieldRequirementType.DEFAULT, 
@@ -4725,14 +4384,14 @@ public class CompactionCoordinator {
       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(updateCompactionState_args.class, metaDataMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(updateCompactionStatus_args.class, metaDataMap);
     }
 
-    public updateCompactionState_args() {
+    public updateCompactionStatus_args() {
     }
 
-    public updateCompactionState_args(
-      CompactionJob compaction,
+    public updateCompactionStatus_args(
+      org.apache.accumulo.core.tabletserver.thrift.CompactionJob compaction,
       CompactionState state,
       java.lang.String message,
       long timestamp)
@@ -4748,10 +4407,10 @@ public class CompactionCoordinator {
     /**
      * Performs a deep copy on <i>other</i>.
      */
-    public updateCompactionState_args(updateCompactionState_args other) {
+    public updateCompactionStatus_args(updateCompactionStatus_args other) {
       __isset_bitfield = other.__isset_bitfield;
       if (other.isSetCompaction()) {
-        this.compaction = new CompactionJob(other.compaction);
+        this.compaction = new org.apache.accumulo.core.tabletserver.thrift.CompactionJob(other.compaction);
       }
       if (other.isSetState()) {
         this.state = other.state;
@@ -4762,8 +4421,8 @@ public class CompactionCoordinator {
       this.timestamp = other.timestamp;
     }
 
-    public updateCompactionState_args deepCopy() {
-      return new updateCompactionState_args(this);
+    public updateCompactionStatus_args deepCopy() {
+      return new updateCompactionStatus_args(this);
     }
 
     @Override
@@ -4776,11 +4435,11 @@ public class CompactionCoordinator {
     }
 
     @org.apache.thrift.annotation.Nullable
-    public CompactionJob getCompaction() {
+    public org.apache.accumulo.core.tabletserver.thrift.CompactionJob getCompaction() {
       return this.compaction;
     }
 
-    public updateCompactionState_args setCompaction(@org.apache.thrift.annotation.Nullable CompactionJob compaction) {
+    public updateCompactionStatus_args setCompaction(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.tabletserver.thrift.CompactionJob compaction) {
       this.compaction = compaction;
       return this;
     }
@@ -4813,7 +4472,7 @@ public class CompactionCoordinator {
      * 
      * @see CompactionState
      */
-    public updateCompactionState_args setState(@org.apache.thrift.annotation.Nullable CompactionState state) {
+    public updateCompactionStatus_args setState(@org.apache.thrift.annotation.Nullable CompactionState state) {
       this.state = state;
       return this;
     }
@@ -4838,7 +4497,7 @@ public class CompactionCoordinator {
       return this.message;
     }
 
-    public updateCompactionState_args setMessage(@org.apache.thrift.annotation.Nullable java.lang.String message) {
+    public updateCompactionStatus_args setMessage(@org.apache.thrift.annotation.Nullable java.lang.String message) {
       this.message = message;
       return this;
     }
@@ -4862,7 +4521,7 @@ public class CompactionCoordinator {
       return this.timestamp;
     }
 
-    public updateCompactionState_args setTimestamp(long timestamp) {
+    public updateCompactionStatus_args setTimestamp(long timestamp) {
       this.timestamp = timestamp;
       setTimestampIsSet(true);
       return this;
@@ -4887,7 +4546,7 @@ public class CompactionCoordinator {
         if (value == null) {
           unsetCompaction();
         } else {
-          setCompaction((CompactionJob)value);
+          setCompaction((org.apache.accumulo.core.tabletserver.thrift.CompactionJob)value);
         }
         break;
 
@@ -4960,12 +4619,12 @@ public class CompactionCoordinator {
     public boolean equals(java.lang.Object that) {
       if (that == null)
         return false;
-      if (that instanceof updateCompactionState_args)
-        return this.equals((updateCompactionState_args)that);
+      if (that instanceof updateCompactionStatus_args)
+        return this.equals((updateCompactionStatus_args)that);
       return false;
     }
 
-    public boolean equals(updateCompactionState_args that) {
+    public boolean equals(updateCompactionStatus_args that) {
       if (that == null)
         return false;
       if (this == that)
@@ -5032,7 +4691,7 @@ public class CompactionCoordinator {
     }
 
     @Override
-    public int compareTo(updateCompactionState_args other) {
+    public int compareTo(updateCompactionStatus_args other) {
       if (!getClass().equals(other.getClass())) {
         return getClass().getName().compareTo(other.getClass().getName());
       }
@@ -5097,7 +4756,7 @@ public class CompactionCoordinator {
 
     @Override
     public java.lang.String toString() {
-      java.lang.StringBuilder sb = new java.lang.StringBuilder("updateCompactionState_args(");
+      java.lang.StringBuilder sb = new java.lang.StringBuilder("updateCompactionStatus_args(");
       boolean first = true;
 
       sb.append("compaction:");
@@ -5157,15 +4816,15 @@ public class CompactionCoordinator {
       }
     }
 
-    private static class updateCompactionState_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      public updateCompactionState_argsStandardScheme getScheme() {
-        return new updateCompactionState_argsStandardScheme();
+    private static class updateCompactionStatus_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public updateCompactionStatus_argsStandardScheme getScheme() {
+        return new updateCompactionStatus_argsStandardScheme();
       }
     }
 
-    private static class updateCompactionState_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme<updateCompactionState_args> {
+    private static class updateCompactionStatus_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme<updateCompactionStatus_args> {
 
-      public void read(org.apache.thrift.protocol.TProtocol iprot, updateCompactionState_args struct) throws org.apache.thrift.TException {
+      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)
@@ -5177,7 +4836,7 @@ public class CompactionCoordinator {
           switch (schemeField.id) {
             case 1: // COMPACTION
               if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
-                struct.compaction = new CompactionJob();
+                struct.compaction = new org.apache.accumulo.core.tabletserver.thrift.CompactionJob();
                 struct.compaction.read(iprot);
                 struct.setCompactionIsSet(true);
               } else { 
@@ -5219,7 +4878,7 @@ public class CompactionCoordinator {
         struct.validate();
       }
 
-      public void write(org.apache.thrift.protocol.TProtocol oprot, updateCompactionState_args struct) throws org.apache.thrift.TException {
+      public void write(org.apache.thrift.protocol.TProtocol oprot, updateCompactionStatus_args struct) throws org.apache.thrift.TException {
         struct.validate();
 
         oprot.writeStructBegin(STRUCT_DESC);
@@ -5247,16 +4906,16 @@ public class CompactionCoordinator {
 
     }
 
-    private static class updateCompactionState_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      public updateCompactionState_argsTupleScheme getScheme() {
-        return new updateCompactionState_argsTupleScheme();
+    private static class updateCompactionStatus_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public updateCompactionStatus_argsTupleScheme getScheme() {
+        return new updateCompactionStatus_argsTupleScheme();
       }
     }
 
-    private static class updateCompactionState_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme<updateCompactionState_args> {
+    private static class updateCompactionStatus_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme<updateCompactionStatus_args> {
 
       @Override
-      public void write(org.apache.thrift.protocol.TProtocol prot, updateCompactionState_args struct) throws org.apache.thrift.TException {
+      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.isSetCompaction()) {
@@ -5287,11 +4946,11 @@ public class CompactionCoordinator {
       }
 
       @Override
-      public void read(org.apache.thrift.protocol.TProtocol prot, updateCompactionState_args struct) throws org.apache.thrift.TException {
+      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(4);
         if (incoming.get(0)) {
-          struct.compaction = new CompactionJob();
+          struct.compaction = new org.apache.accumulo.core.tabletserver.thrift.CompactionJob();
           struct.compaction.read(iprot);
           struct.setCompactionIsSet(true);
         }
@@ -5315,12 +4974,12 @@ public class CompactionCoordinator {
     }
   }
 
-  public static class updateCompactionState_result implements org.apache.thrift.TBase<updateCompactionState_result, updateCompactionState_result._Fields>, java.io.Serializable, Cloneable, Comparable<updateCompactionState_result>   {
-    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("updateCompactionState_result");
+  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.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new updateCompactionState_resultStandardSchemeFactory();
-    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new updateCompactionState_resultTupleSchemeFactory();
+    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();
 
 
     /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
@@ -5384,20 +5043,20 @@ public class CompactionCoordinator {
     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);
       metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
-      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(updateCompactionState_result.class, metaDataMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(updateCompactionStatus_result.class, metaDataMap);
     }
 
-    public updateCompactionState_result() {
+    public updateCompactionStatus_result() {
     }
 
     /**
      * Performs a deep copy on <i>other</i>.
      */
-    public updateCompactionState_result(updateCompactionState_result other) {
+    public updateCompactionStatus_result(updateCompactionStatus_result other) {
     }
 
-    public updateCompactionState_result deepCopy() {
-      return new updateCompactionState_result(this);
+    public updateCompactionStatus_result deepCopy() {
+      return new updateCompactionStatus_result(this);
     }
 
     @Override
@@ -5431,12 +5090,12 @@ public class CompactionCoordinator {
     public boolean equals(java.lang.Object that) {
       if (that == null)
         return false;
-      if (that instanceof updateCompactionState_result)
-        return this.equals((updateCompactionState_result)that);
+      if (that instanceof updateCompactionStatus_result)
+        return this.equals((updateCompactionStatus_result)that);
       return false;
     }
 
-    public boolean equals(updateCompactionState_result that) {
+    public boolean equals(updateCompactionStatus_result that) {
       if (that == null)
         return false;
       if (this == that)
@@ -5453,7 +5112,7 @@ public class CompactionCoordinator {
     }
 
     @Override
-    public int compareTo(updateCompactionState_result other) {
+    public int compareTo(updateCompactionStatus_result other) {
       if (!getClass().equals(other.getClass())) {
         return getClass().getName().compareTo(other.getClass().getName());
       }
@@ -5478,7 +5137,7 @@ public class CompactionCoordinator {
 
     @Override
     public java.lang.String toString() {
-      java.lang.StringBuilder sb = new java.lang.StringBuilder("updateCompactionState_result(");
+      java.lang.StringBuilder sb = new java.lang.StringBuilder("updateCompactionStatus_result(");
       boolean first = true;
 
       sb.append(")");
@@ -5506,15 +5165,15 @@ public class CompactionCoordinator {
       }
     }
 
-    private static class updateCompactionState_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      public updateCompactionState_resultStandardScheme getScheme() {
-        return new updateCompactionState_resultStandardScheme();
+    private static class updateCompactionStatus_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public updateCompactionStatus_resultStandardScheme getScheme() {
+        return new updateCompactionStatus_resultStandardScheme();
       }
     }
 
-    private static class updateCompactionState_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme<updateCompactionState_result> {
+    private static class updateCompactionStatus_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme<updateCompactionStatus_result> {
 
-      public void read(org.apache.thrift.protocol.TProtocol iprot, updateCompactionState_result struct) throws org.apache.thrift.TException {
+      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)
@@ -5535,7 +5194,7 @@ public class CompactionCoordinator {
         struct.validate();
       }
 
-      public void write(org.apache.thrift.protocol.TProtocol oprot, updateCompactionState_result struct) throws org.apache.thrift.TException {
+      public void write(org.apache.thrift.protocol.TProtocol oprot, updateCompactionStatus_result struct) throws org.apache.thrift.TException {
         struct.validate();
 
         oprot.writeStructBegin(STRUCT_DESC);
@@ -5545,21 +5204,21 @@ public class CompactionCoordinator {
 
     }
 
-    private static class updateCompactionState_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      public updateCompactionState_resultTupleScheme getScheme() {
-        return new updateCompactionState_resultTupleScheme();
+    private static class updateCompactionStatus_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public updateCompactionStatus_resultTupleScheme getScheme() {
+        return new updateCompactionStatus_resultTupleScheme();
       }
     }
 
-    private static class updateCompactionState_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme<updateCompactionState_result> {
+    private static class updateCompactionStatus_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme<updateCompactionStatus_result> {
 
       @Override
-      public void write(org.apache.thrift.protocol.TProtocol prot, updateCompactionState_result struct) throws org.apache.thrift.TException {
+      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;
       }
 
       @Override
-      public void read(org.apache.thrift.protocol.TProtocol prot, updateCompactionState_result struct) throws org.apache.thrift.TException {
+      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;
       }
     }
diff --git a/core/src/main/java/org/apache/accumulo/core/compaction/thrift/CompactionStats.java b/core/src/main/java/org/apache/accumulo/core/compaction/thrift/CompactionStats.java
new file mode 100644
index 0000000..b9f9597
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/compaction/thrift/CompactionStats.java
@@ -0,0 +1,582 @@
+/*
+ * 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 CompactionStats implements org.apache.thrift.TBase<CompactionStats, CompactionStats._Fields>, java.io.Serializable, Cloneable, Comparable<CompactionStats> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("CompactionStats");
+
+  private static final org.apache.thrift.protocol.TField ENTRIES_READ_FIELD_DESC = new org.apache.thrift.protocol.TField("entriesRead", org.apache.thrift.protocol.TType.I64, (short)1);
+  private static final org.apache.thrift.protocol.TField ENTRIES_WRITTEN_FIELD_DESC = new org.apache.thrift.protocol.TField("entriesWritten", org.apache.thrift.protocol.TType.I64, (short)2);
+  private static final org.apache.thrift.protocol.TField FILE_SIZE_FIELD_DESC = new org.apache.thrift.protocol.TField("fileSize", org.apache.thrift.protocol.TType.I64, (short)3);
+
+  private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new CompactionStatsStandardSchemeFactory();
+  private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new CompactionStatsTupleSchemeFactory();
+
+  public long entriesRead; // required
+  public long entriesWritten; // required
+  public long fileSize; // 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 {
+    ENTRIES_READ((short)1, "entriesRead"),
+    ENTRIES_WRITTEN((short)2, "entriesWritten"),
+    FILE_SIZE((short)3, "fileSize");
+
+    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: // ENTRIES_READ
+          return ENTRIES_READ;
+        case 2: // ENTRIES_WRITTEN
+          return ENTRIES_WRITTEN;
+        case 3: // FILE_SIZE
+          return FILE_SIZE;
+        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 __ENTRIESREAD_ISSET_ID = 0;
+  private static final int __ENTRIESWRITTEN_ISSET_ID = 1;
+  private static final int __FILESIZE_ISSET_ID = 2;
+  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.ENTRIES_READ, new org.apache.thrift.meta_data.FieldMetaData("entriesRead", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64)));
+    tmpMap.put(_Fields.ENTRIES_WRITTEN, new org.apache.thrift.meta_data.FieldMetaData("entriesWritten", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64)));
+    tmpMap.put(_Fields.FILE_SIZE, new org.apache.thrift.meta_data.FieldMetaData("fileSize", 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(CompactionStats.class, metaDataMap);
+  }
+
+  public CompactionStats() {
+  }
+
+  public CompactionStats(
+    long entriesRead,
+    long entriesWritten,
+    long fileSize)
+  {
+    this();
+    this.entriesRead = entriesRead;
+    setEntriesReadIsSet(true);
+    this.entriesWritten = entriesWritten;
+    setEntriesWrittenIsSet(true);
+    this.fileSize = fileSize;
+    setFileSizeIsSet(true);
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public CompactionStats(CompactionStats other) {
+    __isset_bitfield = other.__isset_bitfield;
+    this.entriesRead = other.entriesRead;
+    this.entriesWritten = other.entriesWritten;
+    this.fileSize = other.fileSize;
+  }
+
+  public CompactionStats deepCopy() {
+    return new CompactionStats(this);
+  }
+
+  @Override
+  public void clear() {
+    setEntriesReadIsSet(false);
+    this.entriesRead = 0;
+    setEntriesWrittenIsSet(false);
+    this.entriesWritten = 0;
+    setFileSizeIsSet(false);
+    this.fileSize = 0;
+  }
+
+  public long getEntriesRead() {
+    return this.entriesRead;
+  }
+
+  public CompactionStats setEntriesRead(long entriesRead) {
+    this.entriesRead = entriesRead;
+    setEntriesReadIsSet(true);
+    return this;
+  }
+
+  public void unsetEntriesRead() {
+    __isset_bitfield = org.apache.thrift.EncodingUtils.clearBit(__isset_bitfield, __ENTRIESREAD_ISSET_ID);
+  }
+
+  /** Returns true if field entriesRead is set (has been assigned a value) and false otherwise */
+  public boolean isSetEntriesRead() {
+    return org.apache.thrift.EncodingUtils.testBit(__isset_bitfield, __ENTRIESREAD_ISSET_ID);
+  }
+
+  public void setEntriesReadIsSet(boolean value) {
+    __isset_bitfield = org.apache.thrift.EncodingUtils.setBit(__isset_bitfield, __ENTRIESREAD_ISSET_ID, value);
+  }
+
+  public long getEntriesWritten() {
+    return this.entriesWritten;
+  }
+
+  public CompactionStats setEntriesWritten(long entriesWritten) {
+    this.entriesWritten = entriesWritten;
+    setEntriesWrittenIsSet(true);
+    return this;
+  }
+
+  public void unsetEntriesWritten() {
+    __isset_bitfield = org.apache.thrift.EncodingUtils.clearBit(__isset_bitfield, __ENTRIESWRITTEN_ISSET_ID);
+  }
+
+  /** Returns true if field entriesWritten is set (has been assigned a value) and false otherwise */
+  public boolean isSetEntriesWritten() {
+    return org.apache.thrift.EncodingUtils.testBit(__isset_bitfield, __ENTRIESWRITTEN_ISSET_ID);
+  }
+
+  public void setEntriesWrittenIsSet(boolean value) {
+    __isset_bitfield = org.apache.thrift.EncodingUtils.setBit(__isset_bitfield, __ENTRIESWRITTEN_ISSET_ID, value);
+  }
+
+  public long getFileSize() {
+    return this.fileSize;
+  }
+
+  public CompactionStats setFileSize(long fileSize) {
+    this.fileSize = fileSize;
+    setFileSizeIsSet(true);
+    return this;
+  }
+
+  public void unsetFileSize() {
+    __isset_bitfield = org.apache.thrift.EncodingUtils.clearBit(__isset_bitfield, __FILESIZE_ISSET_ID);
+  }
+
+  /** Returns true if field fileSize is set (has been assigned a value) and false otherwise */
+  public boolean isSetFileSize() {
+    return org.apache.thrift.EncodingUtils.testBit(__isset_bitfield, __FILESIZE_ISSET_ID);
+  }
+
+  public void setFileSizeIsSet(boolean value) {
+    __isset_bitfield = org.apache.thrift.EncodingUtils.setBit(__isset_bitfield, __FILESIZE_ISSET_ID, value);
+  }
+
+  public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable java.lang.Object value) {
+    switch (field) {
+    case ENTRIES_READ:
+      if (value == null) {
+        unsetEntriesRead();
+      } else {
+        setEntriesRead((java.lang.Long)value);
+      }
+      break;
+
+    case ENTRIES_WRITTEN:
+      if (value == null) {
+        unsetEntriesWritten();
+      } else {
+        setEntriesWritten((java.lang.Long)value);
+      }
+      break;
+
+    case FILE_SIZE:
+      if (value == null) {
+        unsetFileSize();
+      } else {
+        setFileSize((java.lang.Long)value);
+      }
+      break;
+
+    }
+  }
+
+  @org.apache.thrift.annotation.Nullable
+  public java.lang.Object getFieldValue(_Fields field) {
+    switch (field) {
+    case ENTRIES_READ:
+      return getEntriesRead();
+
+    case ENTRIES_WRITTEN:
+      return getEntriesWritten();
+
+    case FILE_SIZE:
+      return getFileSize();
+
+    }
+    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 ENTRIES_READ:
+      return isSetEntriesRead();
+    case ENTRIES_WRITTEN:
+      return isSetEntriesWritten();
+    case FILE_SIZE:
+      return isSetFileSize();
+    }
+    throw new java.lang.IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(java.lang.Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof CompactionStats)
+      return this.equals((CompactionStats)that);
+    return false;
+  }
+
+  public boolean equals(CompactionStats that) {
+    if (that == null)
+      return false;
+    if (this == that)
+      return true;
+
+    boolean this_present_entriesRead = true;
+    boolean that_present_entriesRead = true;
+    if (this_present_entriesRead || that_present_entriesRead) {
+      if (!(this_present_entriesRead && that_present_entriesRead))
+        return false;
+      if (this.entriesRead != that.entriesRead)
+        return false;
+    }
+
+    boolean this_present_entriesWritten = true;
+    boolean that_present_entriesWritten = true;
+    if (this_present_entriesWritten || that_present_entriesWritten) {
+      if (!(this_present_entriesWritten && that_present_entriesWritten))
+        return false;
+      if (this.entriesWritten != that.entriesWritten)
+        return false;
+    }
+
+    boolean this_present_fileSize = true;
+    boolean that_present_fileSize = true;
+    if (this_present_fileSize || that_present_fileSize) {
+      if (!(this_present_fileSize && that_present_fileSize))
+        return false;
+      if (this.fileSize != that.fileSize)
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    int hashCode = 1;
+
+    hashCode = hashCode * 8191 + org.apache.thrift.TBaseHelper.hashCode(entriesRead);
+
+    hashCode = hashCode * 8191 + org.apache.thrift.TBaseHelper.hashCode(entriesWritten);
+
+    hashCode = hashCode * 8191 + org.apache.thrift.TBaseHelper.hashCode(fileSize);
+
+    return hashCode;
+  }
+
+  @Override
+  public int compareTo(CompactionStats other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+
+    lastComparison = java.lang.Boolean.valueOf(isSetEntriesRead()).compareTo(other.isSetEntriesRead());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetEntriesRead()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.entriesRead, other.entriesRead);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = java.lang.Boolean.valueOf(isSetEntriesWritten()).compareTo(other.isSetEntriesWritten());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetEntriesWritten()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.entriesWritten, other.entriesWritten);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = java.lang.Boolean.valueOf(isSetFileSize()).compareTo(other.isSetFileSize());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetFileSize()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.fileSize, other.fileSize);
+      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("CompactionStats(");
+    boolean first = true;
+
+    sb.append("entriesRead:");
+    sb.append(this.entriesRead);
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("entriesWritten:");
+    sb.append(this.entriesWritten);
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("fileSize:");
+    sb.append(this.fileSize);
+    first = false;
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    // check for sub-struct validity
+  }
+
+  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+    try {
+      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, java.lang.ClassNotFoundException {
+    try {
+      // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor.
+      __isset_bitfield = 0;
+      read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private static class CompactionStatsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+    public CompactionStatsStandardScheme getScheme() {
+      return new CompactionStatsStandardScheme();
+    }
+  }
+
+  private static class CompactionStatsStandardScheme extends org.apache.thrift.scheme.StandardScheme<CompactionStats> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, CompactionStats 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: // ENTRIES_READ
+            if (schemeField.type == org.apache.thrift.protocol.TType.I64) {
+              struct.entriesRead = iprot.readI64();
+              struct.setEntriesReadIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 2: // ENTRIES_WRITTEN
+            if (schemeField.type == org.apache.thrift.protocol.TType.I64) {
+              struct.entriesWritten = iprot.readI64();
+              struct.setEntriesWrittenIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 3: // FILE_SIZE
+            if (schemeField.type == org.apache.thrift.protocol.TType.I64) {
+              struct.fileSize = iprot.readI64();
+              struct.setFileSizeIsSet(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, CompactionStats struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      oprot.writeFieldBegin(ENTRIES_READ_FIELD_DESC);
+      oprot.writeI64(struct.entriesRead);
+      oprot.writeFieldEnd();
+      oprot.writeFieldBegin(ENTRIES_WRITTEN_FIELD_DESC);
+      oprot.writeI64(struct.entriesWritten);
+      oprot.writeFieldEnd();
+      oprot.writeFieldBegin(FILE_SIZE_FIELD_DESC);
+      oprot.writeI64(struct.fileSize);
+      oprot.writeFieldEnd();
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class CompactionStatsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+    public CompactionStatsTupleScheme getScheme() {
+      return new CompactionStatsTupleScheme();
+    }
+  }
+
+  private static class CompactionStatsTupleScheme extends org.apache.thrift.scheme.TupleScheme<CompactionStats> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, CompactionStats 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.isSetEntriesRead()) {
+        optionals.set(0);
+      }
+      if (struct.isSetEntriesWritten()) {
+        optionals.set(1);
+      }
+      if (struct.isSetFileSize()) {
+        optionals.set(2);
+      }
+      oprot.writeBitSet(optionals, 3);
+      if (struct.isSetEntriesRead()) {
+        oprot.writeI64(struct.entriesRead);
+      }
+      if (struct.isSetEntriesWritten()) {
+        oprot.writeI64(struct.entriesWritten);
+      }
+      if (struct.isSetFileSize()) {
+        oprot.writeI64(struct.fileSize);
+      }
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, CompactionStats 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.entriesRead = iprot.readI64();
+        struct.setEntriesReadIsSet(true);
+      }
+      if (incoming.get(1)) {
+        struct.entriesWritten = iprot.readI64();
+        struct.setEntriesWrittenIsSet(true);
+      }
+      if (incoming.get(2)) {
+        struct.fileSize = iprot.readI64();
+        struct.setFileSizeIsSet(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/java/org/apache/accumulo/core/compaction/thrift/Compactor.java b/core/src/main/java/org/apache/accumulo/core/compaction/thrift/Compactor.java
index 293d39e..7738b55 100644
--- a/core/src/main/java/org/apache/accumulo/core/compaction/thrift/Compactor.java
+++ b/core/src/main/java/org/apache/accumulo/core/compaction/thrift/Compactor.java
@@ -29,13 +29,13 @@ public class Compactor {
 
   public interface Iface {
 
-    public void cancel(CompactionJob compaction) throws org.apache.thrift.TException;
+    public void cancel(org.apache.accumulo.core.tabletserver.thrift.CompactionJob compaction) throws org.apache.thrift.TException;
 
   }
 
   public interface AsyncIface {
 
-    public void cancel(CompactionJob compaction, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
+    public void cancel(org.apache.accumulo.core.tabletserver.thrift.CompactionJob compaction, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
 
   }
 
@@ -59,13 +59,13 @@ public class Compactor {
       super(iprot, oprot);
     }
 
-    public void cancel(CompactionJob compaction) throws org.apache.thrift.TException
+    public void cancel(org.apache.accumulo.core.tabletserver.thrift.CompactionJob compaction) throws org.apache.thrift.TException
     {
       send_cancel(compaction);
       recv_cancel();
     }
 
-    public void send_cancel(CompactionJob compaction) throws org.apache.thrift.TException
+    public void send_cancel(org.apache.accumulo.core.tabletserver.thrift.CompactionJob compaction) throws org.apache.thrift.TException
     {
       cancel_args args = new cancel_args();
       args.setCompaction(compaction);
@@ -97,7 +97,7 @@ public class Compactor {
       super(protocolFactory, clientManager, transport);
     }
 
-    public void cancel(CompactionJob compaction, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
+    public void cancel(org.apache.accumulo.core.tabletserver.thrift.CompactionJob compaction, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
       checkReady();
       cancel_call method_call = new cancel_call(compaction, resultHandler, this, ___protocolFactory, ___transport);
       this.___currentMethod = method_call;
@@ -105,8 +105,8 @@ public class Compactor {
     }
 
     public static class cancel_call extends org.apache.thrift.async.TAsyncMethodCall<Void> {
-      private CompactionJob compaction;
-      public cancel_call(CompactionJob compaction, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+      private org.apache.accumulo.core.tabletserver.thrift.CompactionJob compaction;
+      public cancel_call(org.apache.accumulo.core.tabletserver.thrift.CompactionJob compaction, 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.compaction = compaction;
       }
@@ -258,7 +258,7 @@ public class Compactor {
     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 CompactionJob compaction; // required
+    public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.tabletserver.thrift.CompactionJob compaction; // 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 {
@@ -325,7 +325,7 @@ public class Compactor {
     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.COMPACTION, new org.apache.thrift.meta_data.FieldMetaData("compaction", org.apache.thrift.TFieldRequirementType.DEFAULT, 
-          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, CompactionJob.class)));
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.tabletserver.thrift.CompactionJob.class)));
       metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
       org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(cancel_args.class, metaDataMap);
     }
@@ -334,7 +334,7 @@ public class Compactor {
     }
 
     public cancel_args(
-      CompactionJob compaction)
+      org.apache.accumulo.core.tabletserver.thrift.CompactionJob compaction)
     {
       this();
       this.compaction = compaction;
@@ -345,7 +345,7 @@ public class Compactor {
      */
     public cancel_args(cancel_args other) {
       if (other.isSetCompaction()) {
-        this.compaction = new CompactionJob(other.compaction);
+        this.compaction = new org.apache.accumulo.core.tabletserver.thrift.CompactionJob(other.compaction);
       }
     }
 
@@ -359,11 +359,11 @@ public class Compactor {
     }
 
     @org.apache.thrift.annotation.Nullable
-    public CompactionJob getCompaction() {
+    public org.apache.accumulo.core.tabletserver.thrift.CompactionJob getCompaction() {
       return this.compaction;
     }
 
-    public cancel_args setCompaction(@org.apache.thrift.annotation.Nullable CompactionJob compaction) {
+    public cancel_args setCompaction(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.tabletserver.thrift.CompactionJob compaction) {
       this.compaction = compaction;
       return this;
     }
@@ -389,7 +389,7 @@ public class Compactor {
         if (value == null) {
           unsetCompaction();
         } else {
-          setCompaction((CompactionJob)value);
+          setCompaction((org.apache.accumulo.core.tabletserver.thrift.CompactionJob)value);
         }
         break;
 
@@ -551,7 +551,7 @@ public class Compactor {
           switch (schemeField.id) {
             case 1: // COMPACTION
               if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
-                struct.compaction = new CompactionJob();
+                struct.compaction = new org.apache.accumulo.core.tabletserver.thrift.CompactionJob();
                 struct.compaction.read(iprot);
                 struct.setCompactionIsSet(true);
               } else { 
@@ -610,7 +610,7 @@ public class Compactor {
         org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
         java.util.BitSet incoming = iprot.readBitSet(1);
         if (incoming.get(0)) {
-          struct.compaction = new CompactionJob();
+          struct.compaction = new org.apache.accumulo.core.tabletserver.thrift.CompactionJob();
           struct.compaction.read(iprot);
           struct.setCompactionIsSet(true);
         }
diff --git a/core/src/main/java/org/apache/accumulo/core/compaction/thrift/Status.java b/core/src/main/java/org/apache/accumulo/core/compaction/thrift/Status.java
index 1f8135a..88779f5 100644
--- a/core/src/main/java/org/apache/accumulo/core/compaction/thrift/Status.java
+++ b/core/src/main/java/org/apache/accumulo/core/compaction/thrift/Status.java
@@ -30,7 +30,7 @@ public class Status implements org.apache.thrift.TBase<Status, Status._Fields>,
 
   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)1);
   private static final org.apache.thrift.protocol.TField COMPACTION_ID_FIELD_DESC = new org.apache.thrift.protocol.TField("compactionId", org.apache.thrift.protocol.TType.I64, (short)2);
-  private static final org.apache.thrift.protocol.TField COMPACTOR_HOST_AND_PORT_FIELD_DESC = new org.apache.thrift.protocol.TField("compactorHostAndPort", 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)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);
 
@@ -39,7 +39,7 @@ public class Status implements org.apache.thrift.TBase<Status, Status._Fields>,
 
   public long timestamp; // required
   public long compactionId; // required
-  public @org.apache.thrift.annotation.Nullable java.lang.String compactorHostAndPort; // required
+  public @org.apache.thrift.annotation.Nullable java.lang.String compactor; // required
   /**
    * 
    * @see CompactionState
@@ -51,7 +51,7 @@ public class Status implements org.apache.thrift.TBase<Status, Status._Fields>,
   public enum _Fields implements org.apache.thrift.TFieldIdEnum {
     TIMESTAMP((short)1, "timestamp"),
     COMPACTION_ID((short)2, "compactionId"),
-    COMPACTOR_HOST_AND_PORT((short)3, "compactorHostAndPort"),
+    COMPACTOR((short)3, "compactor"),
     /**
      * 
      * @see CompactionState
@@ -77,8 +77,8 @@ public class Status implements org.apache.thrift.TBase<Status, Status._Fields>,
           return TIMESTAMP;
         case 2: // COMPACTION_ID
           return COMPACTION_ID;
-        case 3: // COMPACTOR_HOST_AND_PORT
-          return COMPACTOR_HOST_AND_PORT;
+        case 3: // COMPACTOR
+          return COMPACTOR;
         case 4: // STATE
           return STATE;
         case 5: // MESSAGE
@@ -134,7 +134,7 @@ public class Status implements org.apache.thrift.TBase<Status, Status._Fields>,
         new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64)));
     tmpMap.put(_Fields.COMPACTION_ID, new org.apache.thrift.meta_data.FieldMetaData("compactionId", org.apache.thrift.TFieldRequirementType.DEFAULT, 
         new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64)));
-    tmpMap.put(_Fields.COMPACTOR_HOST_AND_PORT, new org.apache.thrift.meta_data.FieldMetaData("compactorHostAndPort", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+    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.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, CompactionState.class)));
@@ -150,7 +150,7 @@ public class Status implements org.apache.thrift.TBase<Status, Status._Fields>,
   public Status(
     long timestamp,
     long compactionId,
-    java.lang.String compactorHostAndPort,
+    java.lang.String compactor,
     CompactionState state,
     java.lang.String message)
   {
@@ -159,7 +159,7 @@ public class Status implements org.apache.thrift.TBase<Status, Status._Fields>,
     setTimestampIsSet(true);
     this.compactionId = compactionId;
     setCompactionIdIsSet(true);
-    this.compactorHostAndPort = compactorHostAndPort;
+    this.compactor = compactor;
     this.state = state;
     this.message = message;
   }
@@ -171,8 +171,8 @@ public class Status implements org.apache.thrift.TBase<Status, Status._Fields>,
     __isset_bitfield = other.__isset_bitfield;
     this.timestamp = other.timestamp;
     this.compactionId = other.compactionId;
-    if (other.isSetCompactorHostAndPort()) {
-      this.compactorHostAndPort = other.compactorHostAndPort;
+    if (other.isSetCompactor()) {
+      this.compactor = other.compactor;
     }
     if (other.isSetState()) {
       this.state = other.state;
@@ -192,7 +192,7 @@ public class Status implements org.apache.thrift.TBase<Status, Status._Fields>,
     this.timestamp = 0;
     setCompactionIdIsSet(false);
     this.compactionId = 0;
-    this.compactorHostAndPort = null;
+    this.compactor = null;
     this.state = null;
     this.message = null;
   }
@@ -244,27 +244,27 @@ public class Status implements org.apache.thrift.TBase<Status, Status._Fields>,
   }
 
   @org.apache.thrift.annotation.Nullable
-  public java.lang.String getCompactorHostAndPort() {
-    return this.compactorHostAndPort;
+  public java.lang.String getCompactor() {
+    return this.compactor;
   }
 
-  public Status setCompactorHostAndPort(@org.apache.thrift.annotation.Nullable java.lang.String compactorHostAndPort) {
-    this.compactorHostAndPort = compactorHostAndPort;
+  public Status setCompactor(@org.apache.thrift.annotation.Nullable java.lang.String compactor) {
+    this.compactor = compactor;
     return this;
   }
 
-  public void unsetCompactorHostAndPort() {
-    this.compactorHostAndPort = null;
+  public void unsetCompactor() {
+    this.compactor = null;
   }
 
-  /** Returns true if field compactorHostAndPort is set (has been assigned a value) and false otherwise */
-  public boolean isSetCompactorHostAndPort() {
-    return this.compactorHostAndPort != 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 setCompactorHostAndPortIsSet(boolean value) {
+  public void setCompactorIsSet(boolean value) {
     if (!value) {
-      this.compactorHostAndPort = null;
+      this.compactor = null;
     }
   }
 
@@ -344,11 +344,11 @@ public class Status implements org.apache.thrift.TBase<Status, Status._Fields>,
       }
       break;
 
-    case COMPACTOR_HOST_AND_PORT:
+    case COMPACTOR:
       if (value == null) {
-        unsetCompactorHostAndPort();
+        unsetCompactor();
       } else {
-        setCompactorHostAndPort((java.lang.String)value);
+        setCompactor((java.lang.String)value);
       }
       break;
 
@@ -380,8 +380,8 @@ public class Status implements org.apache.thrift.TBase<Status, Status._Fields>,
     case COMPACTION_ID:
       return getCompactionId();
 
-    case COMPACTOR_HOST_AND_PORT:
-      return getCompactorHostAndPort();
+    case COMPACTOR:
+      return getCompactor();
 
     case STATE:
       return getState();
@@ -404,8 +404,8 @@ public class Status implements org.apache.thrift.TBase<Status, Status._Fields>,
       return isSetTimestamp();
     case COMPACTION_ID:
       return isSetCompactionId();
-    case COMPACTOR_HOST_AND_PORT:
-      return isSetCompactorHostAndPort();
+    case COMPACTOR:
+      return isSetCompactor();
     case STATE:
       return isSetState();
     case MESSAGE:
@@ -447,12 +447,12 @@ public class Status implements org.apache.thrift.TBase<Status, Status._Fields>,
         return false;
     }
 
-    boolean this_present_compactorHostAndPort = true && this.isSetCompactorHostAndPort();
-    boolean that_present_compactorHostAndPort = true && that.isSetCompactorHostAndPort();
-    if (this_present_compactorHostAndPort || that_present_compactorHostAndPort) {
-      if (!(this_present_compactorHostAndPort && that_present_compactorHostAndPort))
+    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.compactorHostAndPort.equals(that.compactorHostAndPort))
+      if (!this.compactor.equals(that.compactor))
         return false;
     }
 
@@ -485,9 +485,9 @@ public class Status implements org.apache.thrift.TBase<Status, Status._Fields>,
 
     hashCode = hashCode * 8191 + org.apache.thrift.TBaseHelper.hashCode(compactionId);
 
-    hashCode = hashCode * 8191 + ((isSetCompactorHostAndPort()) ? 131071 : 524287);
-    if (isSetCompactorHostAndPort())
-      hashCode = hashCode * 8191 + compactorHostAndPort.hashCode();
+    hashCode = hashCode * 8191 + ((isSetCompactor()) ? 131071 : 524287);
+    if (isSetCompactor())
+      hashCode = hashCode * 8191 + compactor.hashCode();
 
     hashCode = hashCode * 8191 + ((isSetState()) ? 131071 : 524287);
     if (isSetState())
@@ -528,12 +528,12 @@ public class Status implements org.apache.thrift.TBase<Status, Status._Fields>,
         return lastComparison;
       }
     }
-    lastComparison = java.lang.Boolean.valueOf(isSetCompactorHostAndPort()).compareTo(other.isSetCompactorHostAndPort());
+    lastComparison = java.lang.Boolean.valueOf(isSetCompactor()).compareTo(other.isSetCompactor());
     if (lastComparison != 0) {
       return lastComparison;
     }
-    if (isSetCompactorHostAndPort()) {
-      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.compactorHostAndPort, other.compactorHostAndPort);
+    if (isSetCompactor()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.compactor, other.compactor);
       if (lastComparison != 0) {
         return lastComparison;
       }
@@ -587,11 +587,11 @@ public class Status implements org.apache.thrift.TBase<Status, Status._Fields>,
     sb.append(this.compactionId);
     first = false;
     if (!first) sb.append(", ");
-    sb.append("compactorHostAndPort:");
-    if (this.compactorHostAndPort == null) {
+    sb.append("compactor:");
+    if (this.compactor == null) {
       sb.append("null");
     } else {
-      sb.append(this.compactorHostAndPort);
+      sb.append(this.compactor);
     }
     first = false;
     if (!first) sb.append(", ");
@@ -671,10 +671,10 @@ public class Status implements org.apache.thrift.TBase<Status, Status._Fields>,
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
             }
             break;
-          case 3: // COMPACTOR_HOST_AND_PORT
+          case 3: // COMPACTOR
             if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
-              struct.compactorHostAndPort = iprot.readString();
-              struct.setCompactorHostAndPortIsSet(true);
+              struct.compactor = iprot.readString();
+              struct.setCompactorIsSet(true);
             } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
             }
@@ -716,9 +716,9 @@ public class Status implements org.apache.thrift.TBase<Status, Status._Fields>,
       oprot.writeFieldBegin(COMPACTION_ID_FIELD_DESC);
       oprot.writeI64(struct.compactionId);
       oprot.writeFieldEnd();
-      if (struct.compactorHostAndPort != null) {
-        oprot.writeFieldBegin(COMPACTOR_HOST_AND_PORT_FIELD_DESC);
-        oprot.writeString(struct.compactorHostAndPort);
+      if (struct.compactor != null) {
+        oprot.writeFieldBegin(COMPACTOR_FIELD_DESC);
+        oprot.writeString(struct.compactor);
         oprot.writeFieldEnd();
       }
       if (struct.state != null) {
@@ -755,7 +755,7 @@ public class Status implements org.apache.thrift.TBase<Status, Status._Fields>,
       if (struct.isSetCompactionId()) {
         optionals.set(1);
       }
-      if (struct.isSetCompactorHostAndPort()) {
+      if (struct.isSetCompactor()) {
         optionals.set(2);
       }
       if (struct.isSetState()) {
@@ -771,8 +771,8 @@ public class Status implements org.apache.thrift.TBase<Status, Status._Fields>,
       if (struct.isSetCompactionId()) {
         oprot.writeI64(struct.compactionId);
       }
-      if (struct.isSetCompactorHostAndPort()) {
-        oprot.writeString(struct.compactorHostAndPort);
+      if (struct.isSetCompactor()) {
+        oprot.writeString(struct.compactor);
       }
       if (struct.isSetState()) {
         oprot.writeI32(struct.state.getValue());
@@ -795,8 +795,8 @@ public class Status implements org.apache.thrift.TBase<Status, Status._Fields>,
         struct.setCompactionIdIsSet(true);
       }
       if (incoming.get(2)) {
-        struct.compactorHostAndPort = iprot.readString();
-        struct.setCompactorHostAndPortIsSet(true);
+        struct.compactor = iprot.readString();
+        struct.setCompactorIsSet(true);
       }
       if (incoming.get(3)) {
         struct.state = org.apache.accumulo.core.compaction.thrift.CompactionState.findByValue(iprot.readI32());
diff --git a/core/src/main/java/org/apache/accumulo/core/dataImpl/thrift/CompactionStats.java b/core/src/main/java/org/apache/accumulo/core/dataImpl/thrift/CompactionStats.java
new file mode 100644
index 0000000..f795f89
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/dataImpl/thrift/CompactionStats.java
@@ -0,0 +1,582 @@
+/*
+ * 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.dataImpl.thrift;
+
+@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"})
+public class CompactionStats implements org.apache.thrift.TBase<CompactionStats, CompactionStats._Fields>, java.io.Serializable, Cloneable, Comparable<CompactionStats> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("CompactionStats");
+
+  private static final org.apache.thrift.protocol.TField ENTRIES_READ_FIELD_DESC = new org.apache.thrift.protocol.TField("entriesRead", org.apache.thrift.protocol.TType.I64, (short)1);
+  private static final org.apache.thrift.protocol.TField ENTRIES_WRITTEN_FIELD_DESC = new org.apache.thrift.protocol.TField("entriesWritten", org.apache.thrift.protocol.TType.I64, (short)2);
+  private static final org.apache.thrift.protocol.TField FILE_SIZE_FIELD_DESC = new org.apache.thrift.protocol.TField("fileSize", org.apache.thrift.protocol.TType.I64, (short)3);
+
+  private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new CompactionStatsStandardSchemeFactory();
+  private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new CompactionStatsTupleSchemeFactory();
+
+  public long entriesRead; // required
+  public long entriesWritten; // required
+  public long fileSize; // 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 {
+    ENTRIES_READ((short)1, "entriesRead"),
+    ENTRIES_WRITTEN((short)2, "entriesWritten"),
+    FILE_SIZE((short)3, "fileSize");
+
+    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: // ENTRIES_READ
+          return ENTRIES_READ;
+        case 2: // ENTRIES_WRITTEN
+          return ENTRIES_WRITTEN;
+        case 3: // FILE_SIZE
+          return FILE_SIZE;
+        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 __ENTRIESREAD_ISSET_ID = 0;
+  private static final int __ENTRIESWRITTEN_ISSET_ID = 1;
+  private static final int __FILESIZE_ISSET_ID = 2;
+  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.ENTRIES_READ, new org.apache.thrift.meta_data.FieldMetaData("entriesRead", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64)));
+    tmpMap.put(_Fields.ENTRIES_WRITTEN, new org.apache.thrift.meta_data.FieldMetaData("entriesWritten", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64)));
+    tmpMap.put(_Fields.FILE_SIZE, new org.apache.thrift.meta_data.FieldMetaData("fileSize", 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(CompactionStats.class, metaDataMap);
+  }
+
+  public CompactionStats() {
+  }
+
+  public CompactionStats(
+    long entriesRead,
+    long entriesWritten,
+    long fileSize)
+  {
+    this();
+    this.entriesRead = entriesRead;
+    setEntriesReadIsSet(true);
+    this.entriesWritten = entriesWritten;
+    setEntriesWrittenIsSet(true);
+    this.fileSize = fileSize;
+    setFileSizeIsSet(true);
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public CompactionStats(CompactionStats other) {
+    __isset_bitfield = other.__isset_bitfield;
+    this.entriesRead = other.entriesRead;
+    this.entriesWritten = other.entriesWritten;
+    this.fileSize = other.fileSize;
+  }
+
+  public CompactionStats deepCopy() {
+    return new CompactionStats(this);
+  }
+
+  @Override
+  public void clear() {
+    setEntriesReadIsSet(false);
+    this.entriesRead = 0;
+    setEntriesWrittenIsSet(false);
+    this.entriesWritten = 0;
+    setFileSizeIsSet(false);
+    this.fileSize = 0;
+  }
+
+  public long getEntriesRead() {
+    return this.entriesRead;
+  }
+
+  public CompactionStats setEntriesRead(long entriesRead) {
+    this.entriesRead = entriesRead;
+    setEntriesReadIsSet(true);
+    return this;
+  }
+
+  public void unsetEntriesRead() {
+    __isset_bitfield = org.apache.thrift.EncodingUtils.clearBit(__isset_bitfield, __ENTRIESREAD_ISSET_ID);
+  }
+
+  /** Returns true if field entriesRead is set (has been assigned a value) and false otherwise */
+  public boolean isSetEntriesRead() {
+    return org.apache.thrift.EncodingUtils.testBit(__isset_bitfield, __ENTRIESREAD_ISSET_ID);
+  }
+
+  public void setEntriesReadIsSet(boolean value) {
+    __isset_bitfield = org.apache.thrift.EncodingUtils.setBit(__isset_bitfield, __ENTRIESREAD_ISSET_ID, value);
+  }
+
+  public long getEntriesWritten() {
+    return this.entriesWritten;
+  }
+
+  public CompactionStats setEntriesWritten(long entriesWritten) {
+    this.entriesWritten = entriesWritten;
+    setEntriesWrittenIsSet(true);
+    return this;
+  }
+
+  public void unsetEntriesWritten() {
+    __isset_bitfield = org.apache.thrift.EncodingUtils.clearBit(__isset_bitfield, __ENTRIESWRITTEN_ISSET_ID);
+  }
+
+  /** Returns true if field entriesWritten is set (has been assigned a value) and false otherwise */
+  public boolean isSetEntriesWritten() {
+    return org.apache.thrift.EncodingUtils.testBit(__isset_bitfield, __ENTRIESWRITTEN_ISSET_ID);
+  }
+
+  public void setEntriesWrittenIsSet(boolean value) {
+    __isset_bitfield = org.apache.thrift.EncodingUtils.setBit(__isset_bitfield, __ENTRIESWRITTEN_ISSET_ID, value);
+  }
+
+  public long getFileSize() {
+    return this.fileSize;
+  }
+
+  public CompactionStats setFileSize(long fileSize) {
+    this.fileSize = fileSize;
+    setFileSizeIsSet(true);
+    return this;
+  }
+
+  public void unsetFileSize() {
+    __isset_bitfield = org.apache.thrift.EncodingUtils.clearBit(__isset_bitfield, __FILESIZE_ISSET_ID);
+  }
+
+  /** Returns true if field fileSize is set (has been assigned a value) and false otherwise */
+  public boolean isSetFileSize() {
+    return org.apache.thrift.EncodingUtils.testBit(__isset_bitfield, __FILESIZE_ISSET_ID);
+  }
+
+  public void setFileSizeIsSet(boolean value) {
+    __isset_bitfield = org.apache.thrift.EncodingUtils.setBit(__isset_bitfield, __FILESIZE_ISSET_ID, value);
+  }
+
+  public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable java.lang.Object value) {
+    switch (field) {
+    case ENTRIES_READ:
+      if (value == null) {
+        unsetEntriesRead();
+      } else {
+        setEntriesRead((java.lang.Long)value);
+      }
+      break;
+
+    case ENTRIES_WRITTEN:
+      if (value == null) {
+        unsetEntriesWritten();
+      } else {
+        setEntriesWritten((java.lang.Long)value);
+      }
+      break;
+
+    case FILE_SIZE:
+      if (value == null) {
+        unsetFileSize();
+      } else {
+        setFileSize((java.lang.Long)value);
+      }
+      break;
+
+    }
+  }
+
+  @org.apache.thrift.annotation.Nullable
+  public java.lang.Object getFieldValue(_Fields field) {
+    switch (field) {
+    case ENTRIES_READ:
+      return getEntriesRead();
+
+    case ENTRIES_WRITTEN:
+      return getEntriesWritten();
+
+    case FILE_SIZE:
+      return getFileSize();
+
+    }
+    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 ENTRIES_READ:
+      return isSetEntriesRead();
+    case ENTRIES_WRITTEN:
+      return isSetEntriesWritten();
+    case FILE_SIZE:
+      return isSetFileSize();
+    }
+    throw new java.lang.IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(java.lang.Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof CompactionStats)
+      return this.equals((CompactionStats)that);
+    return false;
+  }
+
+  public boolean equals(CompactionStats that) {
+    if (that == null)
+      return false;
+    if (this == that)
+      return true;
+
+    boolean this_present_entriesRead = true;
+    boolean that_present_entriesRead = true;
+    if (this_present_entriesRead || that_present_entriesRead) {
+      if (!(this_present_entriesRead && that_present_entriesRead))
+        return false;
+      if (this.entriesRead != that.entriesRead)
+        return false;
+    }
+
+    boolean this_present_entriesWritten = true;
+    boolean that_present_entriesWritten = true;
+    if (this_present_entriesWritten || that_present_entriesWritten) {
+      if (!(this_present_entriesWritten && that_present_entriesWritten))
+        return false;
+      if (this.entriesWritten != that.entriesWritten)
+        return false;
+    }
+
+    boolean this_present_fileSize = true;
+    boolean that_present_fileSize = true;
+    if (this_present_fileSize || that_present_fileSize) {
+      if (!(this_present_fileSize && that_present_fileSize))
+        return false;
+      if (this.fileSize != that.fileSize)
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    int hashCode = 1;
+
+    hashCode = hashCode * 8191 + org.apache.thrift.TBaseHelper.hashCode(entriesRead);
+
+    hashCode = hashCode * 8191 + org.apache.thrift.TBaseHelper.hashCode(entriesWritten);
+
+    hashCode = hashCode * 8191 + org.apache.thrift.TBaseHelper.hashCode(fileSize);
+
+    return hashCode;
+  }
+
+  @Override
+  public int compareTo(CompactionStats other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+
+    lastComparison = java.lang.Boolean.valueOf(isSetEntriesRead()).compareTo(other.isSetEntriesRead());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetEntriesRead()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.entriesRead, other.entriesRead);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = java.lang.Boolean.valueOf(isSetEntriesWritten()).compareTo(other.isSetEntriesWritten());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetEntriesWritten()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.entriesWritten, other.entriesWritten);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = java.lang.Boolean.valueOf(isSetFileSize()).compareTo(other.isSetFileSize());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetFileSize()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.fileSize, other.fileSize);
+      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("CompactionStats(");
+    boolean first = true;
+
+    sb.append("entriesRead:");
+    sb.append(this.entriesRead);
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("entriesWritten:");
+    sb.append(this.entriesWritten);
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("fileSize:");
+    sb.append(this.fileSize);
+    first = false;
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    // check for sub-struct validity
+  }
+
+  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+    try {
+      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, java.lang.ClassNotFoundException {
+    try {
+      // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor.
+      __isset_bitfield = 0;
+      read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private static class CompactionStatsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+    public CompactionStatsStandardScheme getScheme() {
+      return new CompactionStatsStandardScheme();
+    }
+  }
+
+  private static class CompactionStatsStandardScheme extends org.apache.thrift.scheme.StandardScheme<CompactionStats> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, CompactionStats 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: // ENTRIES_READ
+            if (schemeField.type == org.apache.thrift.protocol.TType.I64) {
+              struct.entriesRead = iprot.readI64();
+              struct.setEntriesReadIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 2: // ENTRIES_WRITTEN
+            if (schemeField.type == org.apache.thrift.protocol.TType.I64) {
+              struct.entriesWritten = iprot.readI64();
+              struct.setEntriesWrittenIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 3: // FILE_SIZE
+            if (schemeField.type == org.apache.thrift.protocol.TType.I64) {
+              struct.fileSize = iprot.readI64();
+              struct.setFileSizeIsSet(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, CompactionStats struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      oprot.writeFieldBegin(ENTRIES_READ_FIELD_DESC);
+      oprot.writeI64(struct.entriesRead);
+      oprot.writeFieldEnd();
+      oprot.writeFieldBegin(ENTRIES_WRITTEN_FIELD_DESC);
+      oprot.writeI64(struct.entriesWritten);
+      oprot.writeFieldEnd();
+      oprot.writeFieldBegin(FILE_SIZE_FIELD_DESC);
+      oprot.writeI64(struct.fileSize);
+      oprot.writeFieldEnd();
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class CompactionStatsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+    public CompactionStatsTupleScheme getScheme() {
+      return new CompactionStatsTupleScheme();
+    }
+  }
+
+  private static class CompactionStatsTupleScheme extends org.apache.thrift.scheme.TupleScheme<CompactionStats> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, CompactionStats 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.isSetEntriesRead()) {
+        optionals.set(0);
+      }
+      if (struct.isSetEntriesWritten()) {
+        optionals.set(1);
+      }
+      if (struct.isSetFileSize()) {
+        optionals.set(2);
+      }
+      oprot.writeBitSet(optionals, 3);
+      if (struct.isSetEntriesRead()) {
+        oprot.writeI64(struct.entriesRead);
+      }
+      if (struct.isSetEntriesWritten()) {
+        oprot.writeI64(struct.entriesWritten);
+      }
+      if (struct.isSetFileSize()) {
+        oprot.writeI64(struct.fileSize);
+      }
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, CompactionStats 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.entriesRead = iprot.readI64();
+        struct.setEntriesReadIsSet(true);
+      }
+      if (incoming.get(1)) {
+        struct.entriesWritten = iprot.readI64();
+        struct.setEntriesWrittenIsSet(true);
+      }
+      if (incoming.get(2)) {
+        struct.fileSize = iprot.readI64();
+        struct.setFileSizeIsSet(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/java/org/apache/accumulo/core/tabletserver/thrift/CompactionJob.java b/core/src/main/java/org/apache/accumulo/core/tabletserver/thrift/CompactionJob.java
new file mode 100644
index 0000000..8ff1214
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/tabletserver/thrift/CompactionJob.java
@@ -0,0 +1,1620 @@
+/*
+ * 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.tabletserver.thrift;
+
+@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"})
+public class CompactionJob implements org.apache.thrift.TBase<CompactionJob, CompactionJob._Fields>, java.io.Serializable, Cloneable, Comparable<CompactionJob> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("CompactionJob");
+
+  private static final org.apache.thrift.protocol.TField TRACE_INFO_FIELD_DESC = new org.apache.thrift.protocol.TField("traceInfo", 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 COMPACTION_ID_FIELD_DESC = new org.apache.thrift.protocol.TField("compactionId", org.apache.thrift.protocol.TType.I64, (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)5);
+  private static final org.apache.thrift.protocol.TField FILES_FIELD_DESC = new org.apache.thrift.protocol.TField("files", org.apache.thrift.protocol.TType.LIST, (short)6);
+  private static final org.apache.thrift.protocol.TField PRIORITY_FIELD_DESC = new org.apache.thrift.protocol.TField("priority", org.apache.thrift.protocol.TType.I32, (short)7);
+  private static final org.apache.thrift.protocol.TField READ_RATE_FIELD_DESC = new org.apache.thrift.protocol.TField("readRate", org.apache.thrift.protocol.TType.I32, (short)8);
+  private static final org.apache.thrift.protocol.TField WRITE_RATE_FIELD_DESC = new org.apache.thrift.protocol.TField("writeRate", org.apache.thrift.protocol.TType.I32, (short)9);
+  private static final org.apache.thrift.protocol.TField ITERATOR_SETTINGS_FIELD_DESC = new org.apache.thrift.protocol.TField("iteratorSettings", org.apache.thrift.protocol.TType.STRUCT, (short)10);
+  private static final org.apache.thrift.protocol.TField TYPE_FIELD_DESC = new org.apache.thrift.protocol.TField("type", org.apache.thrift.protocol.TType.I32, (short)11);
+  private static final org.apache.thrift.protocol.TField REASON_FIELD_DESC = new org.apache.thrift.protocol.TField("reason", org.apache.thrift.protocol.TType.I32, (short)12);
+  private static final org.apache.thrift.protocol.TField OUTPUT_FILE_FIELD_DESC = new org.apache.thrift.protocol.TField("outputFile", org.apache.thrift.protocol.TType.STRING, (short)13);
+
+  private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new CompactionJobStandardSchemeFactory();
+  private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new CompactionJobTupleSchemeFactory();
+
+  public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo traceInfo; // required
+  public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials; // required
+  public long compactionId; // required
+  public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent; // required
+  public @org.apache.thrift.annotation.Nullable java.util.List<java.lang.String> files; // required
+  public int priority; // required
+  public int readRate; // required
+  public int writeRate; // required
+  public @org.apache.thrift.annotation.Nullable IteratorConfig iteratorSettings; // required
+  /**
+   * 
+   * @see CompactionType
+   */
+  public @org.apache.thrift.annotation.Nullable CompactionType type; // required
+  /**
+   * 
+   * @see CompactionReason
+   */
+  public @org.apache.thrift.annotation.Nullable CompactionReason reason; // required
+  public @org.apache.thrift.annotation.Nullable java.lang.String outputFile; // 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 {
+    TRACE_INFO((short)1, "traceInfo"),
+    CREDENTIALS((short)2, "credentials"),
+    COMPACTION_ID((short)3, "compactionId"),
+    EXTENT((short)5, "extent"),
+    FILES((short)6, "files"),
+    PRIORITY((short)7, "priority"),
+    READ_RATE((short)8, "readRate"),
+    WRITE_RATE((short)9, "writeRate"),
+    ITERATOR_SETTINGS((short)10, "iteratorSettings"),
+    /**
+     * 
+     * @see CompactionType
+     */
+    TYPE((short)11, "type"),
+    /**
+     * 
+     * @see CompactionReason
+     */
+    REASON((short)12, "reason"),
+    OUTPUT_FILE((short)13, "outputFile");
+
+    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: // TRACE_INFO
+          return TRACE_INFO;
+        case 2: // CREDENTIALS
+          return CREDENTIALS;
+        case 3: // COMPACTION_ID
+          return COMPACTION_ID;
+        case 5: // EXTENT
+          return EXTENT;
+        case 6: // FILES
+          return FILES;
+        case 7: // PRIORITY
+          return PRIORITY;
+        case 8: // READ_RATE
+          return READ_RATE;
+        case 9: // WRITE_RATE
+          return WRITE_RATE;
+        case 10: // ITERATOR_SETTINGS
+          return ITERATOR_SETTINGS;
+        case 11: // TYPE
+          return TYPE;
+        case 12: // REASON
+          return REASON;
+        case 13: // OUTPUT_FILE
+          return OUTPUT_FILE;
+        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 __COMPACTIONID_ISSET_ID = 0;
+  private static final int __PRIORITY_ISSET_ID = 1;
+  private static final int __READRATE_ISSET_ID = 2;
+  private static final int __WRITERATE_ISSET_ID = 3;
+  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.TRACE_INFO, new org.apache.thrift.meta_data.FieldMetaData("traceInfo", 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.COMPACTION_ID, new org.apache.thrift.meta_data.FieldMetaData("compactionId", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64)));
+    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.FILES, new org.apache.thrift.meta_data.FieldMetaData("files", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+        new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, 
+            new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))));
+    tmpMap.put(_Fields.PRIORITY, new org.apache.thrift.meta_data.FieldMetaData("priority", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32)));
+    tmpMap.put(_Fields.READ_RATE, new org.apache.thrift.meta_data.FieldMetaData("readRate", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32)));
+    tmpMap.put(_Fields.WRITE_RATE, new org.apache.thrift.meta_data.FieldMetaData("writeRate", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32)));
+    tmpMap.put(_Fields.ITERATOR_SETTINGS, new org.apache.thrift.meta_data.FieldMetaData("iteratorSettings", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+        new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, IteratorConfig.class)));
+    tmpMap.put(_Fields.TYPE, new org.apache.thrift.meta_data.FieldMetaData("type", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+        new org.apache.thrift.meta_data.EnumMetaData(org.apache.thrift.protocol.TType.ENUM, CompactionType.class)));
+    tmpMap.put(_Fields.REASON, new org.apache.thrift.meta_data.FieldMetaData("reason", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+        new org.apache.thrift.meta_data.EnumMetaData(org.apache.thrift.protocol.TType.ENUM, CompactionReason.class)));
+    tmpMap.put(_Fields.OUTPUT_FILE, new org.apache.thrift.meta_data.FieldMetaData("outputFile", 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(CompactionJob.class, metaDataMap);
+  }
+
+  public CompactionJob() {
+  }
+
+  public CompactionJob(
+    org.apache.accumulo.core.trace.thrift.TInfo traceInfo,
+    org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials,
+    long compactionId,
+    org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent,
+    java.util.List<java.lang.String> files,
+    int priority,
+    int readRate,
+    int writeRate,
+    IteratorConfig iteratorSettings,
+    CompactionType type,
+    CompactionReason reason,
+    java.lang.String outputFile)
+  {
+    this();
+    this.traceInfo = traceInfo;
+    this.credentials = credentials;
+    this.compactionId = compactionId;
+    setCompactionIdIsSet(true);
+    this.extent = extent;
+    this.files = files;
+    this.priority = priority;
+    setPriorityIsSet(true);
+    this.readRate = readRate;
+    setReadRateIsSet(true);
+    this.writeRate = writeRate;
+    setWriteRateIsSet(true);
+    this.iteratorSettings = iteratorSettings;
+    this.type = type;
+    this.reason = reason;
+    this.outputFile = outputFile;
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public CompactionJob(CompactionJob other) {
+    __isset_bitfield = other.__isset_bitfield;
+    if (other.isSetTraceInfo()) {
+      this.traceInfo = new org.apache.accumulo.core.trace.thrift.TInfo(other.traceInfo);
+    }
+    if (other.isSetCredentials()) {
+      this.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(other.credentials);
+    }
+    this.compactionId = other.compactionId;
+    if (other.isSetExtent()) {
+      this.extent = new org.apache.accumulo.core.dataImpl.thrift.TKeyExtent(other.extent);
+    }
+    if (other.isSetFiles()) {
+      java.util.List<java.lang.String> __this__files = new java.util.ArrayList<java.lang.String>(other.files);
+      this.files = __this__files;
+    }
+    this.priority = other.priority;
+    this.readRate = other.readRate;
+    this.writeRate = other.writeRate;
+    if (other.isSetIteratorSettings()) {
+      this.iteratorSettings = new IteratorConfig(other.iteratorSettings);
+    }
+    if (other.isSetType()) {
+      this.type = other.type;
+    }
+    if (other.isSetReason()) {
+      this.reason = other.reason;
+    }
+    if (other.isSetOutputFile()) {
+      this.outputFile = other.outputFile;
+    }
+  }
+
+  public CompactionJob deepCopy() {
+    return new CompactionJob(this);
+  }
+
+  @Override
+  public void clear() {
+    this.traceInfo = null;
+    this.credentials = null;
+    setCompactionIdIsSet(false);
+    this.compactionId = 0;
+    this.extent = null;
+    this.files = null;
+    setPriorityIsSet(false);
+    this.priority = 0;
+    setReadRateIsSet(false);
+    this.readRate = 0;
+    setWriteRateIsSet(false);
+    this.writeRate = 0;
+    this.iteratorSettings = null;
+    this.type = null;
+    this.reason = null;
+    this.outputFile = null;
+  }
+
+  @org.apache.thrift.annotation.Nullable
+  public org.apache.accumulo.core.trace.thrift.TInfo getTraceInfo() {
+    return this.traceInfo;
+  }
+
+  public CompactionJob setTraceInfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo traceInfo) {
+    this.traceInfo = traceInfo;
+    return this;
+  }
+
+  public void unsetTraceInfo() {
+    this.traceInfo = null;
+  }
+
+  /** Returns true if field traceInfo is set (has been assigned a value) and false otherwise */
+  public boolean isSetTraceInfo() {
+    return this.traceInfo != null;
+  }
+
+  public void setTraceInfoIsSet(boolean value) {
+    if (!value) {
+      this.traceInfo = null;
+    }
+  }
+
+  @org.apache.thrift.annotation.Nullable
+  public org.apache.accumulo.core.securityImpl.thrift.TCredentials getCredentials() {
+    return this.credentials;
+  }
+
+  public CompactionJob 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 long getCompactionId() {
+    return this.compactionId;
+  }
+
+  public CompactionJob setCompactionId(long compactionId) {
+    this.compactionId = compactionId;
+    setCompactionIdIsSet(true);
+    return this;
+  }
+
+  public void unsetCompactionId() {
+    __isset_bitfield = org.apache.thrift.EncodingUtils.clearBit(__isset_bitfield, __COMPACTIONID_ISSET_ID);
+  }
+
+  /** Returns true if field compactionId is set (has been assigned a value) and false otherwise */
+  public boolean isSetCompactionId() {
+    return org.apache.thrift.EncodingUtils.testBit(__isset_bitfield, __COMPACTIONID_ISSET_ID);
+  }
+
+  public void setCompactionIdIsSet(boolean value) {
+    __isset_bitfield = org.apache.thrift.EncodingUtils.setBit(__isset_bitfield, __COMPACTIONID_ISSET_ID, value);
+  }
+
+  @org.apache.thrift.annotation.Nullable
+  public org.apache.accumulo.core.dataImpl.thrift.TKeyExtent getExtent() {
+    return this.extent;
+  }
+
+  public CompactionJob 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 int getFilesSize() {
+    return (this.files == null) ? 0 : this.files.size();
+  }
+
+  @org.apache.thrift.annotation.Nullable
+  public java.util.Iterator<java.lang.String> getFilesIterator() {
+    return (this.files == null) ? null : this.files.iterator();
+  }
+
+  public void addToFiles(java.lang.String elem) {
+    if (this.files == null) {
+      this.files = new java.util.ArrayList<java.lang.String>();
+    }
+    this.files.add(elem);
+  }
+
+  @org.apache.thrift.annotation.Nullable
+  public java.util.List<java.lang.String> getFiles() {
+    return this.files;
+  }
+
+  public CompactionJob setFiles(@org.apache.thrift.annotation.Nullable java.util.List<java.lang.String> files) {
+    this.files = files;
+    return this;
+  }
+
+  public void unsetFiles() {
+    this.files = null;
+  }
+
+  /** Returns true if field files is set (has been assigned a value) and false otherwise */
+  public boolean isSetFiles() {
+    return this.files != null;
+  }
+
+  public void setFilesIsSet(boolean value) {
+    if (!value) {
+      this.files = null;
+    }
+  }
+
+  public int getPriority() {
+    return this.priority;
+  }
+
+  public CompactionJob setPriority(int priority) {
+    this.priority = priority;
+    setPriorityIsSet(true);
+    return this;
+  }
+
+  public void unsetPriority() {
+    __isset_bitfield = org.apache.thrift.EncodingUtils.clearBit(__isset_bitfield, __PRIORITY_ISSET_ID);
+  }
+
+  /** Returns true if field priority is set (has been assigned a value) and false otherwise */
+  public boolean isSetPriority() {
+    return org.apache.thrift.EncodingUtils.testBit(__isset_bitfield, __PRIORITY_ISSET_ID);
+  }
+
+  public void setPriorityIsSet(boolean value) {
+    __isset_bitfield = org.apache.thrift.EncodingUtils.setBit(__isset_bitfield, __PRIORITY_ISSET_ID, value);
+  }
+
+  public int getReadRate() {
+    return this.readRate;
+  }
+
+  public CompactionJob setReadRate(int readRate) {
+    this.readRate = readRate;
+    setReadRateIsSet(true);
+    return this;
+  }
+
+  public void unsetReadRate() {
+    __isset_bitfield = org.apache.thrift.EncodingUtils.clearBit(__isset_bitfield, __READRATE_ISSET_ID);
+  }
+
+  /** Returns true if field readRate is set (has been assigned a value) and false otherwise */
+  public boolean isSetReadRate() {
+    return org.apache.thrift.EncodingUtils.testBit(__isset_bitfield, __READRATE_ISSET_ID);
+  }
+
+  public void setReadRateIsSet(boolean value) {
+    __isset_bitfield = org.apache.thrift.EncodingUtils.setBit(__isset_bitfield, __READRATE_ISSET_ID, value);
+  }
+
+  public int getWriteRate() {
+    return this.writeRate;
+  }
+
+  public CompactionJob setWriteRate(int writeRate) {
+    this.writeRate = writeRate;
+    setWriteRateIsSet(true);
+    return this;
+  }
+
+  public void unsetWriteRate() {
+    __isset_bitfield = org.apache.thrift.EncodingUtils.clearBit(__isset_bitfield, __WRITERATE_ISSET_ID);
+  }
+
+  /** Returns true if field writeRate is set (has been assigned a value) and false otherwise */
+  public boolean isSetWriteRate() {
+    return org.apache.thrift.EncodingUtils.testBit(__isset_bitfield, __WRITERATE_ISSET_ID);
+  }
+
+  public void setWriteRateIsSet(boolean value) {
+    __isset_bitfield = org.apache.thrift.EncodingUtils.setBit(__isset_bitfield, __WRITERATE_ISSET_ID, value);
+  }
+
+  @org.apache.thrift.annotation.Nullable
+  public IteratorConfig getIteratorSettings() {
+    return this.iteratorSettings;
+  }
+
+  public CompactionJob setIteratorSettings(@org.apache.thrift.annotation.Nullable IteratorConfig iteratorSettings) {
+    this.iteratorSettings = iteratorSettings;
+    return this;
+  }
+
+  public void unsetIteratorSettings() {
+    this.iteratorSettings = null;
+  }
+
+  /** Returns true if field iteratorSettings is set (has been assigned a value) and false otherwise */
+  public boolean isSetIteratorSettings() {
+    return this.iteratorSettings != null;
+  }
+
+  public void setIteratorSettingsIsSet(boolean value) {
+    if (!value) {
+      this.iteratorSettings = null;
+    }
+  }
+
+  /**
+   * 
+   * @see CompactionType
+   */
+  @org.apache.thrift.annotation.Nullable
+  public CompactionType getType() {
+    return this.type;
+  }
+
+  /**
+   * 
+   * @see CompactionType
+   */
+  public CompactionJob setType(@org.apache.thrift.annotation.Nullable CompactionType type) {
+    this.type = type;
+    return this;
+  }
+
+  public void unsetType() {
+    this.type = null;
+  }
+
+  /** Returns true if field type is set (has been assigned a value) and false otherwise */
+  public boolean isSetType() {
+    return this.type != null;
+  }
+
+  public void setTypeIsSet(boolean value) {
+    if (!value) {
+      this.type = null;
+    }
+  }
+
+  /**
+   * 
+   * @see CompactionReason
+   */
+  @org.apache.thrift.annotation.Nullable
+  public CompactionReason getReason() {
+    return this.reason;
+  }
+
+  /**
+   * 
+   * @see CompactionReason
+   */
+  public CompactionJob setReason(@org.apache.thrift.annotation.Nullable CompactionReason reason) {
+    this.reason = reason;
+    return this;
+  }
+
+  public void unsetReason() {
+    this.reason = null;
+  }
+
+  /** Returns true if field reason is set (has been assigned a value) and false otherwise */
+  public boolean isSetReason() {
+    return this.reason != null;
+  }
+
+  public void setReasonIsSet(boolean value) {
+    if (!value) {
+      this.reason = null;
+    }
+  }
+
+  @org.apache.thrift.annotation.Nullable
+  public java.lang.String getOutputFile() {
+    return this.outputFile;
+  }
+
+  public CompactionJob setOutputFile(@org.apache.thrift.annotation.Nullable java.lang.String outputFile) {
+    this.outputFile = outputFile;
+    return this;
+  }
+
+  public void unsetOutputFile() {
+    this.outputFile = null;
+  }
+
+  /** Returns true if field outputFile is set (has been assigned a value) and false otherwise */
+  public boolean isSetOutputFile() {
+    return this.outputFile != null;
+  }
+
+  public void setOutputFileIsSet(boolean value) {
+    if (!value) {
+      this.outputFile = null;
+    }
+  }
+
+  public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable java.lang.Object value) {
+    switch (field) {
+    case TRACE_INFO:
+      if (value == null) {
+        unsetTraceInfo();
+      } else {
+        setTraceInfo((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 COMPACTION_ID:
+      if (value == null) {
+        unsetCompactionId();
+      } else {
+        setCompactionId((java.lang.Long)value);
+      }
+      break;
+
+    case EXTENT:
+      if (value == null) {
+        unsetExtent();
+      } else {
+        setExtent((org.apache.accumulo.core.dataImpl.thrift.TKeyExtent)value);
+      }
+      break;
+
+    case FILES:
+      if (value == null) {
+        unsetFiles();
+      } else {
+        setFiles((java.util.List<java.lang.String>)value);
+      }
+      break;
+
+    case PRIORITY:
+      if (value == null) {
+        unsetPriority();
+      } else {
+        setPriority((java.lang.Integer)value);
+      }
+      break;
+
+    case READ_RATE:
+      if (value == null) {
+        unsetReadRate();
+      } else {
+        setReadRate((java.lang.Integer)value);
+      }
+      break;
+
+    case WRITE_RATE:
+      if (value == null) {
+        unsetWriteRate();
+      } else {
+        setWriteRate((java.lang.Integer)value);
+      }
+      break;
+
+    case ITERATOR_SETTINGS:
+      if (value == null) {
+        unsetIteratorSettings();
+      } else {
+        setIteratorSettings((IteratorConfig)value);
+      }
+      break;
+
+    case TYPE:
+      if (value == null) {
+        unsetType();
+      } else {
+        setType((CompactionType)value);
+      }
+      break;
+
+    case REASON:
+      if (value == null) {
+        unsetReason();
+      } else {
+        setReason((CompactionReason)value);
+      }
+      break;
+
+    case OUTPUT_FILE:
+      if (value == null) {
+        unsetOutputFile();
+      } else {
+        setOutputFile((java.lang.String)value);
+      }
+      break;
+
+    }
+  }
+
+  @org.apache.thrift.annotation.Nullable
+  public java.lang.Object getFieldValue(_Fields field) {
+    switch (field) {
+    case TRACE_INFO:
+      return getTraceInfo();
+
+    case CREDENTIALS:
+      return getCredentials();
+
+    case COMPACTION_ID:
+      return getCompactionId();
+
+    case EXTENT:
+      return getExtent();
+
+    case FILES:
+      return getFiles();
+
+    case PRIORITY:
+      return getPriority();
+
+    case READ_RATE:
+      return getReadRate();
+
+    case WRITE_RATE:
+      return getWriteRate();
+
+    case ITERATOR_SETTINGS:
+      return getIteratorSettings();
+
+    case TYPE:
+      return getType();
+
+    case REASON:
+      return getReason();
+
+    case OUTPUT_FILE:
+      return getOutputFile();
+
+    }
+    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 TRACE_INFO:
+      return isSetTraceInfo();
+    case CREDENTIALS:
+      return isSetCredentials();
+    case COMPACTION_ID:
+      return isSetCompactionId();
+    case EXTENT:
+      return isSetExtent();
+    case FILES:
+      return isSetFiles();
+    case PRIORITY:
+      return isSetPriority();
+    case READ_RATE:
+      return isSetReadRate();
+    case WRITE_RATE:
+      return isSetWriteRate();
+    case ITERATOR_SETTINGS:
+      return isSetIteratorSettings();
+    case TYPE:
+      return isSetType();
+    case REASON:
+      return isSetReason();
+    case OUTPUT_FILE:
+      return isSetOutputFile();
+    }
+    throw new java.lang.IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(java.lang.Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof CompactionJob)
+      return this.equals((CompactionJob)that);
+    return false;
+  }
+
+  public boolean equals(CompactionJob that) {
+    if (that == null)
+      return false;
+    if (this == that)
+      return true;
+
+    boolean this_present_traceInfo = true && this.isSetTraceInfo();
+    boolean that_present_traceInfo = true && that.isSetTraceInfo();
+    if (this_present_traceInfo || that_present_traceInfo) {
+      if (!(this_present_traceInfo && that_present_traceInfo))
+        return false;
+      if (!this.traceInfo.equals(that.traceInfo))
+        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_compactionId = true;
+    boolean that_present_compactionId = true;
+    if (this_present_compactionId || that_present_compactionId) {
+      if (!(this_present_compactionId && that_present_compactionId))
+        return false;
+      if (this.compactionId != that.compactionId)
+        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_files = true && this.isSetFiles();
+    boolean that_present_files = true && that.isSetFiles();
+    if (this_present_files || that_present_files) {
+      if (!(this_present_files && that_present_files))
+        return false;
+      if (!this.files.equals(that.files))
+        return false;
+    }
+
+    boolean this_present_priority = true;
+    boolean that_present_priority = true;
+    if (this_present_priority || that_present_priority) {
+      if (!(this_present_priority && that_present_priority))
+        return false;
+      if (this.priority != that.priority)
+        return false;
+    }
+
+    boolean this_present_readRate = true;
+    boolean that_present_readRate = true;
+    if (this_present_readRate || that_present_readRate) {
+      if (!(this_present_readRate && that_present_readRate))
+        return false;
+      if (this.readRate != that.readRate)
+        return false;
+    }
+
+    boolean this_present_writeRate = true;
+    boolean that_present_writeRate = true;
+    if (this_present_writeRate || that_present_writeRate) {
+      if (!(this_present_writeRate && that_present_writeRate))
+        return false;
+      if (this.writeRate != that.writeRate)
+        return false;
+    }
+
+    boolean this_present_iteratorSettings = true && this.isSetIteratorSettings();
+    boolean that_present_iteratorSettings = true && that.isSetIteratorSettings();
+    if (this_present_iteratorSettings || that_present_iteratorSettings) {
+      if (!(this_present_iteratorSettings && that_present_iteratorSettings))
+        return false;
+      if (!this.iteratorSettings.equals(that.iteratorSettings))
+        return false;
+    }
+
+    boolean this_present_type = true && this.isSetType();
+    boolean that_present_type = true && that.isSetType();
+    if (this_present_type || that_present_type) {
+      if (!(this_present_type && that_present_type))
+        return false;
+      if (!this.type.equals(that.type))
+        return false;
+    }
+
+    boolean this_present_reason = true && this.isSetReason();
+    boolean that_present_reason = true && that.isSetReason();
+    if (this_present_reason || that_present_reason) {
+      if (!(this_present_reason && that_present_reason))
+        return false;
+      if (!this.reason.equals(that.reason))
+        return false;
+    }
+
+    boolean this_present_outputFile = true && this.isSetOutputFile();
+    boolean that_present_outputFile = true && that.isSetOutputFile();
+    if (this_present_outputFile || that_present_outputFile) {
+      if (!(this_present_outputFile && that_present_outputFile))
+        return false;
+      if (!this.outputFile.equals(that.outputFile))
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    int hashCode = 1;
+
+    hashCode = hashCode * 8191 + ((isSetTraceInfo()) ? 131071 : 524287);
+    if (isSetTraceInfo())
+      hashCode = hashCode * 8191 + traceInfo.hashCode();
+
+    hashCode = hashCode * 8191 + ((isSetCredentials()) ? 131071 : 524287);
+    if (isSetCredentials())
+      hashCode = hashCode * 8191 + credentials.hashCode();
+
+    hashCode = hashCode * 8191 + org.apache.thrift.TBaseHelper.hashCode(compactionId);
+
+    hashCode = hashCode * 8191 + ((isSetExtent()) ? 131071 : 524287);
+    if (isSetExtent())
+      hashCode = hashCode * 8191 + extent.hashCode();
+
+    hashCode = hashCode * 8191 + ((isSetFiles()) ? 131071 : 524287);
+    if (isSetFiles())
+      hashCode = hashCode * 8191 + files.hashCode();
+
+    hashCode = hashCode * 8191 + priority;
+
+    hashCode = hashCode * 8191 + readRate;
+
+    hashCode = hashCode * 8191 + writeRate;
+
+    hashCode = hashCode * 8191 + ((isSetIteratorSettings()) ? 131071 : 524287);
+    if (isSetIteratorSettings())
+      hashCode = hashCode * 8191 + iteratorSettings.hashCode();
+
+    hashCode = hashCode * 8191 + ((isSetType()) ? 131071 : 524287);
+    if (isSetType())
+      hashCode = hashCode * 8191 + type.getValue();
+
+    hashCode = hashCode * 8191 + ((isSetReason()) ? 131071 : 524287);
+    if (isSetReason())
+      hashCode = hashCode * 8191 + reason.getValue();
+
+    hashCode = hashCode * 8191 + ((isSetOutputFile()) ? 131071 : 524287);
+    if (isSetOutputFile())
+      hashCode = hashCode * 8191 + outputFile.hashCode();
+
+    return hashCode;
+  }
+
+  @Override
+  public int compareTo(CompactionJob other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+
+    lastComparison = java.lang.Boolean.valueOf(isSetTraceInfo()).compareTo(other.isSetTraceInfo());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetTraceInfo()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.traceInfo, other.traceInfo);
+      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(isSetCompactionId()).compareTo(other.isSetCompactionId());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetCompactionId()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.compactionId, other.compactionId);
+      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(isSetFiles()).compareTo(other.isSetFiles());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetFiles()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.files, other.files);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = java.lang.Boolean.valueOf(isSetPriority()).compareTo(other.isSetPriority());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetPriority()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.priority, other.priority);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = java.lang.Boolean.valueOf(isSetReadRate()).compareTo(other.isSetReadRate());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetReadRate()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.readRate, other.readRate);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = java.lang.Boolean.valueOf(isSetWriteRate()).compareTo(other.isSetWriteRate());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetWriteRate()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.writeRate, other.writeRate);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = java.lang.Boolean.valueOf(isSetIteratorSettings()).compareTo(other.isSetIteratorSettings());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetIteratorSettings()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.iteratorSettings, other.iteratorSettings);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = java.lang.Boolean.valueOf(isSetType()).compareTo(other.isSetType());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetType()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.type, other.type);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = java.lang.Boolean.valueOf(isSetReason()).compareTo(other.isSetReason());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetReason()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.reason, other.reason);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = java.lang.Boolean.valueOf(isSetOutputFile()).compareTo(other.isSetOutputFile());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetOutputFile()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.outputFile, other.outputFile);
+      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("CompactionJob(");
+    boolean first = true;
+
+    sb.append("traceInfo:");
+    if (this.traceInfo == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.traceInfo);
+    }
+    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("compactionId:");
+    sb.append(this.compactionId);
+    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("files:");
+    if (this.files == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.files);
+    }
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("priority:");
+    sb.append(this.priority);
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("readRate:");
+    sb.append(this.readRate);
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("writeRate:");
+    sb.append(this.writeRate);
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("iteratorSettings:");
+    if (this.iteratorSettings == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.iteratorSettings);
+    }
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("type:");
+    if (this.type == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.type);
+    }
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("reason:");
+    if (this.reason == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.reason);
+    }
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("outputFile:");
+    if (this.outputFile == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.outputFile);
+    }
+    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 (traceInfo != null) {
+      traceInfo.validate();
+    }
+    if (credentials != null) {
+      credentials.validate();
+    }
+    if (extent != null) {
+      extent.validate();
+    }
+    if (iteratorSettings != null) {
+      iteratorSettings.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 CompactionJobStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+    public CompactionJobStandardScheme getScheme() {
+      return new CompactionJobStandardScheme();
+    }
+  }
+
+  private static class CompactionJobStandardScheme extends org.apache.thrift.scheme.StandardScheme<CompactionJob> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, CompactionJob 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: // TRACE_INFO
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+              struct.traceInfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+              struct.traceInfo.read(iprot);
+              struct.setTraceInfoIsSet(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: // COMPACTION_ID
+            if (schemeField.type == org.apache.thrift.protocol.TType.I64) {
+              struct.compactionId = iprot.readI64();
+              struct.setCompactionIdIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 5: // 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 6: // FILES
+            if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
+              {
+                org.apache.thrift.protocol.TList _list116 = iprot.readListBegin();
+                struct.files = new java.util.ArrayList<java.lang.String>(_list116.size);
+                @org.apache.thrift.annotation.Nullable java.lang.String _elem117;
+                for (int _i118 = 0; _i118 < _list116.size; ++_i118)
+                {
+                  _elem117 = iprot.readString();
+                  struct.files.add(_elem117);
+                }
+                iprot.readListEnd();
+              }
+              struct.setFilesIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 7: // PRIORITY
+            if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
+              struct.priority = iprot.readI32();
+              struct.setPriorityIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 8: // READ_RATE
+            if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
+              struct.readRate = iprot.readI32();
+              struct.setReadRateIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 9: // WRITE_RATE
+            if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
+              struct.writeRate = iprot.readI32();
+              struct.setWriteRateIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 10: // ITERATOR_SETTINGS
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+              struct.iteratorSettings = new IteratorConfig();
+              struct.iteratorSettings.read(iprot);
+              struct.setIteratorSettingsIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 11: // TYPE
+            if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
+              struct.type = org.apache.accumulo.core.tabletserver.thrift.CompactionType.findByValue(iprot.readI32());
+              struct.setTypeIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 12: // REASON
+            if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
+              struct.reason = org.apache.accumulo.core.tabletserver.thrift.CompactionReason.findByValue(iprot.readI32());
+              struct.setReasonIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 13: // OUTPUT_FILE
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.outputFile = iprot.readString();
+              struct.setOutputFileIsSet(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, CompactionJob struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      if (struct.traceInfo != null) {
+        oprot.writeFieldBegin(TRACE_INFO_FIELD_DESC);
+        struct.traceInfo.write(oprot);
+        oprot.writeFieldEnd();
+      }
+      if (struct.credentials != null) {
+        oprot.writeFieldBegin(CREDENTIALS_FIELD_DESC);
+        struct.credentials.write(oprot);
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldBegin(COMPACTION_ID_FIELD_DESC);
+      oprot.writeI64(struct.compactionId);
+      oprot.writeFieldEnd();
+      if (struct.extent != null) {
+        oprot.writeFieldBegin(EXTENT_FIELD_DESC);
+        struct.extent.write(oprot);
+        oprot.writeFieldEnd();
+      }
+      if (struct.files != null) {
+        oprot.writeFieldBegin(FILES_FIELD_DESC);
+        {
+          oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.files.size()));
+          for (java.lang.String _iter119 : struct.files)
+          {
+            oprot.writeString(_iter119);
+          }
+          oprot.writeListEnd();
+        }
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldBegin(PRIORITY_FIELD_DESC);
+      oprot.writeI32(struct.priority);
+      oprot.writeFieldEnd();
+      oprot.writeFieldBegin(READ_RATE_FIELD_DESC);
+      oprot.writeI32(struct.readRate);
+      oprot.writeFieldEnd();
+      oprot.writeFieldBegin(WRITE_RATE_FIELD_DESC);
+      oprot.writeI32(struct.writeRate);
+      oprot.writeFieldEnd();
+      if (struct.iteratorSettings != null) {
+        oprot.writeFieldBegin(ITERATOR_SETTINGS_FIELD_DESC);
+        struct.iteratorSettings.write(oprot);
+        oprot.writeFieldEnd();
+      }
+      if (struct.type != null) {
+        oprot.writeFieldBegin(TYPE_FIELD_DESC);
+        oprot.writeI32(struct.type.getValue());
+        oprot.writeFieldEnd();
+      }
+      if (struct.reason != null) {
+        oprot.writeFieldBegin(REASON_FIELD_DESC);
+        oprot.writeI32(struct.reason.getValue());
+        oprot.writeFieldEnd();
+      }
+      if (struct.outputFile != null) {
+        oprot.writeFieldBegin(OUTPUT_FILE_FIELD_DESC);
+        oprot.writeString(struct.outputFile);
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class CompactionJobTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+    public CompactionJobTupleScheme getScheme() {
+      return new CompactionJobTupleScheme();
+    }
+  }
+
+  private static class CompactionJobTupleScheme extends org.apache.thrift.scheme.TupleScheme<CompactionJob> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, CompactionJob 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.isSetTraceInfo()) {
+        optionals.set(0);
+      }
+      if (struct.isSetCredentials()) {
+        optionals.set(1);
+      }
+      if (struct.isSetCompactionId()) {
+        optionals.set(2);
+      }
+      if (struct.isSetExtent()) {
+        optionals.set(3);
+      }
+      if (struct.isSetFiles()) {
+        optionals.set(4);
+      }
+      if (struct.isSetPriority()) {
+        optionals.set(5);
+      }
+      if (struct.isSetReadRate()) {
+        optionals.set(6);
+      }
+      if (struct.isSetWriteRate()) {
+        optionals.set(7);
+      }
+      if (struct.isSetIteratorSettings()) {
+        optionals.set(8);
+      }
+      if (struct.isSetType()) {
+        optionals.set(9);
+      }
+      if (struct.isSetReason()) {
+        optionals.set(10);
+      }
+      if (struct.isSetOutputFile()) {
+        optionals.set(11);
+      }
+      oprot.writeBitSet(optionals, 12);
+      if (struct.isSetTraceInfo()) {
+        struct.traceInfo.write(oprot);
+      }
+      if (struct.isSetCredentials()) {
+        struct.credentials.write(oprot);
+      }
+      if (struct.isSetCompactionId()) {
+        oprot.writeI64(struct.compactionId);
+      }
+      if (struct.isSetExtent()) {
+        struct.extent.write(oprot);
+      }
+      if (struct.isSetFiles()) {
+        {
+          oprot.writeI32(struct.files.size());
+          for (java.lang.String _iter120 : struct.files)
+          {
+            oprot.writeString(_iter120);
+          }
+        }
+      }
+      if (struct.isSetPriority()) {
+        oprot.writeI32(struct.priority);
+      }
+      if (struct.isSetReadRate()) {
+        oprot.writeI32(struct.readRate);
+      }
+      if (struct.isSetWriteRate()) {
+        oprot.writeI32(struct.writeRate);
+      }
+      if (struct.isSetIteratorSettings()) {
+        struct.iteratorSettings.write(oprot);
+      }
+      if (struct.isSetType()) {
+        oprot.writeI32(struct.type.getValue());
+      }
+      if (struct.isSetReason()) {
+        oprot.writeI32(struct.reason.getValue());
+      }
+      if (struct.isSetOutputFile()) {
+        oprot.writeString(struct.outputFile);
+      }
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, CompactionJob struct) throws org.apache.thrift.TException {
+      org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
+      java.util.BitSet incoming = iprot.readBitSet(12);
+      if (incoming.get(0)) {
+        struct.traceInfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+        struct.traceInfo.read(iprot);
+        struct.setTraceInfoIsSet(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.compactionId = iprot.readI64();
+        struct.setCompactionIdIsSet(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)) {
+        {
+          org.apache.thrift.protocol.TList _list121 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+          struct.files = new java.util.ArrayList<java.lang.String>(_list121.size);
+          @org.apache.thrift.annotation.Nullable java.lang.String _elem122;
+          for (int _i123 = 0; _i123 < _list121.size; ++_i123)
+          {
+            _elem122 = iprot.readString();
+            struct.files.add(_elem122);
+          }
+        }
+        struct.setFilesIsSet(true);
+      }
+      if (incoming.get(5)) {
+        struct.priority = iprot.readI32();
+        struct.setPriorityIsSet(true);
+      }
+      if (incoming.get(6)) {
+        struct.readRate = iprot.readI32();
+        struct.setReadRateIsSet(true);
+      }
+      if (incoming.get(7)) {
+        struct.writeRate = iprot.readI32();
+        struct.setWriteRateIsSet(true);
+      }
+      if (incoming.get(8)) {
+        struct.iteratorSettings = new IteratorConfig();
+        struct.iteratorSettings.read(iprot);
+        struct.setIteratorSettingsIsSet(true);
+      }
+      if (incoming.get(9)) {
+        struct.type = org.apache.accumulo.core.tabletserver.thrift.CompactionType.findByValue(iprot.readI32());
+        struct.setTypeIsSet(true);
+      }
+      if (incoming.get(10)) {
+        struct.reason = org.apache.accumulo.core.tabletserver.thrift.CompactionReason.findByValue(iprot.readI32());
+        struct.setReasonIsSet(true);
+      }
+      if (incoming.get(11)) {
+        struct.outputFile = iprot.readString();
+        struct.setOutputFileIsSet(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/java/org/apache/accumulo/core/tabletserver/thrift/CompactionQueueSummary.java b/core/src/main/java/org/apache/accumulo/core/tabletserver/thrift/CompactionQueueSummary.java
new file mode 100644
index 0000000..c205b22
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/tabletserver/thrift/CompactionQueueSummary.java
@@ -0,0 +1,591 @@
+/*
+ * 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.tabletserver.thrift;
+
+@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"})
+public class CompactionQueueSummary implements org.apache.thrift.TBase<CompactionQueueSummary, CompactionQueueSummary._Fields>, java.io.Serializable, Cloneable, Comparable<CompactionQueueSummary> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("CompactionQueueSummary");
+
+  private static final org.apache.thrift.protocol.TField QUEUE_FIELD_DESC = new org.apache.thrift.protocol.TField("queue", org.apache.thrift.protocol.TType.STRING, (short)1);
+  private static final org.apache.thrift.protocol.TField PRIORITY_FIELD_DESC = new org.apache.thrift.protocol.TField("priority", org.apache.thrift.protocol.TType.I64, (short)2);
+  private static final org.apache.thrift.protocol.TField COUNT_FIELD_DESC = new org.apache.thrift.protocol.TField("count", org.apache.thrift.protocol.TType.I32, (short)3);
+
+  private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new CompactionQueueSummaryStandardSchemeFactory();
+  private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new CompactionQueueSummaryTupleSchemeFactory();
+
+  public @org.apache.thrift.annotation.Nullable java.lang.String queue; // required
+  public long priority; // required
+  public int count; // 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 {
+    QUEUE((short)1, "queue"),
+    PRIORITY((short)2, "priority"),
+    COUNT((short)3, "count");
+
+    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: // QUEUE
+          return QUEUE;
+        case 2: // PRIORITY
+          return PRIORITY;
+        case 3: // COUNT
+          return COUNT;
+        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 __PRIORITY_ISSET_ID = 0;
+  private static final int __COUNT_ISSET_ID = 1;
+  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.QUEUE, new org.apache.thrift.meta_data.FieldMetaData("queue", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.PRIORITY, new org.apache.thrift.meta_data.FieldMetaData("priority", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64)));
+    tmpMap.put(_Fields.COUNT, new org.apache.thrift.meta_data.FieldMetaData("count", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32)));
+    metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(CompactionQueueSummary.class, metaDataMap);
+  }
+
+  public CompactionQueueSummary() {
+  }
+
+  public CompactionQueueSummary(
+    java.lang.String queue,
+    long priority,
+    int count)
+  {
+    this();
+    this.queue = queue;
+    this.priority = priority;
+    setPriorityIsSet(true);
+    this.count = count;
+    setCountIsSet(true);
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public CompactionQueueSummary(CompactionQueueSummary other) {
+    __isset_bitfield = other.__isset_bitfield;
+    if (other.isSetQueue()) {
+      this.queue = other.queue;
+    }
+    this.priority = other.priority;
+    this.count = other.count;
+  }
+
+  public CompactionQueueSummary deepCopy() {
+    return new CompactionQueueSummary(this);
+  }
+
+  @Override
+  public void clear() {
+    this.queue = null;
+    setPriorityIsSet(false);
+    this.priority = 0;
+    setCountIsSet(false);
+    this.count = 0;
+  }
+
+  @org.apache.thrift.annotation.Nullable
+  public java.lang.String getQueue() {
+    return this.queue;
+  }
+
+  public CompactionQueueSummary setQueue(@org.apache.thrift.annotation.Nullable java.lang.String queue) {
+    this.queue = queue;
+    return this;
+  }
+
+  public void unsetQueue() {
+    this.queue = null;
+  }
+
+  /** Returns true if field queue is set (has been assigned a value) and false otherwise */
+  public boolean isSetQueue() {
+    return this.queue != null;
+  }
+
+  public void setQueueIsSet(boolean value) {
+    if (!value) {
+      this.queue = null;
+    }
+  }
+
+  public long getPriority() {
+    return this.priority;
+  }
+
+  public CompactionQueueSummary setPriority(long priority) {
+    this.priority = priority;
+    setPriorityIsSet(true);
+    return this;
+  }
+
+  public void unsetPriority() {
+    __isset_bitfield = org.apache.thrift.EncodingUtils.clearBit(__isset_bitfield, __PRIORITY_ISSET_ID);
+  }
+
+  /** Returns true if field priority is set (has been assigned a value) and false otherwise */
+  public boolean isSetPriority() {
+    return org.apache.thrift.EncodingUtils.testBit(__isset_bitfield, __PRIORITY_ISSET_ID);
+  }
+
+  public void setPriorityIsSet(boolean value) {
+    __isset_bitfield = org.apache.thrift.EncodingUtils.setBit(__isset_bitfield, __PRIORITY_ISSET_ID, value);
+  }
+
+  public int getCount() {
+    return this.count;
+  }
+
+  public CompactionQueueSummary setCount(int count) {
+    this.count = count;
+    setCountIsSet(true);
+    return this;
+  }
+
+  public void unsetCount() {
+    __isset_bitfield = org.apache.thrift.EncodingUtils.clearBit(__isset_bitfield, __COUNT_ISSET_ID);
+  }
+
+  /** Returns true if field count is set (has been assigned a value) and false otherwise */
+  public boolean isSetCount() {
+    return org.apache.thrift.EncodingUtils.testBit(__isset_bitfield, __COUNT_ISSET_ID);
+  }
+
+  public void setCountIsSet(boolean value) {
+    __isset_bitfield = org.apache.thrift.EncodingUtils.setBit(__isset_bitfield, __COUNT_ISSET_ID, value);
+  }
+
+  public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable java.lang.Object value) {
+    switch (field) {
+    case QUEUE:
+      if (value == null) {
+        unsetQueue();
+      } else {
+        setQueue((java.lang.String)value);
+      }
+      break;
+
+    case PRIORITY:
+      if (value == null) {
+        unsetPriority();
+      } else {
+        setPriority((java.lang.Long)value);
+      }
+      break;
+
+    case COUNT:
+      if (value == null) {
+        unsetCount();
+      } else {
+        setCount((java.lang.Integer)value);
+      }
+      break;
+
+    }
+  }
+
+  @org.apache.thrift.annotation.Nullable
+  public java.lang.Object getFieldValue(_Fields field) {
+    switch (field) {
+    case QUEUE:
+      return getQueue();
+
+    case PRIORITY:
+      return getPriority();
+
+    case COUNT:
+      return getCount();
+
+    }
+    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 QUEUE:
+      return isSetQueue();
+    case PRIORITY:
+      return isSetPriority();
+    case COUNT:
+      return isSetCount();
+    }
+    throw new java.lang.IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(java.lang.Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof CompactionQueueSummary)
+      return this.equals((CompactionQueueSummary)that);
+    return false;
+  }
+
+  public boolean equals(CompactionQueueSummary that) {
+    if (that == null)
+      return false;
+    if (this == that)
+      return true;
+
+    boolean this_present_queue = true && this.isSetQueue();
+    boolean that_present_queue = true && that.isSetQueue();
+    if (this_present_queue || that_present_queue) {
+      if (!(this_present_queue && that_present_queue))
+        return false;
+      if (!this.queue.equals(that.queue))
+        return false;
+    }
+
+    boolean this_present_priority = true;
+    boolean that_present_priority = true;
+    if (this_present_priority || that_present_priority) {
+      if (!(this_present_priority && that_present_priority))
+        return false;
+      if (this.priority != that.priority)
+        return false;
+    }
+
+    boolean this_present_count = true;
+    boolean that_present_count = true;
+    if (this_present_count || that_present_count) {
+      if (!(this_present_count && that_present_count))
+        return false;
+      if (this.count != that.count)
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    int hashCode = 1;
+
+    hashCode = hashCode * 8191 + ((isSetQueue()) ? 131071 : 524287);
+    if (isSetQueue())
+      hashCode = hashCode * 8191 + queue.hashCode();
+
+    hashCode = hashCode * 8191 + org.apache.thrift.TBaseHelper.hashCode(priority);
+
+    hashCode = hashCode * 8191 + count;
+
+    return hashCode;
+  }
+
+  @Override
+  public int compareTo(CompactionQueueSummary other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+
+    lastComparison = java.lang.Boolean.valueOf(isSetQueue()).compareTo(other.isSetQueue());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetQueue()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.queue, other.queue);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = java.lang.Boolean.valueOf(isSetPriority()).compareTo(other.isSetPriority());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetPriority()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.priority, other.priority);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = java.lang.Boolean.valueOf(isSetCount()).compareTo(other.isSetCount());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetCount()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.count, other.count);
+      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("CompactionQueueSummary(");
+    boolean first = true;
+
+    sb.append("queue:");
+    if (this.queue == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.queue);
+    }
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("priority:");
+    sb.append(this.priority);
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("count:");
+    sb.append(this.count);
+    first = false;
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    // check for sub-struct validity
+  }
+
+  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+    try {
+      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, java.lang.ClassNotFoundException {
+    try {
+      // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor.
+      __isset_bitfield = 0;
+      read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private static class CompactionQueueSummaryStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+    public CompactionQueueSummaryStandardScheme getScheme() {
+      return new CompactionQueueSummaryStandardScheme();
+    }
+  }
+
+  private static class CompactionQueueSummaryStandardScheme extends org.apache.thrift.scheme.StandardScheme<CompactionQueueSummary> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, CompactionQueueSummary 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: // QUEUE
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.queue = iprot.readString();
+              struct.setQueueIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 2: // PRIORITY
+            if (schemeField.type == org.apache.thrift.protocol.TType.I64) {
+              struct.priority = iprot.readI64();
+              struct.setPriorityIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 3: // COUNT
+            if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
+              struct.count = iprot.readI32();
+              struct.setCountIsSet(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, CompactionQueueSummary struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      if (struct.queue != null) {
+        oprot.writeFieldBegin(QUEUE_FIELD_DESC);
+        oprot.writeString(struct.queue);
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldBegin(PRIORITY_FIELD_DESC);
+      oprot.writeI64(struct.priority);
+      oprot.writeFieldEnd();
+      oprot.writeFieldBegin(COUNT_FIELD_DESC);
+      oprot.writeI32(struct.count);
+      oprot.writeFieldEnd();
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class CompactionQueueSummaryTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+    public CompactionQueueSummaryTupleScheme getScheme() {
+      return new CompactionQueueSummaryTupleScheme();
+    }
+  }
+
+  private static class CompactionQueueSummaryTupleScheme extends org.apache.thrift.scheme.TupleScheme<CompactionQueueSummary> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, CompactionQueueSummary 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.isSetQueue()) {
+        optionals.set(0);
+      }
+      if (struct.isSetPriority()) {
+        optionals.set(1);
+      }
+      if (struct.isSetCount()) {
+        optionals.set(2);
+      }
+      oprot.writeBitSet(optionals, 3);
+      if (struct.isSetQueue()) {
+        oprot.writeString(struct.queue);
+      }
+      if (struct.isSetPriority()) {
+        oprot.writeI64(struct.priority);
+      }
+      if (struct.isSetCount()) {
+        oprot.writeI32(struct.count);
+      }
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, CompactionQueueSummary 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.queue = iprot.readString();
+        struct.setQueueIsSet(true);
+      }
+      if (incoming.get(1)) {
+        struct.priority = iprot.readI64();
+        struct.setPriorityIsSet(true);
+      }
+      if (incoming.get(2)) {
+        struct.count = iprot.readI32();
+        struct.setCountIsSet(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/java/org/apache/accumulo/core/tabletserver/thrift/TabletClientService.java b/core/src/main/java/org/apache/accumulo/core/tabletserver/thrift/TabletClientService.java
index e3d3d38..f56f5ea 100644
--- a/core/src/main/java/org/apache/accumulo/core/tabletserver/thrift/TabletClientService.java
+++ b/core/src/main/java/org/apache/accumulo/core/tabletserver/thrift/TabletClientService.java
@@ -101,6 +101,12 @@ public class TabletClientService {
 
     public org.apache.accumulo.core.dataImpl.thrift.TSummaries contiuneGetSummaries(org.apache.accumulo.core.trace.thrift.TInfo tinfo, long sessionId) throws NoSuchScanIDException, org.apache.thrift.TException;
 
+    public java.util.List<CompactionQueueSummary> getCompactionQueueInfo() throws org.apache.thrift.TException;
+
+    public CompactionJob reserveCompactionJob(java.lang.String queueName, long priority, java.lang.String compactor) throws org.apache.thrift.TException;
+
+    public void compactionJobFinished(CompactionJob job) throws org.apache.thrift.TException;
+
   }
 
   public interface AsyncIface extends org.apache.accumulo.core.clientImpl.thrift.ClientService .AsyncIface {
@@ -177,6 +183,12 @@ public class TabletClientService {
 
     public void contiuneGetSummaries(org.apache.accumulo.core.trace.thrift.TInfo tinfo, long sessionId, org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.dataImpl.thrift.TSummaries> resultHandler) throws org.apache.thrift.TException;
 
+    public void getCompactionQueueInfo(org.apache.thrift.async.AsyncMethodCallback<java.util.List<CompactionQueueSummary>> resultHandler) throws org.apache.thrift.TException;
+
+    public void reserveCompactionJob(java.lang.String queueName, long priority, java.lang.String compactor, org.apache.thrift.async.AsyncMethodCallback<CompactionJob> resultHandler) throws org.apache.thrift.TException;
+
+    public void compactionJobFinished(CompactionJob job, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
+
   }
 
   public static class Client extends org.apache.accumulo.core.clientImpl.thrift.ClientService.Client implements Iface {
@@ -1091,6 +1103,73 @@ public class TabletClientService {
       throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "contiuneGetSummaries failed: unknown result");
     }
 
+    public java.util.List<CompactionQueueSummary> getCompactionQueueInfo() throws org.apache.thrift.TException
+    {
+      send_getCompactionQueueInfo();
+      return recv_getCompactionQueueInfo();
+    }
+
+    public void send_getCompactionQueueInfo() throws org.apache.thrift.TException
+    {
+      getCompactionQueueInfo_args args = new getCompactionQueueInfo_args();
+      sendBase("getCompactionQueueInfo", args);
+    }
+
+    public java.util.List<CompactionQueueSummary> recv_getCompactionQueueInfo() throws org.apache.thrift.TException
+    {
+      getCompactionQueueInfo_result result = new getCompactionQueueInfo_result();
+      receiveBase(result, "getCompactionQueueInfo");
+      if (result.isSetSuccess()) {
+        return result.success;
+      }
+      throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "getCompactionQueueInfo failed: unknown result");
+    }
+
+    public CompactionJob reserveCompactionJob(java.lang.String queueName, long priority, java.lang.String compactor) throws org.apache.thrift.TException
+    {
+      send_reserveCompactionJob(queueName, priority, compactor);
+      return recv_reserveCompactionJob();
+    }
+
+    public void send_reserveCompactionJob(java.lang.String queueName, long priority, java.lang.String compactor) throws org.apache.thrift.TException
+    {
+      reserveCompactionJob_args args = new reserveCompactionJob_args();
+      args.setQueueName(queueName);
+      args.setPriority(priority);
+      args.setCompactor(compactor);
+      sendBase("reserveCompactionJob", args);
+    }
+
+    public CompactionJob recv_reserveCompactionJob() throws org.apache.thrift.TException
+    {
+      reserveCompactionJob_result result = new reserveCompactionJob_result();
+      receiveBase(result, "reserveCompactionJob");
+      if (result.isSetSuccess()) {
+        return result.success;
+      }
+      throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "reserveCompactionJob failed: unknown result");
+    }
+
+    public void compactionJobFinished(CompactionJob job) throws org.apache.thrift.TException
+    {
+      send_compactionJobFinished(job);
+      recv_compactionJobFinished();
+    }
+
+    public void send_compactionJobFinished(CompactionJob job) throws org.apache.thrift.TException
+    {
+      compactionJobFinished_args args = new compactionJobFinished_args();
+      args.setJob(job);
+      sendBase("compactionJobFinished", args);
+    }
+
+    public void recv_compactionJobFinished() throws org.apache.thrift.TException
+    {
+      compactionJobFinished_result result = new compactionJobFinished_result();
+      receiveBase(result, "compactionJobFinished");
+      return;
+    }
+
   }
   public static class AsyncClient extends org.apache.accumulo.core.clientImpl.thrift.ClientService.AsyncClient implements AsyncIface {
     public static class Factory implements org.apache.thrift.async.TAsyncClientFactory<AsyncClient> {
@@ -2588,6 +2667,105 @@ public class TabletClientService {
       }
     }
 
+    public void getCompactionQueueInfo(org.apache.thrift.async.AsyncMethodCallback<java.util.List<CompactionQueueSummary>> resultHandler) throws org.apache.thrift.TException {
+      checkReady();
+      getCompactionQueueInfo_call method_call = new getCompactionQueueInfo_call(resultHandler, this, ___protocolFactory, ___transport);
+      this.___currentMethod = method_call;
+      ___manager.call(method_call);
+    }
+
+    public static class getCompactionQueueInfo_call extends org.apache.thrift.async.TAsyncMethodCall<java.util.List<CompactionQueueSummary>> {
+      public getCompactionQueueInfo_call(org.apache.thrift.async.AsyncMethodCallback<java.util.List<CompactionQueueSummary>> 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);
+      }
+
+      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("getCompactionQueueInfo", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        getCompactionQueueInfo_args args = new getCompactionQueueInfo_args();
+        args.write(prot);
+        prot.writeMessageEnd();
+      }
+
+      public java.util.List<CompactionQueueSummary> 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_getCompactionQueueInfo();
+      }
+    }
+
+    public void reserveCompactionJob(java.lang.String queueName, long priority, java.lang.String compactor, org.apache.thrift.async.AsyncMethodCallback<CompactionJob> resultHandler) throws org.apache.thrift.TException {
+      checkReady();
+      reserveCompactionJob_call method_call = new reserveCompactionJob_call(queueName, priority, compactor, resultHandler, this, ___protocolFactory, ___transport);
+      this.___currentMethod = method_call;
+      ___manager.call(method_call);
+    }
+
+    public static class reserveCompactionJob_call extends org.apache.thrift.async.TAsyncMethodCall<CompactionJob> {
+      private java.lang.String queueName;
+      private long priority;
+      private java.lang.String compactor;
+      public reserveCompactionJob_call(java.lang.String queueName, long priority, java.lang.String compactor, org.apache.thrift.async.AsyncMethodCallback<CompactionJob> 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.queueName = queueName;
+        this.priority = priority;
+        this.compactor = compactor;
+      }
+
+      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("reserveCompactionJob", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        reserveCompactionJob_args args = new reserveCompactionJob_args();
+        args.setQueueName(queueName);
+        args.setPriority(priority);
+        args.setCompactor(compactor);
+        args.write(prot);
+        prot.writeMessageEnd();
+      }
+
+      public CompactionJob 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_reserveCompactionJob();
+      }
+    }
+
+    public void compactionJobFinished(CompactionJob job, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
+      checkReady();
+      compactionJobFinished_call method_call = new compactionJobFinished_call(job, resultHandler, this, ___protocolFactory, ___transport);
+      this.___currentMethod = method_call;
+      ___manager.call(method_call);
+    }
+
+    public static class compactionJobFinished_call extends org.apache.thrift.async.TAsyncMethodCall<Void> {
+      private CompactionJob job;
+      public compactionJobFinished_call(CompactionJob job, 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.job = job;
+      }
+
+      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("compactionJobFinished", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        compactionJobFinished_args args = new compactionJobFinished_args();
+        args.setJob(job);
+        args.write(prot);
+        prot.writeMessageEnd();
+      }
+
+      public Void 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 null;
+      }
+    }
+
   }
 
   public static class Processor<I extends Iface> extends org.apache.accumulo.core.clientImpl.thrift.ClientService.Processor<I> implements org.apache.thrift.TProcessor {
@@ -2637,6 +2815,9 @@ public class TabletClientService {
       processMap.put("startGetSummariesForPartition", new startGetSummariesForPartition());
       processMap.put("startGetSummariesFromFiles", new startGetSummariesFromFiles());
       processMap.put("contiuneGetSummaries", new contiuneGetSummaries());
+      processMap.put("getCompactionQueueInfo", new getCompactionQueueInfo());
+      processMap.put("reserveCompactionJob", new reserveCompactionJob());
+      processMap.put("compactionJobFinished", new compactionJobFinished());
       return processMap;
     }
 
@@ -3641,6 +3822,81 @@ public class TabletClientService {
       }
     }
 
+    public static class getCompactionQueueInfo<I extends Iface> extends org.apache.thrift.ProcessFunction<I, getCompactionQueueInfo_args> {
+      public getCompactionQueueInfo() {
+        super("getCompactionQueueInfo");
+      }
+
+      public getCompactionQueueInfo_args getEmptyArgsInstance() {
+        return new getCompactionQueueInfo_args();
+      }
+
+      protected boolean isOneway() {
+        return false;
+      }
+
+      @Override
+      protected boolean rethrowUnhandledExceptions() {
+        return false;
+      }
+
+      public getCompactionQueueInfo_result getResult(I iface, getCompactionQueueInfo_args args) throws org.apache.thrift.TException {
+        getCompactionQueueInfo_result result = new getCompactionQueueInfo_result();
+        result.success = iface.getCompactionQueueInfo();
+        return result;
+      }
+    }
+
+    public static class reserveCompactionJob<I extends Iface> extends org.apache.thrift.ProcessFunction<I, reserveCompactionJob_args> {
+      public reserveCompactionJob() {
+        super("reserveCompactionJob");
+      }
+
+      public reserveCompactionJob_args getEmptyArgsInstance() {
+        return new reserveCompactionJob_args();
+      }
+
+      protected boolean isOneway() {
+        return false;
+      }
+
+      @Override
+      protected boolean rethrowUnhandledExceptions() {
+        return false;
+      }
+
+      public reserveCompactionJob_result getResult(I iface, reserveCompactionJob_args args) throws org.apache.thrift.TException {
+        reserveCompactionJob_result result = new reserveCompactionJob_result();
+        result.success = iface.reserveCompactionJob(args.queueName, args.priority, args.compactor);
+        return result;
+      }
+    }
+
+    public static class compactionJobFinished<I extends Iface> extends org.apache.thrift.ProcessFunction<I, compactionJobFinished_args> {
+      public compactionJobFinished() {
+        super("compactionJobFinished");
+      }
+
+      public compactionJobFinished_args getEmptyArgsInstance() {
+        return new compactionJobFinished_args();
+      }
+
+      protected boolean isOneway() {
+        return false;
+      }
+
+      @Override
+      protected boolean rethrowUnhandledExceptions() {
+        return false;
+      }
+
+      public compactionJobFinished_result getResult(I iface, compactionJobFinished_args args) throws org.apache.thrift.TException {
+        compactionJobFinished_result result = new compactionJobFinished_result();
+        iface.compactionJobFinished(args.job);
+        return result;
+      }
+    }
+
   }
 
   public static class AsyncProcessor<I extends AsyncIface> extends org.apache.accumulo.core.clientImpl.thrift.ClientService.AsyncProcessor<I> {
@@ -3690,6 +3946,9 @@ public class TabletClientService {
       processMap.put("startGetSummariesForPartition", new startGetSummariesForPartition());
       processMap.put("startGetSummariesFromFiles", new startGetSummariesFromFiles());
       processMap.put("contiuneGetSummaries", new contiuneGetSummaries());
+      processMap.put("getCompactionQueueInfo", new getCompactionQueueInfo());
+      processMap.put("reserveCompactionJob", new reserveCompactionJob());
+      processMap.put("compactionJobFinished", new compactionJobFinished());
       return processMap;
     }
 
@@ -5697,6 +5956,188 @@ public class TabletClientService {
       }
     }
 
+    public static class getCompactionQueueInfo<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, getCompactionQueueInfo_args, java.util.List<CompactionQueueSummary>> {
+      public getCompactionQueueInfo() {
+        super("getCompactionQueueInfo");
+      }
+
+      public getCompactionQueueInfo_args getEmptyArgsInstance() {
+        return new getCompactionQueueInfo_args();
+      }
+
+      public org.apache.thrift.async.AsyncMethodCallback<java.util.List<CompactionQueueSummary>> 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<CompactionQueueSummary>>() { 
+          public void onComplete(java.util.List<CompactionQueueSummary> o) {
+            getCompactionQueueInfo_result result = new getCompactionQueueInfo_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;
+            getCompactionQueueInfo_result result = new getCompactionQueueInfo_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, getCompactionQueueInfo_args args, org.apache.thrift.async.AsyncMethodCallback<java.util.List<CompactionQueueSummary>> resultHandler) throws org.apache.thrift.TException {
+        iface.getCompactionQueueInfo(resultHandler);
+      }
+    }
+
+    public static class reserveCompactionJob<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, reserveCompactionJob_args, CompactionJob> {
+      public reserveCompactionJob() {
+        super("reserveCompactionJob");
+      }
+
+      public reserveCompactionJob_args getEmptyArgsInstance() {
+        return new reserveCompactionJob_args();
+      }
+
+      public org.apache.thrift.async.AsyncMethodCallback<CompactionJob> 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<CompactionJob>() { 
+          public void onComplete(CompactionJob o) {
+            reserveCompactionJob_result result = new reserveCompactionJob_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;
+            reserveCompactionJob_result result = new reserveCompactionJob_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, reserveCompactionJob_args args, org.apache.thrift.async.AsyncMethodCallback<CompactionJob> resultHandler) throws org.apache.thrift.TException {
+        iface.reserveCompactionJob(args.queueName, args.priority, args.compactor,resultHandler);
+      }
+    }
+
+    public static class compactionJobFinished<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, compactionJobFinished_args, Void> {
+      public compactionJobFinished() {
+        super("compactionJobFinished");
+      }
+
+      public compactionJobFinished_args getEmptyArgsInstance() {
+        return new compactionJobFinished_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) {
+            compactionJobFinished_result result = new compactionJobFinished_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;
+            compactionJobFinished_result result = new compactionJobFinished_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, compactionJobFinished_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
+        iface.compactionJobFinished(args.job,resultHandler);
+      }
+    }
+
   }
 
   public static class startScan_args implements org.apache.thrift.TBase<startScan_args, startScan_args._Fields>, java.io.Serializable, Cloneable, Comparable<startScan_args>   {
@@ -7349,14 +7790,14 @@ public class TabletClientService {
             case 4: // COLUMNS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list116 = iprot.readListBegin();
-                  struct.columns = new java.util.ArrayList<org.apache.accumulo.core.dataImpl.thrift.TColumn>(_list116.size);
-                  @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TColumn _elem117;
-                  for (int _i118 = 0; _i118 < _list116.size; ++_i118)
+                  org.apache.thrift.protocol.TList _list124 = iprot.readListBegin();
+                  struct.columns = new java.util.ArrayList<org.apache.accumulo.core.dataImpl.thrift.TColumn>(_list124.size);
+                  @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TColumn _elem125;
+                  for (int _i126 = 0; _i126 < _list124.size; ++_i126)
                   {
-                    _elem117 = new org.apache.accumulo.core.dataImpl.thrift.TColumn();
-                    _elem117.read(iprot);
-                    struct.columns.add(_elem117);
+                    _elem125 = new org.apache.accumulo.core.dataImpl.thrift.TColumn();
+                    _elem125.read(iprot);
+                    struct.columns.add(_elem125);
                   }
                   iprot.readListEnd();
                 }
@@ -7376,14 +7817,14 @@ public class TabletClientService {
             case 6: // SSI_LIST
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list119 = iprot.readListBegin();
-                  struct.ssiList = new java.util.ArrayList<org.apache.accumulo.core.dataImpl.thrift.IterInfo>(_list119.size);
-                  @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.IterInfo _elem120;
-                  for (int _i121 = 0; _i121 < _list119.size; ++_i121)
+                  org.apache.thrift.protocol.TList _list127 = iprot.readListBegin();
+                  struct.ssiList = new java.util.ArrayList<org.apache.accumulo.core.dataImpl.thrift.IterInfo>(_list127.size);
+                  @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.IterInfo _elem128;
+                  for (int _i129 = 0; _i129 < _list127.size; ++_i129)
                   {
-                    _elem120 = new org.apache.accumulo.core.dataImpl.thrift.IterInfo();
-                    _elem120.read(iprot);
-                    struct.ssiList.add(_elem120);
+                    _elem128 = new org.apache.accumulo.core.dataImpl.thrift.IterInfo();
+                    _elem128.read(iprot);
+                    struct.ssiList.add(_elem128);
                   }
                   iprot.readListEnd();
                 }
@@ -7395,27 +7836,27 @@ public class TabletClientService {
             case 7: // SSIO
               if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
                 {
-                  org.apache.thrift.protocol.TMap _map122 = iprot.readMapBegin();
-                  struct.ssio = new java.util.HashMap<java.lang.String,java.util.Map<java.lang.String,java.lang.String>>(2*_map122.size);
-                  @org.apache.thrift.annotation.Nullable java.lang.String _key123;
-                  @org.apache.thrift.annotation.Nullable java.util.Map<java.lang.String,java.lang.String> _val124;
-                  for (int _i125 = 0; _i125 < _map122.size; ++_i125)
+                  org.apache.thrift.protocol.TMap _map130 = iprot.readMapBegin();
+                  struct.ssio = new java.util.HashMap<java.lang.String,java.util.Map<java.lang.String,java.lang.String>>(2*_map130.size);
+                  @org.apache.thrift.annotation.Nullable java.lang.String _key131;
+                  @org.apache.thrift.annotation.Nullable java.util.Map<java.lang.String,java.lang.String> _val132;
+                  for (int _i133 = 0; _i133 < _map130.size; ++_i133)
                   {
-                    _key123 = iprot.readString();
+                    _key131 = iprot.readString();
                     {
-                      org.apache.thrift.protocol.TMap _map126 = iprot.readMapBegin();
-                      _val124 = new java.util.HashMap<java.lang.String,java.lang.String>(2*_map126.size);
-                      @org.apache.thrift.annotation.Nullable java.lang.String _key127;
-                      @org.apache.thrift.annotation.Nullable java.lang.String _val128;
-                      for (int _i129 = 0; _i129 < _map126.size; ++_i129)
+                      org.apache.thrift.protocol.TMap _map134 = iprot.readMapBegin();
+                      _val132 = new java.util.HashMap<java.lang.String,java.lang.String>(2*_map134.size);
+                      @org.apache.thrift.annotation.Nullable java.lang.String _key135;
+                      @org.apache.thrift.annotation.Nullable java.lang.String _val136;
+                      for (int _i137 = 0; _i137 < _map134.size; ++_i137)
                       {
-                        _key127 = iprot.readString();
-                        _val128 = iprot.readString();
-                        _val124.put(_key127, _val128);
+                        _key135 = iprot.readString();
+                        _val136 = iprot.readString();
+                        _val132.put(_key135, _val136);
                       }
                       iprot.readMapEnd();
                     }
-                    struct.ssio.put(_key123, _val124);
+                    struct.ssio.put(_key131, _val132);
                   }
                   iprot.readMapEnd();
                 }
@@ -7427,13 +7868,13 @@ public class TabletClientService {
             case 8: // AUTHORIZATIONS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list130 = iprot.readListBegin();
-                  struct.authorizations = new java.util.ArrayList<java.nio.ByteBuffer>(_list130.size);
-                  @org.apache.thrift.annotation.Nullable java.nio.ByteBuffer _elem131;
-                  for (int _i132 = 0; _i132 < _list130.size; ++_i132)
+                  org.apache.thrift.protocol.TList _list138 = iprot.readListBegin();
+                  struct.authorizations = new java.util.ArrayList<java.nio.ByteBuffer>(_list138.size);
+                  @org.apache.thrift.annotation.Nullable java.nio.ByteBuffer _elem139;
+                  for (int _i140 = 0; _i140 < _list138.size; ++_i140)
                   {
-                    _elem131 = iprot.readBinary();
-                    struct.authorizations.add(_elem131);
+                    _elem139 = iprot.readBinary();
+                    struct.authorizations.add(_elem139);
                   }
                   iprot.readListEnd();
                 }
@@ -7494,15 +7935,15 @@ public class TabletClientService {
             case 16: // EXECUTION_HINTS
               if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
                 {
-                  org.apache.thrift.protocol.TMap _map133 = iprot.readMapBegin();
-                  struct.executionHints = new java.util.HashMap<java.lang.String,java.lang.String>(2*_map133.size);
-                  @org.apache.thrift.annotation.Nullable java.lang.String _key134;
-                  @org.apache.thrift.annotation.Nullable java.lang.String _val135;
-                  for (int _i136 = 0; _i136 < _map133.size; ++_i136)
+                  org.apache.thrift.protocol.TMap _map141 = iprot.readMapBegin();
+                  struct.executionHints = new java.util.HashMap<java.lang.String,java.lang.String>(2*_map141.size);
+                  @org.apache.thrift.annotation.Nullable java.lang.String _key142;
+                  @org.apache.thrift.annotation.Nullable java.lang.String _val143;
+                  for (int _i144 = 0; _i144 < _map141.size; ++_i144)
                   {
-                    _key134 = iprot.readString();
-                    _val135 = iprot.readString();
-                    struct.executionHints.put(_key134, _val135);
+                    _key142 = iprot.readString();
+                    _val143 = iprot.readString();
+                    struct.executionHints.put(_key142, _val143);
                   }
                   iprot.readMapEnd();
                 }
@@ -7545,9 +7986,9 @@ public class TabletClientService {
           oprot.writeFieldBegin(COLUMNS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.columns.size()));
-            for (org.apache.accumulo.core.dataImpl.thrift.TColumn _iter137 : struct.columns)
+            for (org.apache.accumulo.core.dataImpl.thrift.TColumn _iter145 : struct.columns)
             {
-              _iter137.write(oprot);
+              _iter145.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -7560,9 +8001,9 @@ public class TabletClientService {
           oprot.writeFieldBegin(SSI_LIST_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.ssiList.size()));
-            for (org.apache.accumulo.core.dataImpl.thrift.IterInfo _iter138 : struct.ssiList)
+            for (org.apache.accumulo.core.dataImpl.thrift.IterInfo _iter146 : struct.ssiList)
             {
-              _iter138.write(oprot);
+              _iter146.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -7572,15 +8013,15 @@ public class TabletClientService {
           oprot.writeFieldBegin(SSIO_FIELD_DESC);
           {
             oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.MAP, struct.ssio.size()));
-            for (java.util.Map.Entry<java.lang.String, java.util.Map<java.lang.String,java.lang.String>> _iter139 : struct.ssio.entrySet())
+            for (java.util.Map.Entry<java.lang.String, java.util.Map<java.lang.String,java.lang.String>> _iter147 : struct.ssio.entrySet())
             {
-              oprot.writeString(_iter139.getKey());
+              oprot.writeString(_iter147.getKey());
               {
-                oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, _iter139.getValue().size()));
-                for (java.util.Map.Entry<java.lang.String, java.lang.String> _iter140 : _iter139.getValue().entrySet())
+                oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, _iter147.getValue().size()));
+                for (java.util.Map.Entry<java.lang.String, java.lang.String> _iter148 : _iter147.getValue().entrySet())
                 {
-                  oprot.writeString(_iter140.getKey());
-                  oprot.writeString(_iter140.getValue());
+                  oprot.writeString(_iter148.getKey());
+                  oprot.writeString(_iter148.getValue());
                 }
                 oprot.writeMapEnd();
               }
@@ -7593,9 +8034,9 @@ public class TabletClientService {
           oprot.writeFieldBegin(AUTHORIZATIONS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.authorizations.size()));
-            for (java.nio.ByteBuffer _iter141 : struct.authorizations)
+            for (java.nio.ByteBuffer _iter149 : struct.authorizations)
             {
-              oprot.writeBinary(_iter141);
+              oprot.writeBinary(_iter149);
             }
             oprot.writeListEnd();
           }
@@ -7632,10 +8073,10 @@ public class TabletClientService {
           oprot.writeFieldBegin(EXECUTION_HINTS_FIELD_DESC);
           {
             oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, struct.executionHints.size()));
-            for (java.util.Map.Entry<java.lang.String, java.lang.String> _iter142 : struct.executionHints.entrySet())
+            for (java.util.Map.Entry<java.lang.String, java.lang.String> _iter150 : struct.executionHints.entrySet())
             {
-              oprot.writeString(_iter142.getKey());
-              oprot.writeString(_iter142.getValue());
+              oprot.writeString(_iter150.getKey());
+              oprot.writeString(_iter150.getValue());
             }
             oprot.writeMapEnd();
           }
@@ -7723,9 +8164,9 @@ public class TabletClientService {
         if (struct.isSetColumns()) {
           {
             oprot.writeI32(struct.columns.size());
-            for (org.apache.accumulo.core.dataImpl.thrift.TColumn _iter143 : struct.columns)
+            for (org.apache.accumulo.core.dataImpl.thrift.TColumn _iter151 : struct.columns)
             {
-              _iter143.write(oprot);
+              _iter151.write(oprot);
             }
           }
         }
@@ -7735,24 +8176,24 @@ public class TabletClientService {
         if (struct.isSetSsiList()) {
           {
             oprot.writeI32(struct.ssiList.size());
-            for (org.apache.accumulo.core.dataImpl.thrift.IterInfo _iter144 : struct.ssiList)
+            for (org.apache.accumulo.core.dataImpl.thrift.IterInfo _iter152 : struct.ssiList)
             {
-              _iter144.write(oprot);
+              _iter152.write(oprot);
             }
           }
         }
         if (struct.isSetSsio()) {
           {
             oprot.writeI32(struct.ssio.size());
-            for (java.util.Map.Entry<java.lang.String, java.util.Map<java.lang.String,java.lang.String>> _iter145 : struct.ssio.entrySet())
+            for (java.util.Map.Entry<java.lang.String, java.util.Map<java.lang.String,java.lang.String>> _iter153 : struct.ssio.entrySet())
             {
-              oprot.writeString(_iter145.getKey());
+              oprot.writeString(_iter153.getKey());
               {
-                oprot.writeI32(_iter145.getValue().size());
-                for (java.util.Map.Entry<java.lang.String, java.lang.String> _iter146 : _iter145.getValue().entrySet())
+                oprot.writeI32(_iter153.getValue().size());
+                for (java.util.Map.Entry<java.lang.String, java.lang.String> _iter154 : _iter153.getValue().entrySet())
                 {
-                  oprot.writeString(_iter146.getKey());
-                  oprot.writeString(_iter146.getValue());
+                  oprot.writeString(_iter154.getKey());
+                  oprot.writeString(_iter154.getValue());
                 }
               }
             }
@@ -7761,9 +8202,9 @@ public class TabletClientService {
         if (struct.isSetAuthorizations()) {
           {
             oprot.writeI32(struct.authorizations.size());
-            for (java.nio.ByteBuffer _iter147 : struct.authorizations)
+            for (java.nio.ByteBuffer _iter155 : struct.authorizations)
             {
-              oprot.writeBinary(_iter147);
+              oprot.writeBinary(_iter155);
             }
           }
         }
@@ -7788,10 +8229,10 @@ public class TabletClientService {
         if (struct.isSetExecutionHints()) {
           {
             oprot.writeI32(struct.executionHints.size());
-            for (java.util.Map.Entry<java.lang.String, java.lang.String> _iter148 : struct.executionHints.entrySet())
+            for (java.util.Map.Entry<java.lang.String, java.lang.String> _iter156 : struct.executionHints.entrySet())
             {
-              oprot.writeString(_iter148.getKey());
-              oprot.writeString(_iter148.getValue());
+              oprot.writeString(_iter156.getKey());
+              oprot.writeString(_iter156.getValue());
             }
           }
         }
@@ -7823,14 +8264,14 @@ public class TabletClientService {
         }
         if (incoming.get(4)) {
           {
-            org.apache.thrift.protocol.TList _list149 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.columns = new java.util.ArrayList<org.apache.accumulo.core.dataImpl.thrift.TColumn>(_list149.size);
-            @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TColumn _elem150;
-            for (int _i151 = 0; _i151 < _list149.size; ++_i151)
+            org.apache.thrift.protocol.TList _list157 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.columns = new java.util.ArrayList<org.apache.accumulo.core.dataImpl.thrift.TColumn>(_list157.size);
+            @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TColumn _elem158;
+            for (int _i159 = 0; _i159 < _list157.size; ++_i159)
             {
-              _elem150 = new org.apache.accumulo.core.dataImpl.thrift.TColumn();
-              _elem150.read(iprot);
-              struct.columns.add(_elem150);
+              _elem158 = new org.apache.accumulo.core.dataImpl.thrift.TColumn();
+              _elem158.read(iprot);
+              struct.columns.add(_elem158);
             }
           }
           struct.setColumnsIsSet(true);
@@ -7841,53 +8282,53 @@ public class TabletClientService {
         }
         if (incoming.get(6)) {
           {
-            org.apache.thrift.protocol.TList _list152 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.ssiList = new java.util.ArrayList<org.apache.accumulo.core.dataImpl.thrift.IterInfo>(_list152.size);
-            @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.IterInfo _elem153;
-            for (int _i154 = 0; _i154 < _list152.size; ++_i154)
+            org.apache.thrift.protocol.TList _list160 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.ssiList = new java.util.ArrayList<org.apache.accumulo.core.dataImpl.thrift.IterInfo>(_list160.size);
+            @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.IterInfo _elem161;
+            for (int _i162 = 0; _i162 < _list160.size; ++_i162)
             {
-              _elem153 = new org.apache.accumulo.core.dataImpl.thrift.IterInfo();
-              _elem153.read(iprot);
-              struct.ssiList.add(_elem153);
+              _elem161 = new org.apache.accumulo.core.dataImpl.thrift.IterInfo();
+              _elem161.read(iprot);
+              struct.ssiList.add(_elem161);
             }
           }
           struct.setSsiListIsSet(true);
         }
         if (incoming.get(7)) {
           {
-            org.apache.thrift.protocol.TMap _map155 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.MAP, iprot.readI32());
-            struct.ssio = new java.util.HashMap<java.lang.String,java.util.Map<java.lang.String,java.lang.String>>(2*_map155.size);
-            @org.apache.thrift.annotation.Nullable java.lang.String _key156;
-            @org.apache.thrift.annotation.Nullable java.util.Map<java.lang.String,java.lang.String> _val157;
-            for (int _i158 = 0; _i158 < _map155.size; ++_i158)
+            org.apache.thrift.protocol.TMap _map163 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.MAP, iprot.readI32());
+            struct.ssio = new java.util.HashMap<java.lang.String,java.util.Map<java.lang.String,java.lang.String>>(2*_map163.size);
+            @org.apache.thrift.annotation.Nullable java.lang.String _key164;
+            @org.apache.thrift.annotation.Nullable java.util.Map<java.lang.String,java.lang.String> _val165;
+            for (int _i166 = 0; _i166 < _map163.size; ++_i166)
             {
-              _key156 = iprot.readString();
+              _key164 = iprot.readString();
               {
-                org.apache.thrift.protocol.TMap _map159 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-                _val157 = new java.util.HashMap<java.lang.String,java.lang.String>(2*_map159.size);
-                @org.apache.thrift.annotation.Nullable java.lang.String _key160;
-                @org.apache.thrift.annotation.Nullable java.lang.String _val161;
-                for (int _i162 = 0; _i162 < _map159.size; ++_i162)
+                org.apache.thrift.protocol.TMap _map167 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+                _val165 = new java.util.HashMap<java.lang.String,java.lang.String>(2*_map167.size);
+                @org.apache.thrift.annotation.Nullable java.lang.String _key168;
+                @org.apache.thrift.annotation.Nullable java.lang.String _val169;
+                for (int _i170 = 0; _i170 < _map167.size; ++_i170)
                 {
-                  _key160 = iprot.readString();
-                  _val161 = iprot.readString();
-                  _val157.put(_key160, _val161);
+                  _key168 = iprot.readString();
+                  _val169 = iprot.readString();
+                  _val165.put(_key168, _val169);
                 }
               }
-              struct.ssio.put(_key156, _val157);
+              struct.ssio.put(_key164, _val165);
             }
           }
           struct.setSsioIsSet(true);
         }
         if (incoming.get(8)) {
           {
-            org.apache.thrift.protocol.TList _list163 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.authorizations = new java.util.ArrayList<java.nio.ByteBuffer>(_list163.size);
-            @org.apache.thrift.annotation.Nullable java.nio.ByteBuffer _elem164;
-            for (int _i165 = 0; _i165 < _list163.size; ++_i165)
+            org.apache.thrift.protocol.TList _list171 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.authorizations = new java.util.ArrayList<java.nio.ByteBuffer>(_list171.size);
+            @org.apache.thrift.annotation.Nullable java.nio.ByteBuffer _elem172;
+            for (int _i173 = 0; _i173 < _list171.size; ++_i173)
             {
-              _elem164 = iprot.readBinary();
-              struct.authorizations.add(_elem164);
+              _elem172 = iprot.readBinary();
+              struct.authorizations.add(_elem172);
             }
           }
           struct.setAuthorizationsIsSet(true);
@@ -7919,15 +8360,15 @@ public class TabletClientService {
         }
         if (incoming.get(15)) {
           {
-            org.apache.thrift.protocol.TMap _map166 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.executionHints = new java.util.HashMap<java.lang.String,java.lang.String>(2*_map166.size);
-            @org.apache.thrift.annotation.Nullable java.lang.String _key167;
-            @org.apache.thrift.annotation.Nullable java.lang.String _val168;
-            for (int _i169 = 0; _i169 < _map166.size; ++_i169)
+            org.apache.thrift.protocol.TMap _map174 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.executionHints = new java.util.HashMap<java.lang.String,java.lang.String>(2*_map174.size);
+            @org.apache.thrift.annotation.Nullable java.lang.String _key175;
+            @org.apache.thrift.annotation.Nullable java.lang.String _val176;
+            for (int _i177 = 0; _i177 < _map174.size; ++_i177)
             {
-              _key167 = iprot.readString();
-              _val168 = iprot.readString();
-              struct.executionHints.put(_key167, _val168);
+              _key175 = iprot.readString();
+              _val176 = iprot.readString();
+              struct.executionHints.put(_key175, _val176);
             }
           }
           struct.setExecutionHintsIsSet(true);
@@ -11829,27 +12270,27 @@ public class TabletClientService {
             case 2: // BATCH
               if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
                 {
-                  org.apache.thrift.protocol.TMap _map170 = iprot.readMapBegin();
-                  struct.batch = new java.util.HashMap<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent,java.util.List<org.apache.accumulo.core.dataImpl.thrift.TRange>>(2*_map170.size);
-                  @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TKeyExtent _key171;
-                  @org.apache.thrift.annotation.Nullable java.util.List<org.apache.accumulo.core.dataImpl.thrift.TRange> _val172;
-                  for (int _i173 = 0; _i173 < _map170.size; ++_i173)
+                  org.apache.thrift.protocol.TMap _map178 = iprot.readMapBegin();
+                  struct.batch = new java.util.HashMap<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent,java.util.List<org.apache.accumulo.core.dataImpl.thrift.TRange>>(2*_map178.size);
+                  @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TKeyExtent _key179;
+                  @org.apache.thrift.annotation.Nullable java.util.List<org.apache.accumulo.core.dataImpl.thrift.TRange> _val180;
+                  for (int _i181 = 0; _i181 < _map178.size; ++_i181)
                   {
-                    _key171 = new org.apache.accumulo.core.dataImpl.thrift.TKeyExtent();
-                    _key171.read(iprot);
+                    _key179 = new org.apache.accumulo.core.dataImpl.thrift.TKeyExtent();
+                    _key179.read(iprot);
                     {
-                      org.apache.thrift.protocol.TList _list174 = iprot.readListBegin();
-                      _val172 = new java.util.ArrayList<org.apache.accumulo.core.dataImpl.thrift.TRange>(_list174.size);
-                      @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TRange _elem175;
-                      for (int _i176 = 0; _i176 < _list174.size; ++_i176)
+                      org.apache.thrift.protocol.TList _list182 = iprot.readListBegin();
+                      _val180 = new java.util.ArrayList<org.apache.accumulo.core.dataImpl.thrift.TRange>(_list182.size);
+                      @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TRange _elem183;
+                      for (int _i184 = 0; _i184 < _list182.size; ++_i184)
                       {
-                        _elem175 = new org.apache.accumulo.core.dataImpl.thrift.TRange();
-                        _elem175.read(iprot);
-                        _val172.add(_elem175);
+                        _elem183 = new org.apache.accumulo.core.dataImpl.thrift.TRange();
+                        _elem183.read(iprot);
+                        _val180.add(_elem183);
                       }
                       iprot.readListEnd();
                     }
-                    struct.batch.put(_key171, _val172);
+                    struct.batch.put(_key179, _val180);
                   }
                   iprot.readMapEnd();
                 }
@@ -11861,14 +12302,14 @@ public class TabletClientService {
             case 3: // COLUMNS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list177 = iprot.readListBegin();
-                  struct.columns = new java.util.ArrayList<org.apache.accumulo.core.dataImpl.thrift.TColumn>(_list177.size);
-                  @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TColumn _elem178;
-                  for (int _i179 = 0; _i179 < _list177.size; ++_i179)
+                  org.apache.thrift.protocol.TList _list185 = iprot.readListBegin();
+                  struct.columns = new java.util.ArrayList<org.apache.accumulo.core.dataImpl.thrift.TColumn>(_list185.size);
+                  @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TColumn _elem186;
+                  for (int _i187 = 0; _i187 < _list185.size; ++_i187)
                   {
-                    _elem178 = new org.apache.accumulo.core.dataImpl.thrift.TColumn();
-                    _elem178.read(iprot);
-                    struct.columns.add(_elem178);
+                    _elem186 = new org.apache.accumulo.core.dataImpl.thrift.TColumn();
+                    _elem186.read(iprot);
+                    struct.columns.add(_elem186);
                   }
                   iprot.readListEnd();
                 }
@@ -11880,14 +12321,14 @@ public class TabletClientService {
             case 4: // SSI_LIST
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list180 = iprot.readListBegin();
-                  struct.ssiList = new java.util.ArrayList<org.apache.accumulo.core.dataImpl.thrift.IterInfo>(_list180.size);
-                  @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.IterInfo _elem181;
-                  for (int _i182 = 0; _i182 < _list180.size; ++_i182)
+                  org.apache.thrift.protocol.TList _list188 = iprot.readListBegin();
+                  struct.ssiList = new java.util.ArrayList<org.apache.accumulo.core.dataImpl.thrift.IterInfo>(_list188.size);
+                  @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.IterInfo _elem189;
+                  for (int _i190 = 0; _i190 < _list188.size; ++_i190)
                   {
-                    _elem181 = new org.apache.accumulo.core.dataImpl.thrift.IterInfo();
-                    _elem181.read(iprot);
-                    struct.ssiList.add(_elem181);
+                    _elem189 = new org.apache.accumulo.core.dataImpl.thrift.IterInfo();
+                    _elem189.read(iprot);
+                    struct.ssiList.add(_elem189);
                   }
                   iprot.readListEnd();
                 }
@@ -11899,27 +12340,27 @@ public class TabletClientService {
             case 5: // SSIO
               if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
                 {
-                  org.apache.thrift.protocol.TMap _map183 = iprot.readMapBegin();
-                  struct.ssio = new java.util.HashMap<java.lang.String,java.util.Map<java.lang.String,java.lang.String>>(2*_map183.size);
-                  @org.apache.thrift.annotation.Nullable java.lang.String _key184;
-                  @org.apache.thrift.annotation.Nullable java.util.Map<java.lang.String,java.lang.String> _val185;
-                  for (int _i186 = 0; _i186 < _map183.size; ++_i186)
+                  org.apache.thrift.protocol.TMap _map191 = iprot.readMapBegin();
+                  struct.ssio = new java.util.HashMap<java.lang.String,java.util.Map<java.lang.String,java.lang.String>>(2*_map191.size);
+                  @org.apache.thrift.annotation.Nullable java.lang.String _key192;
+                  @org.apache.thrift.annotation.Nullable java.util.Map<java.lang.String,java.lang.String> _val193;
+                  for (int _i194 = 0; _i194 < _map191.size; ++_i194)
                   {
-                    _key184 = iprot.readString();
+                    _key192 = iprot.readString();
                     {
-                      org.apache.thrift.protocol.TMap _map187 = iprot.readMapBegin();
-                      _val185 = new java.util.HashMap<java.lang.String,java.lang.String>(2*_map187.size);
-                      @org.apache.thrift.annotation.Nullable java.lang.String _key188;
-                      @org.apache.thrift.annotation.Nullable java.lang.String _val189;
-                      for (int _i190 = 0; _i190 < _map187.size; ++_i190)
+                      org.apache.thrift.protocol.TMap _map195 = iprot.readMapBegin();
+                      _val193 = new java.util.HashMap<java.lang.String,java.lang.String>(2*_map195.size);
+                      @org.apache.thrift.annotation.Nullable java.lang.String _key196;
+                      @org.apache.thrift.annotation.Nullable java.lang.String _val197;
+                      for (int _i198 = 0; _i198 < _map195.size; ++_i198)
                       {
-                        _key188 = iprot.readString();
-                        _val189 = iprot.readString();
-                        _val185.put(_key188, _val189);
+                        _key196 = iprot.readString();
+                        _val197 = iprot.readString();
+                        _val193.put(_key196, _val197);
                       }
                       iprot.readMapEnd();
                     }
-                    struct.ssio.put(_key184, _val185);
+                    struct.ssio.put(_key192, _val193);
                   }
                   iprot.readMapEnd();
                 }
@@ -11931,13 +12372,13 @@ public class TabletClientService {
             case 6: // AUTHORIZATIONS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list191 = iprot.readListBegin();
-                  struct.authorizations = new java.util.ArrayList<java.nio.ByteBuffer>(_list191.size);
-                  @org.apache.thrift.annotation.Nullable java.nio.ByteBuffer _elem192;
-                  for (int _i193 = 0; _i193 < _list191.size; ++_i193)
+                  org.apache.thrift.protocol.TList _list199 = iprot.readListBegin();
+                  struct.authorizations = new java.util.ArrayList<java.nio.ByteBuffer>(_list199.size);
+                  @org.apache.thrift.annotation.Nullable java.nio.ByteBuffer _elem200;
+                  for (int _i201 = 0; _i201 < _list199.size; ++_i201)
                   {
-                    _elem192 = iprot.readBinary();
-                    struct.authorizations.add(_elem192);
+                    _elem200 = iprot.readBinary();
+                    struct.authorizations.add(_elem200);
                   }
                   iprot.readListEnd();
                 }
@@ -11982,15 +12423,15 @@ public class TabletClientService {
             case 12: // EXECUTION_HINTS
               if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
                 {
-                  org.apache.thrift.protocol.TMap _map194 = iprot.readMapBegin();
-                  struct.executionHints = new java.util.HashMap<java.lang.String,java.lang.String>(2*_map194.size);
-                  @org.apache.thrift.annotation.Nullable java.lang.String _key195;
-                  @org.apache.thrift.annotation.Nullable java.lang.String _val196;
-                  for (int _i197 = 0; _i197 < _map194.size; ++_i197)
+                  org.apache.thrift.protocol.TMap _map202 = iprot.readMapBegin();
+                  struct.executionHints = new java.util.HashMap<java.lang.String,java.lang.String>(2*_map202.size);
+                  @org.apache.thrift.annotation.Nullable java.lang.String _key203;
+                  @org.apache.thrift.annotation.Nullable java.lang.String _val204;
+                  for (int _i205 = 0; _i205 < _map202.size; ++_i205)
                   {
-                    _key195 = iprot.readString();
-                    _val196 = iprot.readString();
-                    struct.executionHints.put(_key195, _val196);
+                    _key203 = iprot.readString();
+                    _val204 = iprot.readString();
+                    struct.executionHints.put(_key203, _val204);
                   }
                   iprot.readMapEnd();
                 }
@@ -12023,14 +12464,14 @@ public class TabletClientService {
           oprot.writeFieldBegin(BATCH_FIELD_DESC);
           {
             oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRUCT, org.apache.thrift.protocol.TType.LIST, struct.batch.size()));
-            for (java.util.Map.Entry<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent, java.util.List<org.apache.accumulo.core.dataImpl.thrift.TRange>> _iter198 : struct.batch.entrySet())
+            for (java.util.Map.Entry<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent, java.util.List<org.apache.accumulo.core.dataImpl.thrift.TRange>> _iter206 : struct.batch.entrySet())
             {
-              _iter198.getKey().write(oprot);
+              _iter206.getKey().write(oprot);
               {
-                oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, _iter198.getValue().size()));
-                for (org.apache.accumulo.core.dataImpl.thrift.TRange _iter199 : _iter198.getValue())
+                oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, _iter206.getValue().size()));
+                for (org.apache.accumulo.core.dataImpl.thrift.TRange _iter207 : _iter206.getValue())
                 {
-                  _iter199.write(oprot);
+                  _iter207.write(oprot);
                 }
                 oprot.writeListEnd();
               }
@@ -12043,9 +12484,9 @@ public class TabletClientService {
           oprot.writeFieldBegin(COLUMNS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.columns.size()));
-            for (org.apache.accumulo.core.dataImpl.thrift.TColumn _iter200 : struct.columns)
+            for (org.apache.accumulo.core.dataImpl.thrift.TColumn _iter208 : struct.columns)
             {
-              _iter200.write(oprot);
+              _iter208.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -12055,9 +12496,9 @@ public class TabletClientService {
           oprot.writeFieldBegin(SSI_LIST_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.ssiList.size()));
-            for (org.apache.accumulo.core.dataImpl.thrift.IterInfo _iter201 : struct.ssiList)
+            for (org.apache.accumulo.core.dataImpl.thrift.IterInfo _iter209 : struct.ssiList)
             {
-              _iter201.write(oprot);
+              _iter209.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -12067,15 +12508,15 @@ public class TabletClientService {
           oprot.writeFieldBegin(SSIO_FIELD_DESC);
           {
             oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.MAP, struct.ssio.size()));
-            for (java.util.Map.Entry<java.lang.String, java.util.Map<java.lang.String,java.lang.String>> _iter202 : struct.ssio.entrySet())
+            for (java.util.Map.Entry<java.lang.String, java.util.Map<java.lang.String,java.lang.String>> _iter210 : struct.ssio.entrySet())
             {
-              oprot.writeString(_iter202.getKey());
+              oprot.writeString(_iter210.getKey());
               {
-                oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, _iter202.getValue().size()));
-                for (java.util.Map.Entry<java.lang.String, java.lang.String> _iter203 : _iter202.getValue().entrySet())
+                oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, _iter210.getValue().size()));
+                for (java.util.Map.Entry<java.lang.String, java.lang.String> _iter211 : _iter210.getValue().entrySet())
                 {
-                  oprot.writeString(_iter203.getKey());
-                  oprot.writeString(_iter203.getValue());
+                  oprot.writeString(_iter211.getKey());
+                  oprot.writeString(_iter211.getValue());
                 }
                 oprot.writeMapEnd();
               }
@@ -12088,9 +12529,9 @@ public class TabletClientService {
           oprot.writeFieldBegin(AUTHORIZATIONS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.authorizations.size()));
-            for (java.nio.ByteBuffer _iter204 : struct.authorizations)
+            for (java.nio.ByteBuffer _iter212 : struct.authorizations)
             {
-              oprot.writeBinary(_iter204);
+              oprot.writeBinary(_iter212);
             }
             oprot.writeListEnd();
           }
@@ -12121,10 +12562,10 @@ public class TabletClientService {
           oprot.writeFieldBegin(EXECUTION_HINTS_FIELD_DESC);
           {
             oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, struct.executionHints.size()));
-            for (java.util.Map.Entry<java.lang.String, java.lang.String> _iter205 : struct.executionHints.entrySet())
+            for (java.util.Map.Entry<java.lang.String, java.lang.String> _iter213 : struct.executionHints.entrySet())
             {
-              oprot.writeString(_iter205.getKey());
-              oprot.writeString(_iter205.getValue());
+              oprot.writeString(_iter213.getKey());
+              oprot.writeString(_iter213.getValue());
             }
             oprot.writeMapEnd();
           }
@@ -12194,14 +12635,14 @@ public class TabletClientService {
         if (struct.isSetBatch()) {
           {
             oprot.writeI32(struct.batch.size());
-            for (java.util.Map.Entry<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent, java.util.List<org.apache.accumulo.core.dataImpl.thrift.TRange>> _iter206 : struct.batch.entrySet())
+            for (java.util.Map.Entry<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent, java.util.List<org.apache.accumulo.core.dataImpl.thrift.TRange>> _iter214 : struct.batch.entrySet())
             {
-              _iter206.getKey().write(oprot);
+              _iter214.getKey().write(oprot);
               {
-                oprot.writeI32(_iter206.getValue().size());
-                for (org.apache.accumulo.core.dataImpl.thrift.TRange _iter207 : _iter206.getValue())
+                oprot.writeI32(_iter214.getValue().size());
+                for (org.apache.accumulo.core.dataImpl.thrift.TRange _iter215 : _iter214.getValue())
                 {
-                  _iter207.write(oprot);
+                  _iter215.write(oprot);
                 }
               }
             }
@@ -12210,33 +12651,33 @@ public class TabletClientService {
         if (struct.isSetColumns()) {
           {
             oprot.writeI32(struct.columns.size());
-            for (org.apache.accumulo.core.dataImpl.thrift.TColumn _iter208 : struct.columns)
+            for (org.apache.accumulo.core.dataImpl.thrift.TColumn _iter216 : struct.columns)
             {
-              _iter208.write(oprot);
+              _iter216.write(oprot);
             }
           }
         }
         if (struct.isSetSsiList()) {
           {
             oprot.writeI32(struct.ssiList.size());
-            for (org.apache.accumulo.core.dataImpl.thrift.IterInfo _iter209 : struct.ssiList)
+            for (org.apache.accumulo.core.dataImpl.thrift.IterInfo _iter217 : struct.ssiList)
             {
-              _iter209.write(oprot);
+              _iter217.write(oprot);
             }
           }
         }
         if (struct.isSetSsio()) {
           {
             oprot.writeI32(struct.ssio.size());
-            for (java.util.Map.Entry<java.lang.String, java.util.Map<java.lang.String,java.lang.String>> _iter210 : struct.ssio.entrySet())
+            for (java.util.Map.Entry<java.lang.String, java.util.Map<java.lang.String,java.lang.String>> _iter218 : struct.ssio.entrySet())
             {
-              oprot.writeString(_iter210.getKey());
+              oprot.writeString(_iter218.getKey());
               {
-                oprot.writeI32(_iter210.getValue().size());
-                for (java.util.Map.Entry<java.lang.String, java.lang.String> _iter211 : _iter210.getValue().entrySet())
+                oprot.writeI32(_iter218.getValue().size());
+                for (java.util.Map.Entry<java.lang.String, java.lang.String> _iter219 : _iter218.getValue().entrySet())
                 {
-                  oprot.writeString(_iter211.getKey());
-                  oprot.writeString(_iter211.getValue());
+                  oprot.writeString(_iter219.getKey());
+                  oprot.writeString(_iter219.getValue());
                 }
               }
             }
@@ -12245,9 +12686,9 @@ public class TabletClientService {
         if (struct.isSetAuthorizations()) {
           {
             oprot.writeI32(struct.authorizations.size());
-            for (java.nio.ByteBuffer _iter212 : struct.authorizations)
+            for (java.nio.ByteBuffer _iter220 : struct.authorizations)
             {
-              oprot.writeBinary(_iter212);
+              oprot.writeBinary(_iter220);
             }
           }
         }
@@ -12266,10 +12707,10 @@ public class TabletClientService {
         if (struct.isSetExecutionHints()) {
           {
             oprot.writeI32(struct.executionHints.size());
-            for (java.util.Map.Entry<java.lang.String, java.lang.String> _iter213 : struct.executionHints.entrySet())
+            for (java.util.Map.Entry<java.lang.String, java.lang.String> _iter221 : struct.executionHints.entrySet())
             {
-              oprot.writeString(_iter213.getKey());
-              oprot.writeString(_iter213.getValue());
+              oprot.writeString(_iter221.getKey());
+              oprot.writeString(_iter221.getValue());
             }
           }
         }
@@ -12291,93 +12732,93 @@ public class TabletClientService {
         }
         if (incoming.get(2)) {
           {
-            org.apache.thrift.protocol.TMap _map214 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRUCT, org.apache.thrift.protocol.TType.LIST, iprot.readI32());
-            struct.batch = new java.util.HashMap<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent,java.util.List<org.apache.accumulo.core.dataImpl.thrift.TRange>>(2*_map214.size);
-            @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TKeyExtent _key215;
-            @org.apache.thrift.annotation.Nullable java.util.List<org.apache.accumulo.core.dataImpl.thrift.TRange> _val216;
-            for (int _i217 = 0; _i217 < _map214.size; ++_i217)
+            org.apache.thrift.protocol.TMap _map222 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRUCT, org.apache.thrift.protocol.TType.LIST, iprot.readI32());
+            struct.batch = new java.util.HashMap<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent,java.util.List<org.apache.accumulo.core.dataImpl.thrift.TRange>>(2*_map222.size);
+            @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TKeyExtent _key223;
+            @org.apache.thrift.annotation.Nullable java.util.List<org.apache.accumulo.core.dataImpl.thrift.TRange> _val224;
+            for (int _i225 = 0; _i225 < _map222.size; ++_i225)
             {
-              _key215 = new org.apache.accumulo.core.dataImpl.thrift.TKeyExtent();
-              _key215.read(iprot);
+              _key223 = new org.apache.accumulo.core.dataImpl.thrift.TKeyExtent();
+              _key223.read(iprot);
               {
-                org.apache.thrift.protocol.TList _list218 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-                _val216 = new java.util.ArrayList<org.apache.accumulo.core.dataImpl.thrift.TRange>(_list218.size);
-                @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TRange _elem219;
-                for (int _i220 = 0; _i220 < _list218.size; ++_i220)
+                org.apache.thrift.protocol.TList _list226 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+                _val224 = new java.util.ArrayList<org.apache.accumulo.core.dataImpl.thrift.TRange>(_list226.size);
+                @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TRange _elem227;
+                for (int _i228 = 0; _i228 < _list226.size; ++_i228)
                 {
-                  _elem219 = new org.apache.accumulo.core.dataImpl.thrift.TRange();
-                  _elem219.read(iprot);
-                  _val216.add(_elem219);
+                  _elem227 = new org.apache.accumulo.core.dataImpl.thrift.TRange();
+                  _elem227.read(iprot);
+                  _val224.add(_elem227);
                 }
               }
-              struct.batch.put(_key215, _val216);
+              struct.batch.put(_key223, _val224);
             }
           }
           struct.setBatchIsSet(true);
         }
         if (incoming.get(3)) {
           {
-            org.apache.thrift.protocol.TList _list221 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.columns = new java.util.ArrayList<org.apache.accumulo.core.dataImpl.thrift.TColumn>(_list221.size);
-            @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TColumn _elem222;
-            for (int _i223 = 0; _i223 < _list221.size; ++_i223)
+            org.apache.thrift.protocol.TList _list229 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.columns = new java.util.ArrayList<org.apache.accumulo.core.dataImpl.thrift.TColumn>(_list229.size);
+            @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TColumn _elem230;
+            for (int _i231 = 0; _i231 < _list229.size; ++_i231)
             {
-              _elem222 = new org.apache.accumulo.core.dataImpl.thrift.TColumn();
-              _elem222.read(iprot);
-              struct.columns.add(_elem222);
+              _elem230 = new org.apache.accumulo.core.dataImpl.thrift.TColumn();
+              _elem230.read(iprot);
+              struct.columns.add(_elem230);
             }
           }
           struct.setColumnsIsSet(true);
         }
         if (incoming.get(4)) {
           {
-            org.apache.thrift.protocol.TList _list224 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.ssiList = new java.util.ArrayList<org.apache.accumulo.core.dataImpl.thrift.IterInfo>(_list224.size);
-            @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.IterInfo _elem225;
-            for (int _i226 = 0; _i226 < _list224.size; ++_i226)
+            org.apache.thrift.protocol.TList _list232 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.ssiList = new java.util.ArrayList<org.apache.accumulo.core.dataImpl.thrift.IterInfo>(_list232.size);
+            @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.IterInfo _elem233;
+            for (int _i234 = 0; _i234 < _list232.size; ++_i234)
             {
-              _elem225 = new org.apache.accumulo.core.dataImpl.thrift.IterInfo();
-              _elem225.read(iprot);
-              struct.ssiList.add(_elem225);
+              _elem233 = new org.apache.accumulo.core.dataImpl.thrift.IterInfo();
+              _elem233.read(iprot);
+              struct.ssiList.add(_elem233);
             }
           }
           struct.setSsiListIsSet(true);
         }
         if (incoming.get(5)) {
           {
-            org.apache.thrift.protocol.TMap _map227 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.MAP, iprot.readI32());
-            struct.ssio = new java.util.HashMap<java.lang.String,java.util.Map<java.lang.String,java.lang.String>>(2*_map227.size);
-            @org.apache.thrift.annotation.Nullable java.lang.String _key228;
-            @org.apache.thrift.annotation.Nullable java.util.Map<java.lang.String,java.lang.String> _val229;
-            for (int _i230 = 0; _i230 < _map227.size; ++_i230)
+            org.apache.thrift.protocol.TMap _map235 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.MAP, iprot.readI32());
+            struct.ssio = new java.util.HashMap<java.lang.String,java.util.Map<java.lang.String,java.lang.String>>(2*_map235.size);
+            @org.apache.thrift.annotation.Nullable java.lang.String _key236;
+            @org.apache.thrift.annotation.Nullable java.util.Map<java.lang.String,java.lang.String> _val237;
+            for (int _i238 = 0; _i238 < _map235.size; ++_i238)
             {
-              _key228 = iprot.readString();
+              _key236 = iprot.readString();
               {
-                org.apache.thrift.protocol.TMap _map231 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-                _val229 = new java.util.HashMap<java.lang.String,java.lang.String>(2*_map231.size);
-                @org.apache.thrift.annotation.Nullable java.lang.String _key232;
-                @org.apache.thrift.annotation.Nullable java.lang.String _val233;
-                for (int _i234 = 0; _i234 < _map231.size; ++_i234)
+                org.apache.thrift.protocol.TMap _map239 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+                _val237 = new java.util.HashMap<java.lang.String,java.lang.String>(2*_map239.size);
+                @org.apache.thrift.annotation.Nullable java.lang.String _key240;
+                @org.apache.thrift.annotation.Nullable java.lang.String _val241;
+                for (int _i242 = 0; _i242 < _map239.size; ++_i242)
                 {
-                  _key232 = iprot.readString();
-                  _val233 = iprot.readString();
-                  _val229.put(_key232, _val233);
+                  _key240 = iprot.readString();
+                  _val241 = iprot.readString();
+                  _val237.put(_key240, _val241);
                 }
               }
-              struct.ssio.put(_key228, _val229);
+              struct.ssio.put(_key236, _val237);
             }
           }
           struct.setSsioIsSet(true);
         }
         if (incoming.get(6)) {
           {
-            org.apache.thrift.protocol.TList _list235 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.authorizations = new java.util.ArrayList<java.nio.ByteBuffer>(_list235.size);
-            @org.apache.thrift.annotation.Nullable java.nio.ByteBuffer _elem236;
-            for (int _i237 = 0; _i237 < _list235.size; ++_i237)
+            org.apache.thrift.protocol.TList _list243 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.authorizations = new java.util.ArrayList<java.nio.ByteBuffer>(_list243.size);
+            @org.apache.thrift.annotation.Nullable java.nio.ByteBuffer _elem244;
+            for (int _i245 = 0; _i245 < _list243.size; ++_i245)
             {
-              _elem236 = iprot.readBinary();
-              struct.authorizations.add(_elem236);
+              _elem244 = iprot.readBinary();
+              struct.authorizations.add(_elem244);
             }
           }
           struct.setAuthorizationsIsSet(true);
@@ -12401,15 +12842,15 @@ public class TabletClientService {
         }
         if (incoming.get(11)) {
           {
-            org.apache.thrift.protocol.TMap _map238 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.executionHints = new java.util.HashMap<java.lang.String,java.lang.String>(2*_map238.size);
-            @org.apache.thrift.annotation.Nullable java.lang.String _key239;
-            @org.apache.thrift.annotation.Nullable java.lang.String _val240;
-            for (int _i241 = 0; _i241 < _map238.size; ++_i241)
+            org.apache.thrift.protocol.TMap _map246 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.executionHints = new java.util.HashMap<java.lang.String,java.lang.String>(2*_map246.size);
+            @org.apache.thrift.annotation.Nullable java.lang.String _key247;
+            @org.apache.thrift.annotation.Nullable java.lang.String _val248;
+            for (int _i249 = 0; _i249 < _map246.size; ++_i249)
             {
-              _key239 = iprot.readString();
-              _val240 = iprot.readString();
-              struct.executionHints.put(_key239, _val240);
+              _key247 = iprot.readString();
+              _val248 = iprot.readString();
+              struct.executionHints.put(_key247, _val248);
             }
           }
           struct.setExecutionHintsIsSet(true);
@@ -16576,14 +17017,14 @@ public class TabletClientService {
             case 4: // MUTATIONS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list242 = iprot.readListBegin();
-                  struct.mutations = new java.util.ArrayList<org.apache.accumulo.core.dataImpl.thrift.TMutation>(_list242.size);
-                  @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TMutation _elem243;
-                  for (int _i244 = 0; _i244 < _list242.size; ++_i244)
+                  org.apache.thrift.protocol.TList _list250 = iprot.readListBegin();
+                  struct.mutations = new java.util.ArrayList<org.apache.accumulo.core.dataImpl.thrift.TMutation>(_list250.size);
+                  @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TMutation _elem251;
+                  for (int _i252 = 0; _i252 < _list250.size; ++_i252)
                   {
-                    _elem243 = new org.apache.accumulo.core.dataImpl.thrift.TMutation();
-                    _elem243.read(iprot);
-                    struct.mutations.add(_elem243);
+                    _elem251 = new org.apache.accumulo.core.dataImpl.thrift.TMutation();
+                    _elem251.read(iprot);
+                    struct.mutations.add(_elem251);
                   }
                   iprot.readListEnd();
                 }
@@ -16624,9 +17065,9 @@ public class TabletClientService {
           oprot.writeFieldBegin(MUTATIONS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.mutations.size()));
-            for (org.apache.accumulo.core.dataImpl.thrift.TMutation _iter245 : struct.mutations)
+            for (org.apache.accumulo.core.dataImpl.thrift.TMutation _iter253 : struct.mutations)
             {
-              _iter245.write(oprot);
+              _iter253.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -16675,9 +17116,9 @@ public class TabletClientService {
         if (struct.isSetMutations()) {
           {
             oprot.writeI32(struct.mutations.size());
-            for (org.apache.accumulo.core.dataImpl.thrift.TMutation _iter246 : struct.mutations)
+            for (org.apache.accumulo.core.dataImpl.thrift.TMutation _iter254 : struct.mutations)
             {
-              _iter246.write(oprot);
+              _iter254.write(oprot);
             }
           }
         }
@@ -16703,14 +17144,14 @@ public class TabletClientService {
         }
         if (incoming.get(3)) {
           {
-            org.apache.thrift.protocol.TList _list247 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.mutations = new java.util.ArrayList<org.apache.accumulo.core.dataImpl.thrift.TMutation>(_list247.size);
-            @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TMutation _elem248;
-            for (int _i249 = 0; _i249 < _list247.size; ++_i249)
+            org.apache.thrift.protocol.TList _list255 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.mutations = new java.util.ArrayList<org.apache.accumulo.core.dataImpl.thrift.TMutation>(_list255.size);
+            @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TMutation _elem256;
+            for (int _i257 = 0; _i257 < _list255.size; ++_i257)
             {
-              _elem248 = new org.apache.accumulo.core.dataImpl.thrift.TMutation();
-              _elem248.read(iprot);
-              struct.mutations.add(_elem248);
+              _elem256 = new org.apache.accumulo.core.dataImpl.thrift.TMutation();
+              _elem256.read(iprot);
+              struct.mutations.add(_elem256);
             }
           }
           struct.setMutationsIsSet(true);
@@ -19847,13 +20288,13 @@ public class TabletClientService {
             case 3: // AUTHORIZATIONS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list250 = iprot.readListBegin();
-                  struct.authorizations = new java.util.ArrayList<java.nio.ByteBuffer>(_list250.size);
-                  @org.apache.thrift.annotation.Nullable java.nio.ByteBuffer _elem251;
-                  for (int _i252 = 0; _i252 < _list250.size; ++_i252)
+                  org.apache.thrift.protocol.TList _list258 = iprot.readListBegin();
+                  struct.authorizations = new java.util.ArrayList<java.nio.ByteBuffer>(_list258.size);
+                  @org.apache.thrift.annotation.Nullable java.nio.ByteBuffer _elem259;
+                  for (int _i260 = 0; _i260 < _list258.size; ++_i260)
                   {
-                    _elem251 = iprot.readBinary();
-                    struct.authorizations.add(_elem251);
+                    _elem259 = iprot.readBinary();
+                    struct.authorizations.add(_elem259);
                   }
                   iprot.readListEnd();
                 }
@@ -19915,9 +20356,9 @@ public class TabletClientService {
           oprot.writeFieldBegin(AUTHORIZATIONS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.authorizations.size()));
-            for (java.nio.ByteBuffer _iter253 : struct.authorizations)
+            for (java.nio.ByteBuffer _iter261 : struct.authorizations)
             {
-              oprot.writeBinary(_iter253);
+              oprot.writeBinary(_iter261);
             }
             oprot.writeListEnd();
           }
@@ -19984,9 +20425,9 @@ public class TabletClientService {
         if (struct.isSetAuthorizations()) {
           {
             oprot.writeI32(struct.authorizations.size());
-            for (java.nio.ByteBuffer _iter254 : struct.authorizations)
+            for (java.nio.ByteBuffer _iter262 : struct.authorizations)
             {
-              oprot.writeBinary(_iter254);
+              oprot.writeBinary(_iter262);
             }
           }
         }
@@ -20017,13 +20458,13 @@ public class TabletClientService {
         }
         if (incoming.get(2)) {
           {
-            org.apache.thrift.protocol.TList _list255 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.authorizations = new java.util.ArrayList<java.nio.ByteBuffer>(_list255.size);
-            @org.apache.thrift.annotation.Nullable java.nio.ByteBuffer _elem256;
-            for (int _i257 = 0; _i257 < _list255.size; ++_i257)
+            org.apache.thrift.protocol.TList _list263 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.authorizations = new java.util.ArrayList<java.nio.ByteBuffer>(_list263.size);
+            @org.apache.thrift.annotation.Nullable java.nio.ByteBuffer _elem264;
+            for (int _i265 = 0; _i265 < _list263.size; ++_i265)
             {
-              _elem256 = iprot.readBinary();
-              struct.authorizations.add(_elem256);
+              _elem264 = iprot.readBinary();
+              struct.authorizations.add(_elem264);
             }
           }
           struct.setAuthorizationsIsSet(true);
@@ -21133,27 +21574,27 @@ public class TabletClientService {
             case 3: // MUTATIONS
               if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
                 {
-                  org.apache.thrift.protocol.TMap _map258 = iprot.readMapBegin();
-                  struct.mutations = new java.util.HashMap<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent,java.util.List<org.apache.accumulo.core.dataImpl.thrift.TConditionalMutation>>(2*_map258.size);
-                  @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TKeyExtent _key259;
-                  @org.apache.thrift.annotation.Nullable java.util.List<org.apache.accumulo.core.dataImpl.thrift.TConditionalMutation> _val260;
-                  for (int _i261 = 0; _i261 < _map258.size; ++_i261)
+                  org.apache.thrift.protocol.TMap _map266 = iprot.readMapBegin();
+                  struct.mutations = new java.util.HashMap<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent,java.util.List<org.apache.accumulo.core.dataImpl.thrift.TConditionalMutation>>(2*_map266.size);
+                  @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TKeyExtent _key267;
+                  @org.apache.thrift.annotation.Nullable java.util.List<org.apache.accumulo.core.dataImpl.thrift.TConditionalMutation> _val268;
+                  for (int _i269 = 0; _i269 < _map266.size; ++_i269)
                   {
-                    _key259 = new org.apache.accumulo.core.dataImpl.thrift.TKeyExtent();
-                    _key259.read(iprot);
+                    _key267 = new org.apache.accumulo.core.dataImpl.thrift.TKeyExtent();
+                    _key267.read(iprot);
                     {
-                      org.apache.thrift.protocol.TList _list262 = iprot.readListBegin();
-                      _val260 = new java.util.ArrayList<org.apache.accumulo.core.dataImpl.thrift.TConditionalMutation>(_list262.size);
-                      @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TConditionalMutation _elem263;
-                      for (int _i264 = 0; _i264 < _list262.size; ++_i264)
+                      org.apache.thrift.protocol.TList _list270 = iprot.readListBegin();
+                      _val268 = new java.util.ArrayList<org.apache.accumulo.core.dataImpl.thrift.TConditionalMutation>(_list270.size);
+                      @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TConditionalMutation _elem271;
+                      for (int _i272 = 0; _i272 < _list270.size; ++_i272)
                       {
-                        _elem263 = new org.apache.accumulo.core.dataImpl.thrift.TConditionalMutation();
-                        _elem263.read(iprot);
-                        _val260.add(_elem263);
+                        _elem271 = new org.apache.accumulo.core.dataImpl.thrift.TConditionalMutation();
+                        _elem271.read(iprot);
+                        _val268.add(_elem271);
                       }
                       iprot.readListEnd();
                     }
-                    struct.mutations.put(_key259, _val260);
+                    struct.mutations.put(_key267, _val268);
                   }
                   iprot.readMapEnd();
                 }
@@ -21165,13 +21606,13 @@ public class TabletClientService {
             case 4: // SYMBOLS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list265 = iprot.readListBegin();
-                  struct.symbols = new java.util.ArrayList<java.lang.String>(_list265.size);
-                  @org.apache.thrift.annotation.Nullable java.lang.String _elem266;
-                  for (int _i267 = 0; _i267 < _list265.size; ++_i267)
+                  org.apache.thrift.protocol.TList _list273 = iprot.readListBegin();
+                  struct.symbols = new java.util.ArrayList<java.lang.String>(_list273.size);
+                  @org.apache.thrift.annotation.Nullable java.lang.String _elem274;
+                  for (int _i275 = 0; _i275 < _list273.size; ++_i275)
                   {
-                    _elem266 = iprot.readString();
-                    struct.symbols.add(_elem266);
+                    _elem274 = iprot.readString();
+                    struct.symbols.add(_elem274);
                   }
                   iprot.readListEnd();
                 }
@@ -21207,14 +21648,14 @@ public class TabletClientService {
           oprot.writeFieldBegin(MUTATIONS_FIELD_DESC);
           {
             oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRUCT, org.apache.thrift.protocol.TType.LIST, struct.mutations.size()));
-            for (java.util.Map.Entry<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent, java.util.List<org.apache.accumulo.core.dataImpl.thrift.TConditionalMutation>> _iter268 : struct.mutations.entrySet())
+            for (java.util.Map.Entry<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent, java.util.List<org.apache.accumulo.core.dataImpl.thrift.TConditionalMutation>> _iter276 : struct.mutations.entrySet())
             {
-              _iter268.getKey().write(oprot);
+              _iter276.getKey().write(oprot);
               {
-                oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, _iter268.getValue().size()));
-                for (org.apache.accumulo.core.dataImpl.thrift.TConditionalMutation _iter269 : _iter268.getValue())
+                oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, _iter276.getValue().size()));
+                for (org.apache.accumulo.core.dataImpl.thrift.TConditionalMutation _iter277 : _iter276.getValue())
                 {
-                  _iter269.write(oprot);
+                  _iter277.write(oprot);
                 }
                 oprot.writeListEnd();
               }
@@ -21227,9 +21668,9 @@ public class TabletClientService {
           oprot.writeFieldBegin(SYMBOLS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.symbols.size()));
-            for (java.lang.String _iter270 : struct.symbols)
+            for (java.lang.String _iter278 : struct.symbols)
             {
-              oprot.writeString(_iter270);
+              oprot.writeString(_iter278);
             }
             oprot.writeListEnd();
           }
@@ -21275,14 +21716,14 @@ public class TabletClientService {
         if (struct.isSetMutations()) {
           {
             oprot.writeI32(struct.mutations.size());
-            for (java.util.Map.Entry<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent, java.util.List<org.apache.accumulo.core.dataImpl.thrift.TConditionalMutation>> _iter271 : struct.mutations.entrySet())
+            for (java.util.Map.Entry<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent, java.util.List<org.apache.accumulo.core.dataImpl.thrift.TConditionalMutation>> _iter279 : struct.mutations.entrySet())
             {
-              _iter271.getKey().write(oprot);
+              _iter279.getKey().write(oprot);
               {
-                oprot.writeI32(_iter271.getValue().size());
-                for (org.apache.accumulo.core.dataImpl.thrift.TConditionalMutation _iter272 : _iter271.getValue())
+                oprot.writeI32(_iter279.getValue().size());
+                for (org.apache.accumulo.core.dataImpl.thrift.TConditionalMutation _iter280 : _iter279.getValue())
                 {
-                  _iter272.write(oprot);
+                  _iter280.write(oprot);
                 }
               }
             }
@@ -21291,9 +21732,9 @@ public class TabletClientService {
         if (struct.isSetSymbols()) {
           {
             oprot.writeI32(struct.symbols.size());
-            for (java.lang.String _iter273 : struct.symbols)
+            for (java.lang.String _iter281 : struct.symbols)
             {
-              oprot.writeString(_iter273);
+              oprot.writeString(_iter281);
             }
           }
         }
@@ -21314,39 +21755,39 @@ public class TabletClientService {
         }
         if (incoming.get(2)) {
           {
-            org.apache.thrift.protocol.TMap _map274 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRUCT, org.apache.thrift.protocol.TType.LIST, iprot.readI32());
-            struct.mutations = new java.util.HashMap<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent,java.util.List<org.apache.accumulo.core.dataImpl.thrift.TConditionalMutation>>(2*_map274.size);
-            @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TKeyExtent _key275;
-            @org.apache.thrift.annotation.Nullable java.util.List<org.apache.accumulo.core.dataImpl.thrift.TConditionalMutation> _val276;
-            for (int _i277 = 0; _i277 < _map274.size; ++_i277)
+            org.apache.thrift.protocol.TMap _map282 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRUCT, org.apache.thrift.protocol.TType.LIST, iprot.readI32());
+            struct.mutations = new java.util.HashMap<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent,java.util.List<org.apache.accumulo.core.dataImpl.thrift.TConditionalMutation>>(2*_map282.size);
+            @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TKeyExtent _key283;
+            @org.apache.thrift.annotation.Nullable java.util.List<org.apache.accumulo.core.dataImpl.thrift.TConditionalMutation> _val284;
+            for (int _i285 = 0; _i285 < _map282.size; ++_i285)
             {
-              _key275 = new org.apache.accumulo.core.dataImpl.thrift.TKeyExtent();
-              _key275.read(iprot);
+              _key283 = new org.apache.accumulo.core.dataImpl.thrift.TKeyExtent();
+              _key283.read(iprot);
               {
-                org.apache.thrift.protocol.TList _list278 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-                _val276 = new java.util.ArrayList<org.apache.accumulo.core.dataImpl.thrift.TConditionalMutation>(_list278.size);
-                @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TConditionalMutation _elem279;
-                for (int _i280 = 0; _i280 < _list278.size; ++_i280)
+                org.apache.thrift.protocol.TList _list286 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+                _val284 = new java.util.ArrayList<org.apache.accumulo.core.dataImpl.thrift.TConditionalMutation>(_list286.size);
+                @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TConditionalMutation _elem287;
+                for (int _i288 = 0; _i288 < _list286.size; ++_i288)
                 {
-                  _elem279 = new org.apache.accumulo.core.dataImpl.thrift.TConditionalMutation();
-                  _elem279.read(iprot);
-                  _val276.add(_elem279);
+                  _elem287 = new org.apache.accumulo.core.dataImpl.thrift.TConditionalMutation();
+                  _elem287.read(iprot);
+                  _val284.add(_elem287);
                 }
               }
-              struct.mutations.put(_key275, _val276);
+              struct.mutations.put(_key283, _val284);
             }
           }
           struct.setMutationsIsSet(true);
         }
         if (incoming.get(3)) {
           {
-            org.apache.thrift.protocol.TList _list281 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.symbols = new java.util.ArrayList<java.lang.String>(_list281.size);
-            @org.apache.thrift.annotation.Nullable java.lang.String _elem282;
-            for (int _i283 = 0; _i283 < _list281.size; ++_i283)
+            org.apache.thrift.protocol.TList _list289 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.symbols = new java.util.ArrayList<java.lang.String>(_list289.size);
+            @org.apache.thrift.annotation.Nullable java.lang.String _elem290;
+            for (int _i291 = 0; _i291 < _list289.size; ++_i291)
             {
-              _elem282 = iprot.readString();
-              struct.symbols.add(_elem282);
+              _elem290 = iprot.readString();
+              struct.symbols.add(_elem290);
             }
           }
           struct.setSymbolsIsSet(true);
@@ -21761,14 +22202,14 @@ public class TabletClientService {
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list284 = iprot.readListBegin();
-                  struct.success = new java.util.ArrayList<org.apache.accumulo.core.dataImpl.thrift.TCMResult>(_list284.size);
-                  @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TCMResult _elem285;
-                  for (int _i286 = 0; _i286 < _list284.size; ++_i286)
+                  org.apache.thrift.protocol.TList _list292 = iprot.readListBegin();
+                  struct.success = new java.util.ArrayList<org.apache.accumulo.core.dataImpl.thrift.TCMResult>(_list292.size);
+                  @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TCMResult _elem293;
+                  for (int _i294 = 0; _i294 < _list292.size; ++_i294)
                   {
-                    _elem285 = new org.apache.accumulo.core.dataImpl.thrift.TCMResult();
-                    _elem285.read(iprot);
-                    struct.success.add(_elem285);
+                    _elem293 = new org.apache.accumulo.core.dataImpl.thrift.TCMResult();
+                    _elem293.read(iprot);
+                    struct.success.add(_elem293);
                   }
                   iprot.readListEnd();
                 }
@@ -21805,9 +22246,9 @@ public class TabletClientService {
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size()));
-            for (org.apache.accumulo.core.dataImpl.thrift.TCMResult _iter287 : struct.success)
+            for (org.apache.accumulo.core.dataImpl.thrift.TCMResult _iter295 : struct.success)
             {
-              _iter287.write(oprot);
+              _iter295.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -21846,9 +22287,9 @@ public class TabletClientService {
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (org.apache.accumulo.core.dataImpl.thrift.TCMResult _iter288 : struct.success)
+            for (org.apache.accumulo.core.dataImpl.thrift.TCMResult _iter296 : struct.success)
             {
-              _iter288.write(oprot);
+              _iter296.write(oprot);
             }
           }
         }
@@ -21863,14 +22304,14 @@ public class TabletClientService {
         java.util.BitSet incoming = iprot.readBitSet(2);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list289 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.success = new java.util.ArrayList<org.apache.accumulo.core.dataImpl.thrift.TCMResult>(_list289.size);
-            @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TCMResult _elem290;
-            for (int _i291 = 0; _i291 < _list289.size; ++_i291)
+            org.apache.thrift.protocol.TList _list297 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.success = new java.util.ArrayList<org.apache.accumulo.core.dataImpl.thrift.TCMResult>(_list297.size);
+            @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TCMResult _elem298;
+            for (int _i299 = 0; _i299 < _list297.size; ++_i299)
             {
-              _elem290 = new org.apache.accumulo.core.dataImpl.thrift.TCMResult();
-              _elem290.read(iprot);
-              struct.success.add(_elem290);
+              _elem298 = new org.apache.accumulo.core.dataImpl.thrift.TCMResult();
+              _elem298.read(iprot);
+              struct.success.add(_elem298);
             }
           }
           struct.setSuccessIsSet(true);
@@ -23769,29 +24210,29 @@ public class TabletClientService {
             case 2: // FILES
               if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
                 {
-                  org.apache.thrift.protocol.TMap _map292 = iprot.readMapBegin();
-                  struct.files = new java.util.HashMap<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent,java.util.Map<java.lang.String,org.apache.accumulo.core.dataImpl.thrift.MapFileInfo>>(2*_map292.size);
-                  @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TKeyExtent _key293;
-                  @org.apache.thrift.annotation.Nullable java.util.Map<java.lang.String,org.apache.accumulo.core.dataImpl.thrift.MapFileInfo> _val294;
-                  for (int _i295 = 0; _i295 < _map292.size; ++_i295)
+                  org.apache.thrift.protocol.TMap _map300 = iprot.readMapBegin();
+                  struct.files = new java.util.HashMap<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent,java.util.Map<java.lang.String,org.apache.accumulo.core.dataImpl.thrift.MapFileInfo>>(2*_map300.size);
+                  @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TKeyExtent _key301;
+                  @org.apache.thrift.annotation.Nullable java.util.Map<java.lang.String,org.apache.accumulo.core.dataImpl.thrift.MapFileInfo> _val302;
+                  for (int _i303 = 0; _i303 < _map300.size; ++_i303)
                   {
-                    _key293 = new org.apache.accumulo.core.dataImpl.thrift.TKeyExtent();
-                    _key293.read(iprot);
+                    _key301 = new org.apache.accumulo.core.dataImpl.thrift.TKeyExtent();
+                    _key301.read(iprot);
                     {
-                      org.apache.thrift.protocol.TMap _map296 = iprot.readMapBegin();
-                      _val294 = new java.util.HashMap<java.lang.String,org.apache.accumulo.core.dataImpl.thrift.MapFileInfo>(2*_map296.size);
-                      @org.apache.thrift.annotation.Nullable java.lang.String _key297;
-                      @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.MapFileInfo _val298;
-                      for (int _i299 = 0; _i299 < _map296.size; ++_i299)
+                      org.apache.thrift.protocol.TMap _map304 = iprot.readMapBegin();
+                      _val302 = new java.util.HashMap<java.lang.String,org.apache.accumulo.core.dataImpl.thrift.MapFileInfo>(2*_map304.size);
+                      @org.apache.thrift.annotation.Nullable java.lang.String _key305;
+                      @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.MapFileInfo _val306;
+                      for (int _i307 = 0; _i307 < _map304.size; ++_i307)
                       {
-                        _key297 = iprot.readString();
-                        _val298 = new org.apache.accumulo.core.dataImpl.thrift.MapFileInfo();
-                        _val298.read(iprot);
-                        _val294.put(_key297, _val298);
+                        _key305 = iprot.readString();
+                        _val306 = new org.apache.accumulo.core.dataImpl.thrift.MapFileInfo();
+                        _val306.read(iprot);
+                        _val302.put(_key305, _val306);
                       }
                       iprot.readMapEnd();
                     }
-                    struct.files.put(_key293, _val294);
+                    struct.files.put(_key301, _val302);
                   }
                   iprot.readMapEnd();
                 }
@@ -23832,15 +24273,15 @@ public class TabletClientService {
           oprot.writeFieldBegin(FILES_FIELD_DESC);
           {
             oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRUCT, org.apache.thrift.protocol.TType.MAP, struct.files.size()));
-            for (java.util.Map.Entry<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent, java.util.Map<java.lang.String,org.apache.accumulo.core.dataImpl.thrift.MapFileInfo>> _iter300 : struct.files.entrySet())
+            for (java.util.Map.Entry<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent, java.util.Map<java.lang.String,org.apache.accumulo.core.dataImpl.thrift.MapFileInfo>> _iter308 : struct.files.entrySet())
             {
-              _iter300.getKey().write(oprot);
+              _iter308.getKey().write(oprot);
               {
-                oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT, _iter300.getValue().size()));
-                for (java.util.Map.Entry<java.lang.String, org.apache.accumulo.core.dataImpl.thrift.MapFileInfo> _iter301 : _iter300.getValue().entrySet())
+                oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT, _iter308.getValue().size()));
+                for (java.util.Map.Entry<java.lang.String, org.apache.accumulo.core.dataImpl.thrift.MapFileInfo> _iter309 : _iter308.getValue().entrySet())
                 {
-                  oprot.writeString(_iter301.getKey());
-                  _iter301.getValue().write(oprot);
+                  oprot.writeString(_iter309.getKey());
+                  _iter309.getValue().write(oprot);
                 }
                 oprot.writeMapEnd();
               }
@@ -23906,15 +24347,15 @@ public class TabletClientService {
         if (struct.isSetFiles()) {
           {
             oprot.writeI32(struct.files.size());
-            for (java.util.Map.Entry<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent, java.util.Map<java.lang.String,org.apache.accumulo.core.dataImpl.thrift.MapFileInfo>> _iter302 : struct.files.entrySet())
+            for (java.util.Map.Entry<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent, java.util.Map<java.lang.String,org.apache.accumulo.core.dataImpl.thrift.MapFileInfo>> _iter310 : struct.files.entrySet())
             {
-              _iter302.getKey().write(oprot);
+              _iter310.getKey().write(oprot);
               {
-                oprot.writeI32(_iter302.getValue().size());
-                for (java.util.Map.Entry<java.lang.String, org.apache.accumulo.core.dataImpl.thrift.MapFileInfo> _iter303 : _iter302.getValue().entrySet())
+                oprot.writeI32(_iter310.getValue().size());
+                for (java.util.Map.Entry<java.lang.String, org.apache.accumulo.core.dataImpl.thrift.MapFileInfo> _iter311 : _iter310.getValue().entrySet())
                 {
-                  oprot.writeString(_iter303.getKey());
-                  _iter303.getValue().write(oprot);
+                  oprot.writeString(_iter311.getKey());
+                  _iter311.getValue().write(oprot);
                 }
               }
             }
@@ -23945,28 +24386,28 @@ public class TabletClientService {
         }
         if (incoming.get(3)) {
           {
-            org.apache.thrift.protocol.TMap _map304 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRUCT, org.apache.thrift.protocol.TType.MAP, iprot.readI32());
-            struct.files = new java.util.HashMap<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent,java.util.Map<java.lang.String,org.apache.accumulo.core.dataImpl.thrift.MapFileInfo>>(2*_map304.size);
-            @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TKeyExtent _key305;
-            @org.apache.thrift.annotation.Nullable java.util.Map<java.lang.String,org.apache.accumulo.core.dataImpl.thrift.MapFileInfo> _val306;
-            for (int _i307 = 0; _i307 < _map304.size; ++_i307)
+            org.apache.thrift.protocol.TMap _map312 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRUCT, org.apache.thrift.protocol.TType.MAP, iprot.readI32());
+            struct.files = new java.util.HashMap<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent,java.util.Map<java.lang.String,org.apache.accumulo.core.dataImpl.thrift.MapFileInfo>>(2*_map312.size);
+            @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TKeyExtent _key313;
+            @org.apache.thrift.annotation.Nullable java.util.Map<java.lang.String,org.apache.accumulo.core.dataImpl.thrift.MapFileInfo> _val314;
+            for (int _i315 = 0; _i315 < _map312.size; ++_i315)
             {
-              _key305 = new org.apache.accumulo.core.dataImpl.thrift.TKeyExtent();
-              _key305.read(iprot);
+              _key313 = new org.apache.accumulo.core.dataImpl.thrift.TKeyExtent();
+              _key313.read(iprot);
               {
-                org.apache.thrift.protocol.TMap _map308 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-                _val306 = new java.util.HashMap<java.lang.String,org.apache.accumulo.core.dataImpl.thrift.MapFileInfo>(2*_map308.size);
-                @org.apache.thrift.annotation.Nullable java.lang.String _key309;
-                @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.MapFileInfo _val310;
-                for (int _i311 = 0; _i311 < _map308.size; ++_i311)
+                org.apache.thrift.protocol.TMap _map316 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+                _val314 = new java.util.HashMap<java.lang.String,org.apache.accumulo.core.dataImpl.thrift.MapFileInfo>(2*_map316.size);
+                @org.apache.thrift.annotation.Nullable java.lang.String _key317;
+                @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.MapFileInfo _val318;
+                for (int _i319 = 0; _i319 < _map316.size; ++_i319)
                 {
-                  _key309 = iprot.readString();
-                  _val310 = new org.apache.accumulo.core.dataImpl.thrift.MapFileInfo();
-                  _val310.read(iprot);
-                  _val306.put(_key309, _val310);
+                  _key317 = iprot.readString();
+                  _val318 = new org.apache.accumulo.core.dataImpl.thrift.MapFileInfo();
+                  _val318.read(iprot);
+                  _val314.put(_key317, _val318);
... 3109 lines suppressed ...