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 2022/04/21 12:03:21 UTC

[accumulo] branch main updated: Separated tablet scan functions from TabletClientService into a new Thrift service (#2643)

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

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


The following commit(s) were added to refs/heads/main by this push:
     new 50b9267f52 Separated tablet scan functions from TabletClientService into a new Thrift service (#2643)
50b9267f52 is described below

commit 50b9267f526736fc2c510a2626693b072ca4dd04
Author: Dave Marion <dl...@apache.org>
AuthorDate: Thu Apr 21 08:03:15 2022 -0400

    Separated tablet scan functions from TabletClientService into a new Thrift service (#2643)
    
    
    related to #2640
---
 .../core/clientImpl/InstanceOperationsImpl.java    |     5 +-
 .../TabletServerBatchReaderIterator.java           |     8 +-
 .../accumulo/core/clientImpl/ThriftScanner.java    |    16 +-
 .../accumulo/core/rpc/ThriftClientTypes.java       |     4 +
 .../tabletserver/thrift/TabletClientService.java   | 26681 ++++++-------------
 .../thrift/TabletScanClientService.java            | 11552 ++++++++
 core/src/main/thrift/tabletserver.thrift           |    20 +-
 .../accumulo/server/rpc/ThriftProcessorTypes.java  |    12 +-
 .../server/util/VerifyTabletAssignments.java       |     6 +-
 .../java/org/apache/accumulo/monitor/Monitor.java  |     5 +-
 .../accumulo/tserver/TabletClientHandler.java      |   408 +-
 .../org/apache/accumulo/tserver/TabletServer.java  |    16 +-
 .../accumulo/tserver/ThriftScanClientHandler.java  |   473 +
 .../org/apache/accumulo/tserver/WriteTracker.java  |     2 +-
 .../test/compaction/ExternalCompactionTServer.java |     6 +-
 ...ttingExternalCompactionTabletClientHandler.java |     5 +-
 .../accumulo/test/functional/ZombieTServer.java    |     7 +-
 .../accumulo/test/performance/NullTserver.java     |     8 +-
 18 files changed, 19740 insertions(+), 19494 deletions(-)

diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/InstanceOperationsImpl.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/InstanceOperationsImpl.java
index 627a739761..2effbb7dad 100644
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/InstanceOperationsImpl.java
+++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/InstanceOperationsImpl.java
@@ -48,6 +48,7 @@ import org.apache.accumulo.core.conf.DeprecatedPropertyUtil;
 import org.apache.accumulo.core.data.InstanceId;
 import org.apache.accumulo.core.rpc.ThriftClientTypes;
 import org.apache.accumulo.core.tabletserver.thrift.TabletClientService.Client;
+import org.apache.accumulo.core.tabletserver.thrift.TabletScanClientService;
 import org.apache.accumulo.core.trace.TraceUtil;
 import org.apache.accumulo.core.util.AddressUtil;
 import org.apache.accumulo.core.util.HostAndPort;
@@ -158,9 +159,9 @@ public class InstanceOperationsImpl implements InstanceOperations {
   public List<ActiveScan> getActiveScans(String tserver)
       throws AccumuloException, AccumuloSecurityException {
     final var parsedTserver = HostAndPort.fromString(tserver);
-    Client client = null;
+    TabletScanClientService.Client client = null;
     try {
-      client = getClient(ThriftClientTypes.TABLET_SERVER, parsedTserver, context);
+      client = getClient(ThriftClientTypes.TABLET_SCAN, parsedTserver, context);
 
       List<ActiveScan> as = new ArrayList<>();
       for (var activeScan : client.getActiveScans(TraceUtil.traceInfo(), context.rpcCreds())) {
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/TabletServerBatchReaderIterator.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/TabletServerBatchReaderIterator.java
index 5cbc595c09..d9896fd441 100644
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/TabletServerBatchReaderIterator.java
+++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/TabletServerBatchReaderIterator.java
@@ -64,7 +64,7 @@ import org.apache.accumulo.core.sample.impl.SamplerConfigurationImpl;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.tabletserver.thrift.NoSuchScanIDException;
 import org.apache.accumulo.core.tabletserver.thrift.TSampleNotPresentException;
-import org.apache.accumulo.core.tabletserver.thrift.TabletClientService;
+import org.apache.accumulo.core.tabletserver.thrift.TabletScanClientService;
 import org.apache.accumulo.core.trace.TraceUtil;
 import org.apache.accumulo.core.util.ByteBufferUtil;
 import org.apache.accumulo.core.util.HostAndPort;
@@ -652,12 +652,12 @@ public class TabletServerBatchReaderIterator implements Iterator<Entry<Key,Value
     timeoutTracker.startingScan();
     try {
       final HostAndPort parsedServer = HostAndPort.fromString(server);
-      final TabletClientService.Client client;
+      final TabletScanClientService.Client client;
       if (timeoutTracker.getTimeOut() < context.getClientTimeoutInMillis())
-        client = ThriftUtil.getClient(ThriftClientTypes.TABLET_SERVER, parsedServer, context,
+        client = ThriftUtil.getClient(ThriftClientTypes.TABLET_SCAN, parsedServer, context,
             timeoutTracker.getTimeOut());
       else
-        client = ThriftUtil.getClient(ThriftClientTypes.TABLET_SERVER, parsedServer, context);
+        client = ThriftUtil.getClient(ThriftClientTypes.TABLET_SCAN, parsedServer, context);
 
       try {
 
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/ThriftScanner.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/ThriftScanner.java
index 41e4aeeba1..bd976e9cc5 100644
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/ThriftScanner.java
+++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/ThriftScanner.java
@@ -61,7 +61,7 @@ import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.tabletserver.thrift.NoSuchScanIDException;
 import org.apache.accumulo.core.tabletserver.thrift.NotServingTabletException;
 import org.apache.accumulo.core.tabletserver.thrift.TSampleNotPresentException;
-import org.apache.accumulo.core.tabletserver.thrift.TabletClientService;
+import org.apache.accumulo.core.tabletserver.thrift.TabletScanClientService;
 import org.apache.accumulo.core.tabletserver.thrift.TooManyFilesException;
 import org.apache.accumulo.core.trace.TraceUtil;
 import org.apache.accumulo.core.trace.thrift.TInfo;
@@ -101,8 +101,8 @@ public class ThriftScanner {
     final HostAndPort parsedServer = HostAndPort.fromString(server);
     try {
       TInfo tinfo = TraceUtil.traceInfo();
-      TabletClientService.Client client =
-          ThriftUtil.getClient(ThriftClientTypes.TABLET_SERVER, parsedServer, context);
+      TabletScanClientService.Client client =
+          ThriftUtil.getClient(ThriftClientTypes.TABLET_SCAN, parsedServer, context);
       try {
         // not reading whole rows (or stopping on row boundaries) so there is no need to enable
         // isolation below
@@ -452,8 +452,8 @@ public class ThriftScanner {
 
     final TInfo tinfo = TraceUtil.traceInfo();
     final HostAndPort parsedLocation = HostAndPort.fromString(loc.tablet_location);
-    TabletClientService.Client client =
-        ThriftUtil.getClient(ThriftClientTypes.TABLET_SERVER, parsedLocation, context);
+    TabletScanClientService.Client client =
+        ThriftUtil.getClient(ThriftClientTypes.TABLET_SCAN, parsedLocation, context);
 
     String old = Thread.currentThread().getName();
     try {
@@ -584,10 +584,10 @@ public class ThriftScanner {
 
       log.debug("Closing active scan {} {}", scanState.prevLoc, scanState.scanID);
       HostAndPort parsedLocation = HostAndPort.fromString(scanState.prevLoc.tablet_location);
-      TabletClientService.Client client = null;
+      TabletScanClientService.Client client = null;
       try {
-        client = ThriftUtil.getClient(ThriftClientTypes.TABLET_SERVER, parsedLocation,
-            scanState.context);
+        client =
+            ThriftUtil.getClient(ThriftClientTypes.TABLET_SCAN, parsedLocation, scanState.context);
         client.closeScan(tinfo, scanState.scanID);
       } catch (TException e) {
         // ignore this is a best effort
diff --git a/core/src/main/java/org/apache/accumulo/core/rpc/ThriftClientTypes.java b/core/src/main/java/org/apache/accumulo/core/rpc/ThriftClientTypes.java
index dd567c2de1..3e26d18b73 100644
--- a/core/src/main/java/org/apache/accumulo/core/rpc/ThriftClientTypes.java
+++ b/core/src/main/java/org/apache/accumulo/core/rpc/ThriftClientTypes.java
@@ -27,6 +27,7 @@ import org.apache.accumulo.core.manager.thrift.ManagerClientService;
 import org.apache.accumulo.core.replication.thrift.ReplicationCoordinator;
 import org.apache.accumulo.core.replication.thrift.ReplicationServicer;
 import org.apache.accumulo.core.tabletserver.thrift.TabletClientService;
+import org.apache.accumulo.core.tabletserver.thrift.TabletScanClientService;
 import org.apache.thrift.TServiceClient;
 import org.apache.thrift.TServiceClientFactory;
 import org.apache.thrift.protocol.TMultiplexedProtocol;
@@ -94,4 +95,7 @@ public class ThriftClientTypes {
       TabletClientService.Client.Factory> TABLET_SERVER =
           new ThriftClientType<>("TabletClientService", new TabletClientService.Client.Factory());
 
+  public static final ThriftClientType<TabletScanClientService.Client,
+      TabletScanClientService.Client.Factory> TABLET_SCAN = new ThriftClientType<>(
+          "TabletScanClientService", new TabletScanClientService.Client.Factory());
 }
diff --git a/core/src/main/thrift-gen-java/org/apache/accumulo/core/tabletserver/thrift/TabletClientService.java b/core/src/main/thrift-gen-java/org/apache/accumulo/core/tabletserver/thrift/TabletClientService.java
index 8d799d2c5e..287d4195cb 100644
--- a/core/src/main/thrift-gen-java/org/apache/accumulo/core/tabletserver/thrift/TabletClientService.java
+++ b/core/src/main/thrift-gen-java/org/apache/accumulo/core/tabletserver/thrift/TabletClientService.java
@@ -29,18 +29,6 @@ public class TabletClientService {
 
   public interface Iface {
 
-    public org.apache.accumulo.core.dataImpl.thrift.InitialScan startScan(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent, org.apache.accumulo.core.dataImpl.thrift.TRange range, java.util.List<org.apache.accumulo.core.dataImpl.thrift.TColumn> columns, int batchSize, java.util.List<org.apache.accumulo.core.dataImpl.thrift.IterInfo> ssiList, java.util.Map<jav [...]
-
-    public org.apache.accumulo.core.dataImpl.thrift.ScanResult continueScan(org.apache.accumulo.core.trace.thrift.TInfo tinfo, long scanID) throws NoSuchScanIDException, NotServingTabletException, TooManyFilesException, TSampleNotPresentException, org.apache.thrift.TException;
-
-    public void closeScan(org.apache.accumulo.core.trace.thrift.TInfo tinfo, long scanID) throws org.apache.thrift.TException;
-
-    public org.apache.accumulo.core.dataImpl.thrift.InitialMultiScan startMultiScan(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.util.Map<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent,java.util.List<org.apache.accumulo.core.dataImpl.thrift.TRange>> batch, java.util.List<org.apache.accumulo.core.dataImpl.thrift.TColumn> columns, java.util.List<org.apache.accumulo.core.dataImpl.thrift.IterInfo> ssiList, [...]
-
-    public org.apache.accumulo.core.dataImpl.thrift.MultiScanResult continueMultiScan(org.apache.accumulo.core.trace.thrift.TInfo tinfo, long scanID) throws NoSuchScanIDException, TSampleNotPresentException, org.apache.thrift.TException;
-
-    public void closeMultiScan(org.apache.accumulo.core.trace.thrift.TInfo tinfo, long scanID) throws NoSuchScanIDException, org.apache.thrift.TException;
-
     public long startUpdate(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, TDurability durability) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.thrift.TException;
 
     public void applyUpdates(org.apache.accumulo.core.trace.thrift.TInfo tinfo, long updateID, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent keyExtent, java.util.List<org.apache.accumulo.core.dataImpl.thrift.TMutation> mutations) throws org.apache.thrift.TException;
@@ -85,8 +73,6 @@ public class TabletClientService {
 
     public void fastHalt(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String lock) throws org.apache.thrift.TException;
 
-    public java.util.List<ActiveScan> getActiveScans(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.thrift.TException;
-
     public java.util.List<ActiveCompaction> getActiveCompactions(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.thrift.TException;
 
     public void removeLogs(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.util.List<java.lang.String> filenames) throws org.apache.thrift.TException;
@@ -113,18 +99,6 @@ public class TabletClientService {
 
   public interface AsyncIface {
 
-    public void startScan(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent, org.apache.accumulo.core.dataImpl.thrift.TRange range, java.util.List<org.apache.accumulo.core.dataImpl.thrift.TColumn> columns, int batchSize, java.util.List<org.apache.accumulo.core.dataImpl.thrift.IterInfo> ssiList, java.util.Map<java.lang.String,java.util.Map<java.lang.String,jav [...]
-
-    public void continueScan(org.apache.accumulo.core.trace.thrift.TInfo tinfo, long scanID, org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.dataImpl.thrift.ScanResult> resultHandler) throws org.apache.thrift.TException;
-
-    public void closeScan(org.apache.accumulo.core.trace.thrift.TInfo tinfo, long scanID, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
-
-    public void startMultiScan(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.util.Map<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent,java.util.List<org.apache.accumulo.core.dataImpl.thrift.TRange>> batch, java.util.List<org.apache.accumulo.core.dataImpl.thrift.TColumn> columns, java.util.List<org.apache.accumulo.core.dataImpl.thrift.IterInfo> ssiList, java.util.Map<java.lang.String,java.util.Map<java.la [...]
-
-    public void continueMultiScan(org.apache.accumulo.core.trace.thrift.TInfo tinfo, long scanID, org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.dataImpl.thrift.MultiScanResult> resultHandler) throws org.apache.thrift.TException;
-
-    public void closeMultiScan(org.apache.accumulo.core.trace.thrift.TInfo tinfo, long scanID, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
-
     public void startUpdate(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, TDurability durability, org.apache.thrift.async.AsyncMethodCallback<java.lang.Long> resultHandler) throws org.apache.thrift.TException;
 
     public void applyUpdates(org.apache.accumulo.core.trace.thrift.TInfo tinfo, long updateID, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent keyExtent, java.util.List<org.apache.accumulo.core.dataImpl.thrift.TMutation> mutations, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
@@ -169,8 +143,6 @@ public class TabletClientService {
 
     public void fastHalt(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String lock, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
 
-    public void getActiveScans(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<java.util.List<ActiveScan>> resultHandler) throws org.apache.thrift.TException;
-
     public void getActiveCompactions(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<java.util.List<ActiveCompaction>> resultHandler) throws org.apache.thrift.TException;
 
     public void removeLogs(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.util.List<java.lang.String> filenames, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
@@ -215,199 +187,6 @@ public class TabletClientService {
       super(iprot, oprot);
     }
 
-    public org.apache.accumulo.core.dataImpl.thrift.InitialScan startScan(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent, org.apache.accumulo.core.dataImpl.thrift.TRange range, java.util.List<org.apache.accumulo.core.dataImpl.thrift.TColumn> columns, int batchSize, java.util.List<org.apache.accumulo.core.dataImpl.thrift.IterInfo> ssiList, java.util.Map<jav [...]
-    {
-      send_startScan(tinfo, credentials, extent, range, columns, batchSize, ssiList, ssio, authorizations, waitForWrites, isolated, readaheadThreshold, samplerConfig, batchTimeOut, classLoaderContext, executionHints);
-      return recv_startScan();
-    }
-
-    public void send_startScan(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent, org.apache.accumulo.core.dataImpl.thrift.TRange range, java.util.List<org.apache.accumulo.core.dataImpl.thrift.TColumn> columns, int batchSize, java.util.List<org.apache.accumulo.core.dataImpl.thrift.IterInfo> ssiList, java.util.Map<java.lang.String,java.util.Map<java.lang.Strin [...]
-    {
-      startScan_args args = new startScan_args();
-      args.setTinfo(tinfo);
-      args.setCredentials(credentials);
-      args.setExtent(extent);
-      args.setRange(range);
-      args.setColumns(columns);
-      args.setBatchSize(batchSize);
-      args.setSsiList(ssiList);
-      args.setSsio(ssio);
-      args.setAuthorizations(authorizations);
-      args.setWaitForWrites(waitForWrites);
-      args.setIsolated(isolated);
-      args.setReadaheadThreshold(readaheadThreshold);
-      args.setSamplerConfig(samplerConfig);
-      args.setBatchTimeOut(batchTimeOut);
-      args.setClassLoaderContext(classLoaderContext);
-      args.setExecutionHints(executionHints);
-      sendBase("startScan", args);
-    }
-
-    public org.apache.accumulo.core.dataImpl.thrift.InitialScan recv_startScan() throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, NotServingTabletException, TooManyFilesException, TSampleNotPresentException, org.apache.thrift.TException
-    {
-      startScan_result result = new startScan_result();
-      receiveBase(result, "startScan");
-      if (result.isSetSuccess()) {
-        return result.success;
-      }
-      if (result.sec != null) {
-        throw result.sec;
-      }
-      if (result.nste != null) {
-        throw result.nste;
-      }
-      if (result.tmfe != null) {
-        throw result.tmfe;
-      }
-      if (result.tsnpe != null) {
-        throw result.tsnpe;
-      }
-      throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "startScan failed: unknown result");
-    }
-
-    public org.apache.accumulo.core.dataImpl.thrift.ScanResult continueScan(org.apache.accumulo.core.trace.thrift.TInfo tinfo, long scanID) throws NoSuchScanIDException, NotServingTabletException, TooManyFilesException, TSampleNotPresentException, org.apache.thrift.TException
-    {
-      send_continueScan(tinfo, scanID);
-      return recv_continueScan();
-    }
-
-    public void send_continueScan(org.apache.accumulo.core.trace.thrift.TInfo tinfo, long scanID) throws org.apache.thrift.TException
-    {
-      continueScan_args args = new continueScan_args();
-      args.setTinfo(tinfo);
-      args.setScanID(scanID);
-      sendBase("continueScan", args);
-    }
-
-    public org.apache.accumulo.core.dataImpl.thrift.ScanResult recv_continueScan() throws NoSuchScanIDException, NotServingTabletException, TooManyFilesException, TSampleNotPresentException, org.apache.thrift.TException
-    {
-      continueScan_result result = new continueScan_result();
-      receiveBase(result, "continueScan");
-      if (result.isSetSuccess()) {
-        return result.success;
-      }
-      if (result.nssi != null) {
-        throw result.nssi;
-      }
-      if (result.nste != null) {
-        throw result.nste;
-      }
-      if (result.tmfe != null) {
-        throw result.tmfe;
-      }
-      if (result.tsnpe != null) {
-        throw result.tsnpe;
-      }
-      throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "continueScan failed: unknown result");
-    }
-
-    public void closeScan(org.apache.accumulo.core.trace.thrift.TInfo tinfo, long scanID) throws org.apache.thrift.TException
-    {
-      send_closeScan(tinfo, scanID);
-    }
-
-    public void send_closeScan(org.apache.accumulo.core.trace.thrift.TInfo tinfo, long scanID) throws org.apache.thrift.TException
-    {
-      closeScan_args args = new closeScan_args();
-      args.setTinfo(tinfo);
-      args.setScanID(scanID);
-      sendBaseOneway("closeScan", args);
-    }
-
-    public org.apache.accumulo.core.dataImpl.thrift.InitialMultiScan startMultiScan(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.util.Map<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent,java.util.List<org.apache.accumulo.core.dataImpl.thrift.TRange>> batch, java.util.List<org.apache.accumulo.core.dataImpl.thrift.TColumn> columns, java.util.List<org.apache.accumulo.core.dataImpl.thrift.IterInfo> ssiList, [...]
-    {
-      send_startMultiScan(tinfo, credentials, batch, columns, ssiList, ssio, authorizations, waitForWrites, samplerConfig, batchTimeOut, classLoaderContext, executionHints);
-      return recv_startMultiScan();
-    }
-
-    public void send_startMultiScan(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.util.Map<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent,java.util.List<org.apache.accumulo.core.dataImpl.thrift.TRange>> batch, java.util.List<org.apache.accumulo.core.dataImpl.thrift.TColumn> columns, java.util.List<org.apache.accumulo.core.dataImpl.thrift.IterInfo> ssiList, java.util.Map<java.lang.String,java.util.Map<ja [...]
-    {
-      startMultiScan_args args = new startMultiScan_args();
-      args.setTinfo(tinfo);
-      args.setCredentials(credentials);
-      args.setBatch(batch);
-      args.setColumns(columns);
-      args.setSsiList(ssiList);
-      args.setSsio(ssio);
-      args.setAuthorizations(authorizations);
-      args.setWaitForWrites(waitForWrites);
-      args.setSamplerConfig(samplerConfig);
-      args.setBatchTimeOut(batchTimeOut);
-      args.setClassLoaderContext(classLoaderContext);
-      args.setExecutionHints(executionHints);
-      sendBase("startMultiScan", args);
-    }
-
-    public org.apache.accumulo.core.dataImpl.thrift.InitialMultiScan recv_startMultiScan() throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, TSampleNotPresentException, org.apache.thrift.TException
-    {
-      startMultiScan_result result = new startMultiScan_result();
-      receiveBase(result, "startMultiScan");
-      if (result.isSetSuccess()) {
-        return result.success;
-      }
-      if (result.sec != null) {
-        throw result.sec;
-      }
-      if (result.tsnpe != null) {
-        throw result.tsnpe;
-      }
-      throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "startMultiScan failed: unknown result");
-    }
-
-    public org.apache.accumulo.core.dataImpl.thrift.MultiScanResult continueMultiScan(org.apache.accumulo.core.trace.thrift.TInfo tinfo, long scanID) throws NoSuchScanIDException, TSampleNotPresentException, org.apache.thrift.TException
-    {
-      send_continueMultiScan(tinfo, scanID);
-      return recv_continueMultiScan();
-    }
-
-    public void send_continueMultiScan(org.apache.accumulo.core.trace.thrift.TInfo tinfo, long scanID) throws org.apache.thrift.TException
-    {
-      continueMultiScan_args args = new continueMultiScan_args();
-      args.setTinfo(tinfo);
-      args.setScanID(scanID);
-      sendBase("continueMultiScan", args);
-    }
-
-    public org.apache.accumulo.core.dataImpl.thrift.MultiScanResult recv_continueMultiScan() throws NoSuchScanIDException, TSampleNotPresentException, org.apache.thrift.TException
-    {
-      continueMultiScan_result result = new continueMultiScan_result();
-      receiveBase(result, "continueMultiScan");
-      if (result.isSetSuccess()) {
-        return result.success;
-      }
-      if (result.nssi != null) {
-        throw result.nssi;
-      }
-      if (result.tsnpe != null) {
-        throw result.tsnpe;
-      }
-      throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "continueMultiScan failed: unknown result");
-    }
-
-    public void closeMultiScan(org.apache.accumulo.core.trace.thrift.TInfo tinfo, long scanID) throws NoSuchScanIDException, org.apache.thrift.TException
-    {
-      send_closeMultiScan(tinfo, scanID);
-      recv_closeMultiScan();
-    }
-
-    public void send_closeMultiScan(org.apache.accumulo.core.trace.thrift.TInfo tinfo, long scanID) throws org.apache.thrift.TException
-    {
-      closeMultiScan_args args = new closeMultiScan_args();
-      args.setTinfo(tinfo);
-      args.setScanID(scanID);
-      sendBase("closeMultiScan", args);
-    }
-
-    public void recv_closeMultiScan() throws NoSuchScanIDException, org.apache.thrift.TException
-    {
-      closeMultiScan_result result = new closeMultiScan_result();
-      receiveBase(result, "closeMultiScan");
-      if (result.nssi != null) {
-        throw result.nssi;
-      }
-      return;
-    }
-
     public long startUpdate(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, TDurability durability) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.thrift.TException
     {
       send_startUpdate(tinfo, credentials, durability);
@@ -898,33 +677,6 @@ public class TabletClientService {
       sendBaseOneway("fastHalt", args);
     }
 
-    public java.util.List<ActiveScan> getActiveScans(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.thrift.TException
-    {
-      send_getActiveScans(tinfo, credentials);
-      return recv_getActiveScans();
-    }
-
-    public void send_getActiveScans(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws org.apache.thrift.TException
-    {
-      getActiveScans_args args = new getActiveScans_args();
-      args.setTinfo(tinfo);
-      args.setCredentials(credentials);
-      sendBase("getActiveScans", args);
-    }
-
-    public java.util.List<ActiveScan> recv_getActiveScans() throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.thrift.TException
-    {
-      getActiveScans_result result = new getActiveScans_result();
-      receiveBase(result, "getActiveScans");
-      if (result.isSetSuccess()) {
-        return result.success;
-      }
-      if (result.sec != null) {
-        throw result.sec;
-      }
-      throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "getActiveScans failed: unknown result");
-    }
-
     public java.util.List<ActiveCompaction> getActiveCompactions(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.thrift.TException
     {
       send_getActiveCompactions(tinfo, credentials);
@@ -1215,535 +967,253 @@ public class TabletClientService {
       super(protocolFactory, clientManager, transport);
     }
 
-    public void startScan(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent, org.apache.accumulo.core.dataImpl.thrift.TRange range, java.util.List<org.apache.accumulo.core.dataImpl.thrift.TColumn> columns, int batchSize, java.util.List<org.apache.accumulo.core.dataImpl.thrift.IterInfo> ssiList, java.util.Map<java.lang.String,java.util.Map<java.lang.String,jav [...]
+    public void startUpdate(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, TDurability durability, org.apache.thrift.async.AsyncMethodCallback<java.lang.Long> resultHandler) throws org.apache.thrift.TException {
       checkReady();
-      startScan_call method_call = new startScan_call(tinfo, credentials, extent, range, columns, batchSize, ssiList, ssio, authorizations, waitForWrites, isolated, readaheadThreshold, samplerConfig, batchTimeOut, classLoaderContext, executionHints, resultHandler, this, ___protocolFactory, ___transport);
+      startUpdate_call method_call = new startUpdate_call(tinfo, credentials, durability, resultHandler, this, ___protocolFactory, ___transport);
       this.___currentMethod = method_call;
       ___manager.call(method_call);
     }
 
-    public static class startScan_call extends org.apache.thrift.async.TAsyncMethodCall<org.apache.accumulo.core.dataImpl.thrift.InitialScan> {
+    public static class startUpdate_call extends org.apache.thrift.async.TAsyncMethodCall<java.lang.Long> {
       private org.apache.accumulo.core.trace.thrift.TInfo tinfo;
       private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials;
-      private org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent;
-      private org.apache.accumulo.core.dataImpl.thrift.TRange range;
-      private java.util.List<org.apache.accumulo.core.dataImpl.thrift.TColumn> columns;
-      private int batchSize;
-      private java.util.List<org.apache.accumulo.core.dataImpl.thrift.IterInfo> ssiList;
-      private java.util.Map<java.lang.String,java.util.Map<java.lang.String,java.lang.String>> ssio;
-      private java.util.List<java.nio.ByteBuffer> authorizations;
-      private boolean waitForWrites;
-      private boolean isolated;
-      private long readaheadThreshold;
-      private TSamplerConfiguration samplerConfig;
-      private long batchTimeOut;
-      private java.lang.String classLoaderContext;
-      private java.util.Map<java.lang.String,java.lang.String> executionHints;
-      public startScan_call(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent, org.apache.accumulo.core.dataImpl.thrift.TRange range, java.util.List<org.apache.accumulo.core.dataImpl.thrift.TColumn> columns, int batchSize, java.util.List<org.apache.accumulo.core.dataImpl.thrift.IterInfo> ssiList, java.util.Map<java.lang.String,java.util.Map<java.lang.String,j [...]
+      private TDurability durability;
+      public startUpdate_call(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, TDurability durability, org.apache.thrift.async.AsyncMethodCallback<java.lang.Long> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
         super(client, protocolFactory, transport, resultHandler, false);
         this.tinfo = tinfo;
         this.credentials = credentials;
-        this.extent = extent;
-        this.range = range;
-        this.columns = columns;
-        this.batchSize = batchSize;
-        this.ssiList = ssiList;
-        this.ssio = ssio;
-        this.authorizations = authorizations;
-        this.waitForWrites = waitForWrites;
-        this.isolated = isolated;
-        this.readaheadThreshold = readaheadThreshold;
-        this.samplerConfig = samplerConfig;
-        this.batchTimeOut = batchTimeOut;
-        this.classLoaderContext = classLoaderContext;
-        this.executionHints = executionHints;
+        this.durability = durability;
       }
 
       public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
-        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("startScan", org.apache.thrift.protocol.TMessageType.CALL, 0));
-        startScan_args args = new startScan_args();
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("startUpdate", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        startUpdate_args args = new startUpdate_args();
         args.setTinfo(tinfo);
         args.setCredentials(credentials);
-        args.setExtent(extent);
-        args.setRange(range);
-        args.setColumns(columns);
-        args.setBatchSize(batchSize);
-        args.setSsiList(ssiList);
-        args.setSsio(ssio);
-        args.setAuthorizations(authorizations);
-        args.setWaitForWrites(waitForWrites);
-        args.setIsolated(isolated);
-        args.setReadaheadThreshold(readaheadThreshold);
-        args.setSamplerConfig(samplerConfig);
-        args.setBatchTimeOut(batchTimeOut);
-        args.setClassLoaderContext(classLoaderContext);
-        args.setExecutionHints(executionHints);
+        args.setDurability(durability);
         args.write(prot);
         prot.writeMessageEnd();
       }
 
-      public org.apache.accumulo.core.dataImpl.thrift.InitialScan getResult() throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, NotServingTabletException, TooManyFilesException, TSampleNotPresentException, org.apache.thrift.TException {
+      public java.lang.Long getResult() throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.thrift.TException {
         if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
           throw new java.lang.IllegalStateException("Method call not finished!");
         }
         org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
         org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
-        return (new Client(prot)).recv_startScan();
+        return (new Client(prot)).recv_startUpdate();
       }
     }
 
-    public void continueScan(org.apache.accumulo.core.trace.thrift.TInfo tinfo, long scanID, org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.dataImpl.thrift.ScanResult> resultHandler) throws org.apache.thrift.TException {
+    public void applyUpdates(org.apache.accumulo.core.trace.thrift.TInfo tinfo, long updateID, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent keyExtent, java.util.List<org.apache.accumulo.core.dataImpl.thrift.TMutation> mutations, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
       checkReady();
-      continueScan_call method_call = new continueScan_call(tinfo, scanID, resultHandler, this, ___protocolFactory, ___transport);
+      applyUpdates_call method_call = new applyUpdates_call(tinfo, updateID, keyExtent, mutations, resultHandler, this, ___protocolFactory, ___transport);
       this.___currentMethod = method_call;
       ___manager.call(method_call);
     }
 
-    public static class continueScan_call extends org.apache.thrift.async.TAsyncMethodCall<org.apache.accumulo.core.dataImpl.thrift.ScanResult> {
+    public static class applyUpdates_call extends org.apache.thrift.async.TAsyncMethodCall<Void> {
       private org.apache.accumulo.core.trace.thrift.TInfo tinfo;
-      private long scanID;
-      public continueScan_call(org.apache.accumulo.core.trace.thrift.TInfo tinfo, long scanID, org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.dataImpl.thrift.ScanResult> 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);
+      private long updateID;
+      private org.apache.accumulo.core.dataImpl.thrift.TKeyExtent keyExtent;
+      private java.util.List<org.apache.accumulo.core.dataImpl.thrift.TMutation> mutations;
+      public applyUpdates_call(org.apache.accumulo.core.trace.thrift.TInfo tinfo, long updateID, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent keyExtent, java.util.List<org.apache.accumulo.core.dataImpl.thrift.TMutation> mutations, 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.thri [...]
+        super(client, protocolFactory, transport, resultHandler, true);
         this.tinfo = tinfo;
-        this.scanID = scanID;
+        this.updateID = updateID;
+        this.keyExtent = keyExtent;
+        this.mutations = mutations;
       }
 
       public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
-        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("continueScan", org.apache.thrift.protocol.TMessageType.CALL, 0));
-        continueScan_args args = new continueScan_args();
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("applyUpdates", org.apache.thrift.protocol.TMessageType.ONEWAY, 0));
+        applyUpdates_args args = new applyUpdates_args();
         args.setTinfo(tinfo);
-        args.setScanID(scanID);
+        args.setUpdateID(updateID);
+        args.setKeyExtent(keyExtent);
+        args.setMutations(mutations);
         args.write(prot);
         prot.writeMessageEnd();
       }
 
-      public org.apache.accumulo.core.dataImpl.thrift.ScanResult getResult() throws NoSuchScanIDException, NotServingTabletException, TooManyFilesException, TSampleNotPresentException, 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_continueScan();
+        return null;
       }
     }
 
-    public void closeScan(org.apache.accumulo.core.trace.thrift.TInfo tinfo, long scanID, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
+    public void closeUpdate(org.apache.accumulo.core.trace.thrift.TInfo tinfo, long updateID, org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.dataImpl.thrift.UpdateErrors> resultHandler) throws org.apache.thrift.TException {
       checkReady();
-      closeScan_call method_call = new closeScan_call(tinfo, scanID, resultHandler, this, ___protocolFactory, ___transport);
+      closeUpdate_call method_call = new closeUpdate_call(tinfo, updateID, resultHandler, this, ___protocolFactory, ___transport);
       this.___currentMethod = method_call;
       ___manager.call(method_call);
     }
 
-    public static class closeScan_call extends org.apache.thrift.async.TAsyncMethodCall<Void> {
+    public static class closeUpdate_call extends org.apache.thrift.async.TAsyncMethodCall<org.apache.accumulo.core.dataImpl.thrift.UpdateErrors> {
       private org.apache.accumulo.core.trace.thrift.TInfo tinfo;
-      private long scanID;
-      public closeScan_call(org.apache.accumulo.core.trace.thrift.TInfo tinfo, long scanID, 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, true);
+      private long updateID;
+      public closeUpdate_call(org.apache.accumulo.core.trace.thrift.TInfo tinfo, long updateID, org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.dataImpl.thrift.UpdateErrors> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+        super(client, protocolFactory, transport, resultHandler, false);
         this.tinfo = tinfo;
-        this.scanID = scanID;
+        this.updateID = updateID;
       }
 
       public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
-        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("closeScan", org.apache.thrift.protocol.TMessageType.ONEWAY, 0));
-        closeScan_args args = new closeScan_args();
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("closeUpdate", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        closeUpdate_args args = new closeUpdate_args();
         args.setTinfo(tinfo);
-        args.setScanID(scanID);
+        args.setUpdateID(updateID);
         args.write(prot);
         prot.writeMessageEnd();
       }
 
-      public Void getResult() throws org.apache.thrift.TException {
+      public org.apache.accumulo.core.dataImpl.thrift.UpdateErrors getResult() throws NoSuchScanIDException, 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;
+        return (new Client(prot)).recv_closeUpdate();
       }
     }
 
-    public void startMultiScan(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.util.Map<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent,java.util.List<org.apache.accumulo.core.dataImpl.thrift.TRange>> batch, java.util.List<org.apache.accumulo.core.dataImpl.thrift.TColumn> columns, java.util.List<org.apache.accumulo.core.dataImpl.thrift.IterInfo> ssiList, java.util.Map<java.lang.String,java.util.Map<java.la [...]
+    public void update(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent keyExtent, org.apache.accumulo.core.dataImpl.thrift.TMutation mutation, TDurability durability, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
       checkReady();
-      startMultiScan_call method_call = new startMultiScan_call(tinfo, credentials, batch, columns, ssiList, ssio, authorizations, waitForWrites, samplerConfig, batchTimeOut, classLoaderContext, executionHints, resultHandler, this, ___protocolFactory, ___transport);
+      update_call method_call = new update_call(tinfo, credentials, keyExtent, mutation, durability, resultHandler, this, ___protocolFactory, ___transport);
       this.___currentMethod = method_call;
       ___manager.call(method_call);
     }
 
-    public static class startMultiScan_call extends org.apache.thrift.async.TAsyncMethodCall<org.apache.accumulo.core.dataImpl.thrift.InitialMultiScan> {
+    public static class update_call extends org.apache.thrift.async.TAsyncMethodCall<Void> {
       private org.apache.accumulo.core.trace.thrift.TInfo tinfo;
       private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials;
-      private java.util.Map<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent,java.util.List<org.apache.accumulo.core.dataImpl.thrift.TRange>> batch;
-      private java.util.List<org.apache.accumulo.core.dataImpl.thrift.TColumn> columns;
-      private java.util.List<org.apache.accumulo.core.dataImpl.thrift.IterInfo> ssiList;
-      private java.util.Map<java.lang.String,java.util.Map<java.lang.String,java.lang.String>> ssio;
-      private java.util.List<java.nio.ByteBuffer> authorizations;
-      private boolean waitForWrites;
-      private TSamplerConfiguration samplerConfig;
-      private long batchTimeOut;
-      private java.lang.String classLoaderContext;
-      private java.util.Map<java.lang.String,java.lang.String> executionHints;
-      public startMultiScan_call(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.util.Map<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent,java.util.List<org.apache.accumulo.core.dataImpl.thrift.TRange>> batch, java.util.List<org.apache.accumulo.core.dataImpl.thrift.TColumn> columns, java.util.List<org.apache.accumulo.core.dataImpl.thrift.IterInfo> ssiList, java.util.Map<java.lang.String,java.util.Map<java. [...]
+      private org.apache.accumulo.core.dataImpl.thrift.TKeyExtent keyExtent;
+      private org.apache.accumulo.core.dataImpl.thrift.TMutation mutation;
+      private TDurability durability;
+      public update_call(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent keyExtent, org.apache.accumulo.core.dataImpl.thrift.TMutation mutation, TDurability durability, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transpor [...]
         super(client, protocolFactory, transport, resultHandler, false);
         this.tinfo = tinfo;
         this.credentials = credentials;
-        this.batch = batch;
-        this.columns = columns;
-        this.ssiList = ssiList;
-        this.ssio = ssio;
-        this.authorizations = authorizations;
-        this.waitForWrites = waitForWrites;
-        this.samplerConfig = samplerConfig;
-        this.batchTimeOut = batchTimeOut;
-        this.classLoaderContext = classLoaderContext;
-        this.executionHints = executionHints;
+        this.keyExtent = keyExtent;
+        this.mutation = mutation;
+        this.durability = durability;
       }
 
       public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
-        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("startMultiScan", org.apache.thrift.protocol.TMessageType.CALL, 0));
-        startMultiScan_args args = new startMultiScan_args();
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("update", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        update_args args = new update_args();
         args.setTinfo(tinfo);
         args.setCredentials(credentials);
-        args.setBatch(batch);
-        args.setColumns(columns);
-        args.setSsiList(ssiList);
-        args.setSsio(ssio);
-        args.setAuthorizations(authorizations);
-        args.setWaitForWrites(waitForWrites);
-        args.setSamplerConfig(samplerConfig);
-        args.setBatchTimeOut(batchTimeOut);
-        args.setClassLoaderContext(classLoaderContext);
-        args.setExecutionHints(executionHints);
+        args.setKeyExtent(keyExtent);
+        args.setMutation(mutation);
+        args.setDurability(durability);
         args.write(prot);
         prot.writeMessageEnd();
       }
 
-      public org.apache.accumulo.core.dataImpl.thrift.InitialMultiScan getResult() throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, TSampleNotPresentException, org.apache.thrift.TException {
+      public Void getResult() throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, NotServingTabletException, ConstraintViolationException, 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_startMultiScan();
+        return null;
       }
     }
 
-    public void continueMultiScan(org.apache.accumulo.core.trace.thrift.TInfo tinfo, long scanID, org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.dataImpl.thrift.MultiScanResult> resultHandler) throws org.apache.thrift.TException {
+    public void startConditionalUpdate(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.util.List<java.nio.ByteBuffer> authorizations, java.lang.String tableID, TDurability durability, java.lang.String classLoaderContext, org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.dataImpl.thrift.TConditionalSession> resultHandler) throws org.apache.thrift.TException {
       checkReady();
-      continueMultiScan_call method_call = new continueMultiScan_call(tinfo, scanID, resultHandler, this, ___protocolFactory, ___transport);
+      startConditionalUpdate_call method_call = new startConditionalUpdate_call(tinfo, credentials, authorizations, tableID, durability, classLoaderContext, resultHandler, this, ___protocolFactory, ___transport);
       this.___currentMethod = method_call;
       ___manager.call(method_call);
     }
 
-    public static class continueMultiScan_call extends org.apache.thrift.async.TAsyncMethodCall<org.apache.accumulo.core.dataImpl.thrift.MultiScanResult> {
+    public static class startConditionalUpdate_call extends org.apache.thrift.async.TAsyncMethodCall<org.apache.accumulo.core.dataImpl.thrift.TConditionalSession> {
       private org.apache.accumulo.core.trace.thrift.TInfo tinfo;
-      private long scanID;
-      public continueMultiScan_call(org.apache.accumulo.core.trace.thrift.TInfo tinfo, long scanID, org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.dataImpl.thrift.MultiScanResult> 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.securityImpl.thrift.TCredentials credentials;
+      private java.util.List<java.nio.ByteBuffer> authorizations;
+      private java.lang.String tableID;
+      private TDurability durability;
+      private java.lang.String classLoaderContext;
+      public startConditionalUpdate_call(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.util.List<java.nio.ByteBuffer> authorizations, java.lang.String tableID, TDurability durability, java.lang.String classLoaderContext, org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.dataImpl.thrift.TConditionalSession> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protoc [...]
         super(client, protocolFactory, transport, resultHandler, false);
         this.tinfo = tinfo;
-        this.scanID = scanID;
+        this.credentials = credentials;
+        this.authorizations = authorizations;
+        this.tableID = tableID;
+        this.durability = durability;
+        this.classLoaderContext = classLoaderContext;
       }
 
       public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
-        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("continueMultiScan", org.apache.thrift.protocol.TMessageType.CALL, 0));
-        continueMultiScan_args args = new continueMultiScan_args();
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("startConditionalUpdate", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        startConditionalUpdate_args args = new startConditionalUpdate_args();
         args.setTinfo(tinfo);
-        args.setScanID(scanID);
+        args.setCredentials(credentials);
+        args.setAuthorizations(authorizations);
+        args.setTableID(tableID);
+        args.setDurability(durability);
+        args.setClassLoaderContext(classLoaderContext);
         args.write(prot);
         prot.writeMessageEnd();
       }
 
-      public org.apache.accumulo.core.dataImpl.thrift.MultiScanResult getResult() throws NoSuchScanIDException, TSampleNotPresentException, org.apache.thrift.TException {
+      public org.apache.accumulo.core.dataImpl.thrift.TConditionalSession getResult() throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.thrift.TException {
         if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
           throw new java.lang.IllegalStateException("Method call not finished!");
         }
         org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
         org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
-        return (new Client(prot)).recv_continueMultiScan();
+        return (new Client(prot)).recv_startConditionalUpdate();
       }
     }
 
-    public void closeMultiScan(org.apache.accumulo.core.trace.thrift.TInfo tinfo, long scanID, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
+    public void conditionalUpdate(org.apache.accumulo.core.trace.thrift.TInfo tinfo, long sessID, java.util.Map<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent,java.util.List<org.apache.accumulo.core.dataImpl.thrift.TConditionalMutation>> mutations, java.util.List<java.lang.String> symbols, org.apache.thrift.async.AsyncMethodCallback<java.util.List<org.apache.accumulo.core.dataImpl.thrift.TCMResult>> resultHandler) throws org.apache.thrift.TException {
       checkReady();
-      closeMultiScan_call method_call = new closeMultiScan_call(tinfo, scanID, resultHandler, this, ___protocolFactory, ___transport);
+      conditionalUpdate_call method_call = new conditionalUpdate_call(tinfo, sessID, mutations, symbols, resultHandler, this, ___protocolFactory, ___transport);
       this.___currentMethod = method_call;
       ___manager.call(method_call);
     }
 
-    public static class closeMultiScan_call extends org.apache.thrift.async.TAsyncMethodCall<Void> {
+    public static class conditionalUpdate_call extends org.apache.thrift.async.TAsyncMethodCall<java.util.List<org.apache.accumulo.core.dataImpl.thrift.TCMResult>> {
       private org.apache.accumulo.core.trace.thrift.TInfo tinfo;
-      private long scanID;
-      public closeMultiScan_call(org.apache.accumulo.core.trace.thrift.TInfo tinfo, long scanID, 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 long sessID;
+      private java.util.Map<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent,java.util.List<org.apache.accumulo.core.dataImpl.thrift.TConditionalMutation>> mutations;
+      private java.util.List<java.lang.String> symbols;
+      public conditionalUpdate_call(org.apache.accumulo.core.trace.thrift.TInfo tinfo, long sessID, java.util.Map<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent,java.util.List<org.apache.accumulo.core.dataImpl.thrift.TConditionalMutation>> mutations, java.util.List<java.lang.String> symbols, org.apache.thrift.async.AsyncMethodCallback<java.util.List<org.apache.accumulo.core.dataImpl.thrift.TCMResult>> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.prot [...]
         super(client, protocolFactory, transport, resultHandler, false);
         this.tinfo = tinfo;
-        this.scanID = scanID;
+        this.sessID = sessID;
+        this.mutations = mutations;
+        this.symbols = symbols;
       }
 
       public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
-        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("closeMultiScan", org.apache.thrift.protocol.TMessageType.CALL, 0));
-        closeMultiScan_args args = new closeMultiScan_args();
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("conditionalUpdate", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        conditionalUpdate_args args = new conditionalUpdate_args();
         args.setTinfo(tinfo);
-        args.setScanID(scanID);
+        args.setSessID(sessID);
+        args.setMutations(mutations);
+        args.setSymbols(symbols);
         args.write(prot);
         prot.writeMessageEnd();
       }
 
-      public Void getResult() throws NoSuchScanIDException, org.apache.thrift.TException {
+      public java.util.List<org.apache.accumulo.core.dataImpl.thrift.TCMResult> getResult() throws NoSuchScanIDException, 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;
+        return (new Client(prot)).recv_conditionalUpdate();
       }
     }
 
-    public void startUpdate(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, TDurability durability, org.apache.thrift.async.AsyncMethodCallback<java.lang.Long> resultHandler) throws org.apache.thrift.TException {
-      checkReady();
-      startUpdate_call method_call = new startUpdate_call(tinfo, credentials, durability, resultHandler, this, ___protocolFactory, ___transport);
-      this.___currentMethod = method_call;
-      ___manager.call(method_call);
-    }
-
-    public static class startUpdate_call extends org.apache.thrift.async.TAsyncMethodCall<java.lang.Long> {
-      private org.apache.accumulo.core.trace.thrift.TInfo tinfo;
-      private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials;
-      private TDurability durability;
-      public startUpdate_call(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, TDurability durability, org.apache.thrift.async.AsyncMethodCallback<java.lang.Long> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
-        super(client, protocolFactory, transport, resultHandler, false);
-        this.tinfo = tinfo;
-        this.credentials = credentials;
-        this.durability = durability;
-      }
-
-      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
-        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("startUpdate", org.apache.thrift.protocol.TMessageType.CALL, 0));
-        startUpdate_args args = new startUpdate_args();
-        args.setTinfo(tinfo);
-        args.setCredentials(credentials);
-        args.setDurability(durability);
-        args.write(prot);
-        prot.writeMessageEnd();
-      }
-
-      public java.lang.Long getResult() throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.thrift.TException {
-        if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
-          throw new java.lang.IllegalStateException("Method call not finished!");
-        }
-        org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
-        org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
-        return (new Client(prot)).recv_startUpdate();
-      }
-    }
-
-    public void applyUpdates(org.apache.accumulo.core.trace.thrift.TInfo tinfo, long updateID, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent keyExtent, java.util.List<org.apache.accumulo.core.dataImpl.thrift.TMutation> mutations, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
-      checkReady();
-      applyUpdates_call method_call = new applyUpdates_call(tinfo, updateID, keyExtent, mutations, resultHandler, this, ___protocolFactory, ___transport);
-      this.___currentMethod = method_call;
-      ___manager.call(method_call);
-    }
-
-    public static class applyUpdates_call extends org.apache.thrift.async.TAsyncMethodCall<Void> {
-      private org.apache.accumulo.core.trace.thrift.TInfo tinfo;
-      private long updateID;
-      private org.apache.accumulo.core.dataImpl.thrift.TKeyExtent keyExtent;
-      private java.util.List<org.apache.accumulo.core.dataImpl.thrift.TMutation> mutations;
-      public applyUpdates_call(org.apache.accumulo.core.trace.thrift.TInfo tinfo, long updateID, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent keyExtent, java.util.List<org.apache.accumulo.core.dataImpl.thrift.TMutation> mutations, 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.thri [...]
-        super(client, protocolFactory, transport, resultHandler, true);
-        this.tinfo = tinfo;
-        this.updateID = updateID;
-        this.keyExtent = keyExtent;
-        this.mutations = mutations;
-      }
-
-      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
-        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("applyUpdates", org.apache.thrift.protocol.TMessageType.ONEWAY, 0));
-        applyUpdates_args args = new applyUpdates_args();
-        args.setTinfo(tinfo);
-        args.setUpdateID(updateID);
-        args.setKeyExtent(keyExtent);
-        args.setMutations(mutations);
-        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 void closeUpdate(org.apache.accumulo.core.trace.thrift.TInfo tinfo, long updateID, org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.dataImpl.thrift.UpdateErrors> resultHandler) throws org.apache.thrift.TException {
-      checkReady();
-      closeUpdate_call method_call = new closeUpdate_call(tinfo, updateID, resultHandler, this, ___protocolFactory, ___transport);
-      this.___currentMethod = method_call;
-      ___manager.call(method_call);
-    }
-
-    public static class closeUpdate_call extends org.apache.thrift.async.TAsyncMethodCall<org.apache.accumulo.core.dataImpl.thrift.UpdateErrors> {
-      private org.apache.accumulo.core.trace.thrift.TInfo tinfo;
-      private long updateID;
-      public closeUpdate_call(org.apache.accumulo.core.trace.thrift.TInfo tinfo, long updateID, org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.dataImpl.thrift.UpdateErrors> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
-        super(client, protocolFactory, transport, resultHandler, false);
-        this.tinfo = tinfo;
-        this.updateID = updateID;
-      }
-
-      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
-        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("closeUpdate", org.apache.thrift.protocol.TMessageType.CALL, 0));
-        closeUpdate_args args = new closeUpdate_args();
-        args.setTinfo(tinfo);
-        args.setUpdateID(updateID);
-        args.write(prot);
-        prot.writeMessageEnd();
-      }
-
-      public org.apache.accumulo.core.dataImpl.thrift.UpdateErrors getResult() throws NoSuchScanIDException, 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_closeUpdate();
-      }
-    }
-
-    public void update(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent keyExtent, org.apache.accumulo.core.dataImpl.thrift.TMutation mutation, TDurability durability, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
-      checkReady();
-      update_call method_call = new update_call(tinfo, credentials, keyExtent, mutation, durability, resultHandler, this, ___protocolFactory, ___transport);
-      this.___currentMethod = method_call;
-      ___manager.call(method_call);
-    }
-
-    public static class update_call extends org.apache.thrift.async.TAsyncMethodCall<Void> {
-      private org.apache.accumulo.core.trace.thrift.TInfo tinfo;
-      private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials;
-      private org.apache.accumulo.core.dataImpl.thrift.TKeyExtent keyExtent;
-      private org.apache.accumulo.core.dataImpl.thrift.TMutation mutation;
-      private TDurability durability;
-      public update_call(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent keyExtent, org.apache.accumulo.core.dataImpl.thrift.TMutation mutation, TDurability durability, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transpor [...]
-        super(client, protocolFactory, transport, resultHandler, false);
-        this.tinfo = tinfo;
-        this.credentials = credentials;
-        this.keyExtent = keyExtent;
-        this.mutation = mutation;
-        this.durability = durability;
-      }
-
-      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
-        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("update", org.apache.thrift.protocol.TMessageType.CALL, 0));
-        update_args args = new update_args();
-        args.setTinfo(tinfo);
-        args.setCredentials(credentials);
-        args.setKeyExtent(keyExtent);
-        args.setMutation(mutation);
-        args.setDurability(durability);
-        args.write(prot);
-        prot.writeMessageEnd();
-      }
-
-      public Void getResult() throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, NotServingTabletException, ConstraintViolationException, org.apache.thrift.TException {
-        if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
-          throw new java.lang.IllegalStateException("Method call not finished!");
-        }
-        org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
-        org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
-        return null;
-      }
-    }
-
-    public void startConditionalUpdate(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.util.List<java.nio.ByteBuffer> authorizations, java.lang.String tableID, TDurability durability, java.lang.String classLoaderContext, org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.dataImpl.thrift.TConditionalSession> resultHandler) throws org.apache.thrift.TException {
-      checkReady();
-      startConditionalUpdate_call method_call = new startConditionalUpdate_call(tinfo, credentials, authorizations, tableID, durability, classLoaderContext, resultHandler, this, ___protocolFactory, ___transport);
-      this.___currentMethod = method_call;
-      ___manager.call(method_call);
-    }
-
-    public static class startConditionalUpdate_call extends org.apache.thrift.async.TAsyncMethodCall<org.apache.accumulo.core.dataImpl.thrift.TConditionalSession> {
-      private org.apache.accumulo.core.trace.thrift.TInfo tinfo;
-      private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials;
-      private java.util.List<java.nio.ByteBuffer> authorizations;
-      private java.lang.String tableID;
-      private TDurability durability;
-      private java.lang.String classLoaderContext;
-      public startConditionalUpdate_call(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.util.List<java.nio.ByteBuffer> authorizations, java.lang.String tableID, TDurability durability, java.lang.String classLoaderContext, org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.dataImpl.thrift.TConditionalSession> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protoc [...]
-        super(client, protocolFactory, transport, resultHandler, false);
-        this.tinfo = tinfo;
-        this.credentials = credentials;
-        this.authorizations = authorizations;
-        this.tableID = tableID;
-        this.durability = durability;
-        this.classLoaderContext = classLoaderContext;
-      }
-
-      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
-        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("startConditionalUpdate", org.apache.thrift.protocol.TMessageType.CALL, 0));
-        startConditionalUpdate_args args = new startConditionalUpdate_args();
-        args.setTinfo(tinfo);
-        args.setCredentials(credentials);
-        args.setAuthorizations(authorizations);
-        args.setTableID(tableID);
-        args.setDurability(durability);
-        args.setClassLoaderContext(classLoaderContext);
-        args.write(prot);
-        prot.writeMessageEnd();
-      }
-
-      public org.apache.accumulo.core.dataImpl.thrift.TConditionalSession getResult() throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.thrift.TException {
-        if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
-          throw new java.lang.IllegalStateException("Method call not finished!");
-        }
-        org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
-        org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
-        return (new Client(prot)).recv_startConditionalUpdate();
-      }
-    }
-
-    public void conditionalUpdate(org.apache.accumulo.core.trace.thrift.TInfo tinfo, long sessID, java.util.Map<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent,java.util.List<org.apache.accumulo.core.dataImpl.thrift.TConditionalMutation>> mutations, java.util.List<java.lang.String> symbols, org.apache.thrift.async.AsyncMethodCallback<java.util.List<org.apache.accumulo.core.dataImpl.thrift.TCMResult>> resultHandler) throws org.apache.thrift.TException {
-      checkReady();
-      conditionalUpdate_call method_call = new conditionalUpdate_call(tinfo, sessID, mutations, symbols, resultHandler, this, ___protocolFactory, ___transport);
-      this.___currentMethod = method_call;
-      ___manager.call(method_call);
-    }
-
-    public static class conditionalUpdate_call extends org.apache.thrift.async.TAsyncMethodCall<java.util.List<org.apache.accumulo.core.dataImpl.thrift.TCMResult>> {
-      private org.apache.accumulo.core.trace.thrift.TInfo tinfo;
-      private long sessID;
-      private java.util.Map<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent,java.util.List<org.apache.accumulo.core.dataImpl.thrift.TConditionalMutation>> mutations;
-      private java.util.List<java.lang.String> symbols;
-      public conditionalUpdate_call(org.apache.accumulo.core.trace.thrift.TInfo tinfo, long sessID, java.util.Map<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent,java.util.List<org.apache.accumulo.core.dataImpl.thrift.TConditionalMutation>> mutations, java.util.List<java.lang.String> symbols, org.apache.thrift.async.AsyncMethodCallback<java.util.List<org.apache.accumulo.core.dataImpl.thrift.TCMResult>> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.prot [...]
-        super(client, protocolFactory, transport, resultHandler, false);
-        this.tinfo = tinfo;
-        this.sessID = sessID;
-        this.mutations = mutations;
-        this.symbols = symbols;
-      }
-
-      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
-        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("conditionalUpdate", org.apache.thrift.protocol.TMessageType.CALL, 0));
-        conditionalUpdate_args args = new conditionalUpdate_args();
-        args.setTinfo(tinfo);
-        args.setSessID(sessID);
-        args.setMutations(mutations);
-        args.setSymbols(symbols);
-        args.write(prot);
-        prot.writeMessageEnd();
-      }
-
-      public java.util.List<org.apache.accumulo.core.dataImpl.thrift.TCMResult> getResult() throws NoSuchScanIDException, 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_conditionalUpdate();
-      }
-    }
-
-    public void invalidateConditionalUpdate(org.apache.accumulo.core.trace.thrift.TInfo tinfo, long sessID, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
+    public void invalidateConditionalUpdate(org.apache.accumulo.core.trace.thrift.TInfo tinfo, long sessID, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
       checkReady();
       invalidateConditionalUpdate_call method_call = new invalidateConditionalUpdate_call(tinfo, sessID, resultHandler, this, ___protocolFactory, ___transport);
       this.___currentMethod = method_call;
@@ -2393,77 +1863,42 @@ public class TabletClientService {
       }
     }
 
-    public void getActiveScans(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<java.util.List<ActiveScan>> resultHandler) throws org.apache.thrift.TException {
+    public void getActiveCompactions(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<java.util.List<ActiveCompaction>> resultHandler) throws org.apache.thrift.TException {
       checkReady();
-      getActiveScans_call method_call = new getActiveScans_call(tinfo, credentials, resultHandler, this, ___protocolFactory, ___transport);
+      getActiveCompactions_call method_call = new getActiveCompactions_call(tinfo, credentials, resultHandler, this, ___protocolFactory, ___transport);
       this.___currentMethod = method_call;
       ___manager.call(method_call);
     }
 
-    public static class getActiveScans_call extends org.apache.thrift.async.TAsyncMethodCall<java.util.List<ActiveScan>> {
+    public static class getActiveCompactions_call extends org.apache.thrift.async.TAsyncMethodCall<java.util.List<ActiveCompaction>> {
       private org.apache.accumulo.core.trace.thrift.TInfo tinfo;
       private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials;
-      public getActiveScans_call(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<java.util.List<ActiveScan>> 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 getActiveCompactions_call(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<java.util.List<ActiveCompaction>> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
         super(client, protocolFactory, transport, resultHandler, false);
         this.tinfo = tinfo;
         this.credentials = credentials;
       }
 
       public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
-        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("getActiveScans", org.apache.thrift.protocol.TMessageType.CALL, 0));
-        getActiveScans_args args = new getActiveScans_args();
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("getActiveCompactions", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        getActiveCompactions_args args = new getActiveCompactions_args();
         args.setTinfo(tinfo);
         args.setCredentials(credentials);
         args.write(prot);
         prot.writeMessageEnd();
       }
 
-      public java.util.List<ActiveScan> getResult() throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.thrift.TException {
+      public java.util.List<ActiveCompaction> getResult() throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.thrift.TException {
         if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
           throw new java.lang.IllegalStateException("Method call not finished!");
         }
         org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
         org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
-        return (new Client(prot)).recv_getActiveScans();
+        return (new Client(prot)).recv_getActiveCompactions();
       }
     }
 
-    public void getActiveCompactions(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<java.util.List<ActiveCompaction>> resultHandler) throws org.apache.thrift.TException {
-      checkReady();
-      getActiveCompactions_call method_call = new getActiveCompactions_call(tinfo, credentials, resultHandler, this, ___protocolFactory, ___transport);
-      this.___currentMethod = method_call;
-      ___manager.call(method_call);
-    }
-
-    public static class getActiveCompactions_call extends org.apache.thrift.async.TAsyncMethodCall<java.util.List<ActiveCompaction>> {
-      private org.apache.accumulo.core.trace.thrift.TInfo tinfo;
-      private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials;
-      public getActiveCompactions_call(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<java.util.List<ActiveCompaction>> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
-        super(client, protocolFactory, transport, resultHandler, false);
-        this.tinfo = tinfo;
-        this.credentials = credentials;
-      }
-
-      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
-        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("getActiveCompactions", org.apache.thrift.protocol.TMessageType.CALL, 0));
-        getActiveCompactions_args args = new getActiveCompactions_args();
-        args.setTinfo(tinfo);
-        args.setCredentials(credentials);
-        args.write(prot);
-        prot.writeMessageEnd();
-      }
-
-      public java.util.List<ActiveCompaction> getResult() throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.thrift.TException {
-        if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
-          throw new java.lang.IllegalStateException("Method call not finished!");
-        }
-        org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
-        org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
-        return (new Client(prot)).recv_getActiveCompactions();
-      }
-    }
-
-    public void removeLogs(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.util.List<java.lang.String> filenames, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
+    public void removeLogs(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.util.List<java.lang.String> filenames, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
       checkReady();
       removeLogs_call method_call = new removeLogs_call(tinfo, credentials, filenames, resultHandler, this, ___protocolFactory, ___transport);
       this.___currentMethod = method_call;
@@ -2877,12 +2312,6 @@ public class TabletClientService {
     }
 
     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("startScan", new startScan());
-      processMap.put("continueScan", new continueScan());
-      processMap.put("closeScan", new closeScan());
-      processMap.put("startMultiScan", new startMultiScan());
-      processMap.put("continueMultiScan", new continueMultiScan());
-      processMap.put("closeMultiScan", new closeMultiScan());
       processMap.put("startUpdate", new startUpdate());
       processMap.put("applyUpdates", new applyUpdates());
       processMap.put("closeUpdate", new closeUpdate());
@@ -2905,7 +2334,6 @@ public class TabletClientService {
       processMap.put("getHistoricalStats", new getHistoricalStats());
       processMap.put("halt", new halt());
       processMap.put("fastHalt", new fastHalt());
-      processMap.put("getActiveScans", new getActiveScans());
       processMap.put("getActiveCompactions", new getActiveCompactions());
       processMap.put("removeLogs", new removeLogs());
       processMap.put("getActiveLogs", new getActiveLogs());
@@ -2920,191 +2348,6 @@ public class TabletClientService {
       return processMap;
     }
 
-    public static class startScan<I extends Iface> extends org.apache.thrift.ProcessFunction<I, startScan_args> {
-      public startScan() {
-        super("startScan");
-      }
-
-      public startScan_args getEmptyArgsInstance() {
-        return new startScan_args();
-      }
-
-      protected boolean isOneway() {
-        return false;
-      }
-
-      @Override
-      protected boolean rethrowUnhandledExceptions() {
-        return false;
-      }
-
-      public startScan_result getResult(I iface, startScan_args args) throws org.apache.thrift.TException {
-        startScan_result result = new startScan_result();
-        try {
-          result.success = iface.startScan(args.tinfo, args.credentials, args.extent, args.range, args.columns, args.batchSize, args.ssiList, args.ssio, args.authorizations, args.waitForWrites, args.isolated, args.readaheadThreshold, args.samplerConfig, args.batchTimeOut, args.classLoaderContext, args.executionHints);
-        } catch (org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException sec) {
-          result.sec = sec;
-        } catch (NotServingTabletException nste) {
-          result.nste = nste;
-        } catch (TooManyFilesException tmfe) {
-          result.tmfe = tmfe;
-        } catch (TSampleNotPresentException tsnpe) {
-          result.tsnpe = tsnpe;
-        }
-        return result;
-      }
-    }
-
-    public static class continueScan<I extends Iface> extends org.apache.thrift.ProcessFunction<I, continueScan_args> {
-      public continueScan() {
-        super("continueScan");
-      }
-
-      public continueScan_args getEmptyArgsInstance() {
-        return new continueScan_args();
-      }
-
-      protected boolean isOneway() {
-        return false;
-      }
-
-      @Override
-      protected boolean rethrowUnhandledExceptions() {
-        return false;
-      }
-
-      public continueScan_result getResult(I iface, continueScan_args args) throws org.apache.thrift.TException {
-        continueScan_result result = new continueScan_result();
-        try {
-          result.success = iface.continueScan(args.tinfo, args.scanID);
-        } catch (NoSuchScanIDException nssi) {
-          result.nssi = nssi;
-        } catch (NotServingTabletException nste) {
-          result.nste = nste;
-        } catch (TooManyFilesException tmfe) {
-          result.tmfe = tmfe;
-        } catch (TSampleNotPresentException tsnpe) {
-          result.tsnpe = tsnpe;
-        }
-        return result;
-      }
-    }
-
-    public static class closeScan<I extends Iface> extends org.apache.thrift.ProcessFunction<I, closeScan_args> {
-      public closeScan() {
-        super("closeScan");
-      }
-
-      public closeScan_args getEmptyArgsInstance() {
-        return new closeScan_args();
-      }
-
-      protected boolean isOneway() {
-        return true;
-      }
-
-      @Override
-      protected boolean rethrowUnhandledExceptions() {
-        return false;
-      }
-
-      public org.apache.thrift.TBase getResult(I iface, closeScan_args args) throws org.apache.thrift.TException {
-        iface.closeScan(args.tinfo, args.scanID);
-        return null;
-      }
-    }
-
-    public static class startMultiScan<I extends Iface> extends org.apache.thrift.ProcessFunction<I, startMultiScan_args> {
-      public startMultiScan() {
-        super("startMultiScan");
-      }
-
-      public startMultiScan_args getEmptyArgsInstance() {
-        return new startMultiScan_args();
-      }
-
-      protected boolean isOneway() {
-        return false;
-      }
-
-      @Override
-      protected boolean rethrowUnhandledExceptions() {
-        return false;
-      }
-
-      public startMultiScan_result getResult(I iface, startMultiScan_args args) throws org.apache.thrift.TException {
-        startMultiScan_result result = new startMultiScan_result();
-        try {
-          result.success = iface.startMultiScan(args.tinfo, args.credentials, args.batch, args.columns, args.ssiList, args.ssio, args.authorizations, args.waitForWrites, args.samplerConfig, args.batchTimeOut, args.classLoaderContext, args.executionHints);
-        } catch (org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException sec) {
-          result.sec = sec;
-        } catch (TSampleNotPresentException tsnpe) {
-          result.tsnpe = tsnpe;
-        }
-        return result;
-      }
-    }
-
-    public static class continueMultiScan<I extends Iface> extends org.apache.thrift.ProcessFunction<I, continueMultiScan_args> {
-      public continueMultiScan() {
-        super("continueMultiScan");
-      }
-
-      public continueMultiScan_args getEmptyArgsInstance() {
-        return new continueMultiScan_args();
-      }
-
-      protected boolean isOneway() {
-        return false;
-      }
-
-      @Override
-      protected boolean rethrowUnhandledExceptions() {
-        return false;
-      }
-
-      public continueMultiScan_result getResult(I iface, continueMultiScan_args args) throws org.apache.thrift.TException {
-        continueMultiScan_result result = new continueMultiScan_result();
-        try {
-          result.success = iface.continueMultiScan(args.tinfo, args.scanID);
-        } catch (NoSuchScanIDException nssi) {
-          result.nssi = nssi;
-        } catch (TSampleNotPresentException tsnpe) {
-          result.tsnpe = tsnpe;
-        }
-        return result;
-      }
-    }
-
-    public static class closeMultiScan<I extends Iface> extends org.apache.thrift.ProcessFunction<I, closeMultiScan_args> {
-      public closeMultiScan() {
-        super("closeMultiScan");
-      }
-
-      public closeMultiScan_args getEmptyArgsInstance() {
-        return new closeMultiScan_args();
-      }
-
-      protected boolean isOneway() {
-        return false;
-      }
-
-      @Override
-      protected boolean rethrowUnhandledExceptions() {
-        return false;
-      }
-
-      public closeMultiScan_result getResult(I iface, closeMultiScan_args args) throws org.apache.thrift.TException {
-        closeMultiScan_result result = new closeMultiScan_result();
-        try {
-          iface.closeMultiScan(args.tinfo, args.scanID);
-        } catch (NoSuchScanIDException nssi) {
-          result.nssi = nssi;
-        }
-        return result;
-      }
-    }
-
     public static class startUpdate<I extends Iface> extends org.apache.thrift.ProcessFunction<I, startUpdate_args> {
       public startUpdate() {
         super("startUpdate");
@@ -3696,35 +2939,6 @@ public class TabletClientService {
       }
     }
 
-    public static class getActiveScans<I extends Iface> extends org.apache.thrift.ProcessFunction<I, getActiveScans_args> {
-      public getActiveScans() {
-        super("getActiveScans");
-      }
-
-      public getActiveScans_args getEmptyArgsInstance() {
-        return new getActiveScans_args();
-      }
-
-      protected boolean isOneway() {
-        return false;
-      }
-
-      @Override
-      protected boolean rethrowUnhandledExceptions() {
-        return false;
-      }
-
-      public getActiveScans_result getResult(I iface, getActiveScans_args args) throws org.apache.thrift.TException {
-        getActiveScans_result result = new getActiveScans_result();
-        try {
-          result.success = iface.getActiveScans(args.tinfo, args.credentials);
-        } catch (org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException sec) {
-          result.sec = sec;
-        }
-        return result;
-      }
-    }
-
     public static class getActiveCompactions<I extends Iface> extends org.apache.thrift.ProcessFunction<I, getActiveCompactions_args> {
       public getActiveCompactions() {
         super("getActiveCompactions");
@@ -4040,12 +3254,6 @@ public class TabletClientService {
     }
 
     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("startScan", new startScan());
-      processMap.put("continueScan", new continueScan());
-      processMap.put("closeScan", new closeScan());
-      processMap.put("startMultiScan", new startMultiScan());
-      processMap.put("continueMultiScan", new continueMultiScan());
-      processMap.put("closeMultiScan", new closeMultiScan());
       processMap.put("startUpdate", new startUpdate());
       processMap.put("applyUpdates", new applyUpdates());
       processMap.put("closeUpdate", new closeUpdate());
@@ -4068,7 +3276,6 @@ public class TabletClientService {
       processMap.put("getHistoricalStats", new getHistoricalStats());
       processMap.put("halt", new halt());
       processMap.put("fastHalt", new fastHalt());
-      processMap.put("getActiveScans", new getActiveScans());
       processMap.put("getActiveCompactions", new getActiveCompactions());
       processMap.put("removeLogs", new removeLogs());
       processMap.put("getActiveLogs", new getActiveLogs());
@@ -4083,21 +3290,22 @@ public class TabletClientService {
       return processMap;
     }
 
-    public static class startScan<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, startScan_args, org.apache.accumulo.core.dataImpl.thrift.InitialScan> {
-      public startScan() {
-        super("startScan");
+    public static class startUpdate<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, startUpdate_args, java.lang.Long> {
+      public startUpdate() {
+        super("startUpdate");
       }
 
-      public startScan_args getEmptyArgsInstance() {
-        return new startScan_args();
+      public startUpdate_args getEmptyArgsInstance() {
+        return new startUpdate_args();
       }
 
-      public org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.dataImpl.thrift.InitialScan> getResultHandler(final org.apache.thrift.server.AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) {
+      public org.apache.thrift.async.AsyncMethodCallback<java.lang.Long> getResultHandler(final org.apache.thrift.server.AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) {
         final org.apache.thrift.AsyncProcessFunction fcall = this;
-        return new org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.dataImpl.thrift.InitialScan>() { 
-          public void onComplete(org.apache.accumulo.core.dataImpl.thrift.InitialScan o) {
-            startScan_result result = new startScan_result();
+        return new org.apache.thrift.async.AsyncMethodCallback<java.lang.Long>() { 
+          public void onComplete(java.lang.Long o) {
+            startUpdate_result result = new startUpdate_result();
             result.success = o;
+            result.setSuccessIsSet(true);
             try {
               fcall.sendResponse(fb, result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
             } catch (org.apache.thrift.transport.TTransportException e) {
@@ -4111,23 +3319,11 @@ public class TabletClientService {
           public void onError(java.lang.Exception e) {
             byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
             org.apache.thrift.TSerializable msg;
-            startScan_result result = new startScan_result();
+            startUpdate_result result = new startUpdate_result();
             if (e instanceof org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException) {
               result.sec = (org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException) e;
               result.setSecIsSet(true);
               msg = result;
-            } else if (e instanceof NotServingTabletException) {
-              result.nste = (NotServingTabletException) e;
-              result.setNsteIsSet(true);
-              msg = result;
-            } else if (e instanceof TooManyFilesException) {
-              result.tmfe = (TooManyFilesException) e;
-              result.setTmfeIsSet(true);
-              msg = result;
-            } else if (e instanceof TSampleNotPresentException) {
-              result.tsnpe = (TSampleNotPresentException) e;
-              result.setTsnpeIsSet(true);
-              msg = result;
             } else if (e instanceof org.apache.thrift.transport.TTransportException) {
               _LOGGER.error("TTransportException inside handler", e);
               fb.close();
@@ -4155,25 +3351,59 @@ public class TabletClientService {
         return false;
       }
 
-      public void start(I iface, startScan_args args, org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.dataImpl.thrift.InitialScan> resultHandler) throws org.apache.thrift.TException {
-        iface.startScan(args.tinfo, args.credentials, args.extent, args.range, args.columns, args.batchSize, args.ssiList, args.ssio, args.authorizations, args.waitForWrites, args.isolated, args.readaheadThreshold, args.samplerConfig, args.batchTimeOut, args.classLoaderContext, args.executionHints,resultHandler);
+      public void start(I iface, startUpdate_args args, org.apache.thrift.async.AsyncMethodCallback<java.lang.Long> resultHandler) throws org.apache.thrift.TException {
+        iface.startUpdate(args.tinfo, args.credentials, args.durability,resultHandler);
+      }
+    }
+
+    public static class applyUpdates<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, applyUpdates_args, Void> {
+      public applyUpdates() {
+        super("applyUpdates");
+      }
+
+      public applyUpdates_args getEmptyArgsInstance() {
+        return new applyUpdates_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) {
+          }
+          public void onError(java.lang.Exception e) {
+            if (e instanceof org.apache.thrift.transport.TTransportException) {
+              _LOGGER.error("TTransportException inside handler", e);
+              fb.close();
+            } else {
+              _LOGGER.error("Exception inside oneway handler", e);
+            }
+          }
+        };
+      }
+
+      protected boolean isOneway() {
+        return true;
+      }
+
+      public void start(I iface, applyUpdates_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
+        iface.applyUpdates(args.tinfo, args.updateID, args.keyExtent, args.mutations,resultHandler);
       }
     }
 
-    public static class continueScan<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, continueScan_args, org.apache.accumulo.core.dataImpl.thrift.ScanResult> {
-      public continueScan() {
-        super("continueScan");
+    public static class closeUpdate<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, closeUpdate_args, org.apache.accumulo.core.dataImpl.thrift.UpdateErrors> {
+      public closeUpdate() {
+        super("closeUpdate");
       }
 
-      public continueScan_args getEmptyArgsInstance() {
-        return new continueScan_args();
+      public closeUpdate_args getEmptyArgsInstance() {
+        return new closeUpdate_args();
       }
 
-      public org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.dataImpl.thrift.ScanResult> getResultHandler(final org.apache.thrift.server.AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) {
+      public org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.dataImpl.thrift.UpdateErrors> getResultHandler(final org.apache.thrift.server.AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) {
         final org.apache.thrift.AsyncProcessFunction fcall = this;
-        return new org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.dataImpl.thrift.ScanResult>() { 
-          public void onComplete(org.apache.accumulo.core.dataImpl.thrift.ScanResult o) {
-            continueScan_result result = new continueScan_result();
+        return new org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.dataImpl.thrift.UpdateErrors>() { 
+          public void onComplete(org.apache.accumulo.core.dataImpl.thrift.UpdateErrors o) {
+            closeUpdate_result result = new closeUpdate_result();
             result.success = o;
             try {
               fcall.sendResponse(fb, result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
@@ -4188,23 +3418,11 @@ public class TabletClientService {
           public void onError(java.lang.Exception e) {
             byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
             org.apache.thrift.TSerializable msg;
-            continueScan_result result = new continueScan_result();
+            closeUpdate_result result = new closeUpdate_result();
             if (e instanceof NoSuchScanIDException) {
               result.nssi = (NoSuchScanIDException) e;
               result.setNssiIsSet(true);
               msg = result;
-            } else if (e instanceof NotServingTabletException) {
-              result.nste = (NotServingTabletException) e;
-              result.setNsteIsSet(true);
-              msg = result;
-            } else if (e instanceof TooManyFilesException) {
-              result.tmfe = (TooManyFilesException) e;
-              result.setTmfeIsSet(true);
-              msg = result;
-            } else if (e instanceof TSampleNotPresentException) {
-              result.tsnpe = (TSampleNotPresentException) e;
-              result.setTsnpeIsSet(true);
-              msg = result;
             } else if (e instanceof org.apache.thrift.transport.TTransportException) {
               _LOGGER.error("TTransportException inside handler", e);
               fb.close();
@@ -4232,60 +3450,25 @@ public class TabletClientService {
         return false;
       }
 
-      public void start(I iface, continueScan_args args, org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.dataImpl.thrift.ScanResult> resultHandler) throws org.apache.thrift.TException {
-        iface.continueScan(args.tinfo, args.scanID,resultHandler);
+      public void start(I iface, closeUpdate_args args, org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.dataImpl.thrift.UpdateErrors> resultHandler) throws org.apache.thrift.TException {
+        iface.closeUpdate(args.tinfo, args.updateID,resultHandler);
       }
     }
 
-    public static class closeScan<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, closeScan_args, Void> {
-      public closeScan() {
-        super("closeScan");
+    public static class update<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, update_args, Void> {
+      public update() {
+        super("update");
       }
 
-      public closeScan_args getEmptyArgsInstance() {
-        return new closeScan_args();
-      }
+      public update_args getEmptyArgsInstance() {
+        return new update_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) {
-          }
-          public void onError(java.lang.Exception e) {
-            if (e instanceof org.apache.thrift.transport.TTransportException) {
-              _LOGGER.error("TTransportException inside handler", e);
-              fb.close();
-            } else {
-              _LOGGER.error("Exception inside oneway handler", e);
-            }
-          }
-        };
-      }
-
-      protected boolean isOneway() {
-        return true;
-      }
-
-      public void start(I iface, closeScan_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
-        iface.closeScan(args.tinfo, args.scanID,resultHandler);
-      }
-    }
-
-    public static class startMultiScan<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, startMultiScan_args, org.apache.accumulo.core.dataImpl.thrift.InitialMultiScan> {
-      public startMultiScan() {
-        super("startMultiScan");
-      }
-
-      public startMultiScan_args getEmptyArgsInstance() {
-        return new startMultiScan_args();
-      }
-
-      public org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.dataImpl.thrift.InitialMultiScan> getResultHandler(final org.apache.thrift.server.AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) {
-        final org.apache.thrift.AsyncProcessFunction fcall = this;
-        return new org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.dataImpl.thrift.InitialMultiScan>() { 
-          public void onComplete(org.apache.accumulo.core.dataImpl.thrift.InitialMultiScan o) {
-            startMultiScan_result result = new startMultiScan_result();
-            result.success = o;
+            update_result result = new update_result();
             try {
               fcall.sendResponse(fb, result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
             } catch (org.apache.thrift.transport.TTransportException e) {
@@ -4299,14 +3482,18 @@ public class TabletClientService {
           public void onError(java.lang.Exception e) {
             byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
             org.apache.thrift.TSerializable msg;
-            startMultiScan_result result = new startMultiScan_result();
+            update_result result = new update_result();
             if (e instanceof org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException) {
               result.sec = (org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException) e;
               result.setSecIsSet(true);
               msg = result;
-            } else if (e instanceof TSampleNotPresentException) {
-              result.tsnpe = (TSampleNotPresentException) e;
-              result.setTsnpeIsSet(true);
+            } else if (e instanceof NotServingTabletException) {
+              result.nste = (NotServingTabletException) e;
+              result.setNsteIsSet(true);
+              msg = result;
+            } else if (e instanceof ConstraintViolationException) {
+              result.cve = (ConstraintViolationException) e;
+              result.setCveIsSet(true);
               msg = result;
             } else if (e instanceof org.apache.thrift.transport.TTransportException) {
               _LOGGER.error("TTransportException inside handler", e);
@@ -4335,25 +3522,25 @@ public class TabletClientService {
         return false;
       }
 
-      public void start(I iface, startMultiScan_args args, org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.dataImpl.thrift.InitialMultiScan> resultHandler) throws org.apache.thrift.TException {
-        iface.startMultiScan(args.tinfo, args.credentials, args.batch, args.columns, args.ssiList, args.ssio, args.authorizations, args.waitForWrites, args.samplerConfig, args.batchTimeOut, args.classLoaderContext, args.executionHints,resultHandler);
+      public void start(I iface, update_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
+        iface.update(args.tinfo, args.credentials, args.keyExtent, args.mutation, args.durability,resultHandler);
       }
     }
 
-    public static class continueMultiScan<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, continueMultiScan_args, org.apache.accumulo.core.dataImpl.thrift.MultiScanResult> {
-      public continueMultiScan() {
-        super("continueMultiScan");
+    public static class startConditionalUpdate<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, startConditionalUpdate_args, org.apache.accumulo.core.dataImpl.thrift.TConditionalSession> {
+      public startConditionalUpdate() {
+        super("startConditionalUpdate");
       }
 
-      public continueMultiScan_args getEmptyArgsInstance() {
-        return new continueMultiScan_args();
+      public startConditionalUpdate_args getEmptyArgsInstance() {
+        return new startConditionalUpdate_args();
       }
 
-      public org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.dataImpl.thrift.MultiScanResult> getResultHandler(final org.apache.thrift.server.AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) {
+      public org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.dataImpl.thrift.TConditionalSession> getResultHandler(final org.apache.thrift.server.AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) {
         final org.apache.thrift.AsyncProcessFunction fcall = this;
-        return new org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.dataImpl.thrift.MultiScanResult>() { 
-          public void onComplete(org.apache.accumulo.core.dataImpl.thrift.MultiScanResult o) {
-            continueMultiScan_result result = new continueMultiScan_result();
+        return new org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.dataImpl.thrift.TConditionalSession>() { 
+          public void onComplete(org.apache.accumulo.core.dataImpl.thrift.TConditionalSession o) {
+            startConditionalUpdate_result result = new startConditionalUpdate_result();
             result.success = o;
             try {
               fcall.sendResponse(fb, result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
@@ -4368,14 +3555,10 @@ public class TabletClientService {
           public void onError(java.lang.Exception e) {
             byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
             org.apache.thrift.TSerializable msg;
-            continueMultiScan_result result = new continueMultiScan_result();
-            if (e instanceof NoSuchScanIDException) {
-              result.nssi = (NoSuchScanIDException) e;
-              result.setNssiIsSet(true);
-              msg = result;
-            } else if (e instanceof TSampleNotPresentException) {
-              result.tsnpe = (TSampleNotPresentException) e;
-              result.setTsnpeIsSet(true);
+            startConditionalUpdate_result result = new startConditionalUpdate_result();
+            if (e instanceof org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException) {
+              result.sec = (org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException) e;
+              result.setSecIsSet(true);
               msg = result;
             } else if (e instanceof org.apache.thrift.transport.TTransportException) {
               _LOGGER.error("TTransportException inside handler", e);
@@ -4404,25 +3587,26 @@ public class TabletClientService {
         return false;
       }
 
-      public void start(I iface, continueMultiScan_args args, org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.dataImpl.thrift.MultiScanResult> resultHandler) throws org.apache.thrift.TException {
-        iface.continueMultiScan(args.tinfo, args.scanID,resultHandler);
+      public void start(I iface, startConditionalUpdate_args args, org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.dataImpl.thrift.TConditionalSession> resultHandler) throws org.apache.thrift.TException {
+        iface.startConditionalUpdate(args.tinfo, args.credentials, args.authorizations, args.tableID, args.durability, args.classLoaderContext,resultHandler);
       }
     }
 
-    public static class closeMultiScan<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, closeMultiScan_args, Void> {
-      public closeMultiScan() {
-        super("closeMultiScan");
+    public static class conditionalUpdate<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, conditionalUpdate_args, java.util.List<org.apache.accumulo.core.dataImpl.thrift.TCMResult>> {
+      public conditionalUpdate() {
+        super("conditionalUpdate");
       }
 
-      public closeMultiScan_args getEmptyArgsInstance() {
-        return new closeMultiScan_args();
+      public conditionalUpdate_args getEmptyArgsInstance() {
+        return new conditionalUpdate_args();
       }
 
-      public org.apache.thrift.async.AsyncMethodCallback<Void> getResultHandler(final org.apache.thrift.server.AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) {
+      public org.apache.thrift.async.AsyncMethodCallback<java.util.List<org.apache.accumulo.core.dataImpl.thrift.TCMResult>> 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) {
-            closeMultiScan_result result = new closeMultiScan_result();
+        return new org.apache.thrift.async.AsyncMethodCallback<java.util.List<org.apache.accumulo.core.dataImpl.thrift.TCMResult>>() { 
+          public void onComplete(java.util.List<org.apache.accumulo.core.dataImpl.thrift.TCMResult> o) {
+            conditionalUpdate_result result = new conditionalUpdate_result();
+            result.success = o;
             try {
               fcall.sendResponse(fb, result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
             } catch (org.apache.thrift.transport.TTransportException e) {
@@ -4436,7 +3620,7 @@ public class TabletClientService {
           public void onError(java.lang.Exception e) {
             byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
             org.apache.thrift.TSerializable msg;
-            closeMultiScan_result result = new closeMultiScan_result();
+            conditionalUpdate_result result = new conditionalUpdate_result();
             if (e instanceof NoSuchScanIDException) {
               result.nssi = (NoSuchScanIDException) e;
               result.setNssiIsSet(true);
@@ -4468,27 +3652,25 @@ public class TabletClientService {
         return false;
       }
 
-      public void start(I iface, closeMultiScan_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
-        iface.closeMultiScan(args.tinfo, args.scanID,resultHandler);
+      public void start(I iface, conditionalUpdate_args args, org.apache.thrift.async.AsyncMethodCallback<java.util.List<org.apache.accumulo.core.dataImpl.thrift.TCMResult>> resultHandler) throws org.apache.thrift.TException {
+        iface.conditionalUpdate(args.tinfo, args.sessID, args.mutations, args.symbols,resultHandler);
       }
     }
 
-    public static class startUpdate<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, startUpdate_args, java.lang.Long> {
-      public startUpdate() {
-        super("startUpdate");
+    public static class invalidateConditionalUpdate<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, invalidateConditionalUpdate_args, Void> {
+      public invalidateConditionalUpdate() {
+        super("invalidateConditionalUpdate");
       }
 
-      public startUpdate_args getEmptyArgsInstance() {
-        return new startUpdate_args();
+      public invalidateConditionalUpdate_args getEmptyArgsInstance() {
+        return new invalidateConditionalUpdate_args();
       }
 
-      public org.apache.thrift.async.AsyncMethodCallback<java.lang.Long> getResultHandler(final org.apache.thrift.server.AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) {
+      public org.apache.thrift.async.AsyncMethodCallback<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<java.lang.Long>() { 
-          public void onComplete(java.lang.Long o) {
-            startUpdate_result result = new startUpdate_result();
-            result.success = o;
-            result.setSuccessIsSet(true);
+        return new org.apache.thrift.async.AsyncMethodCallback<Void>() { 
+          public void onComplete(Void o) {
+            invalidateConditionalUpdate_result result = new invalidateConditionalUpdate_result();
             try {
               fcall.sendResponse(fb, result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
             } catch (org.apache.thrift.transport.TTransportException e) {
@@ -4502,12 +3684,8 @@ public class TabletClientService {
           public void onError(java.lang.Exception e) {
             byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
             org.apache.thrift.TSerializable msg;
-            startUpdate_result result = new startUpdate_result();
-            if (e instanceof org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException) {
-              result.sec = (org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException) e;
-              result.setSecIsSet(true);
-              msg = result;
-            } else if (e instanceof org.apache.thrift.transport.TTransportException) {
+            invalidateConditionalUpdate_result result = new invalidateConditionalUpdate_result();
+            if (e instanceof org.apache.thrift.transport.TTransportException) {
               _LOGGER.error("TTransportException inside handler", e);
               fb.close();
               return;
@@ -4534,18 +3712,18 @@ public class TabletClientService {
         return false;
       }
 
-      public void start(I iface, startUpdate_args args, org.apache.thrift.async.AsyncMethodCallback<java.lang.Long> resultHandler) throws org.apache.thrift.TException {
-        iface.startUpdate(args.tinfo, args.credentials, args.durability,resultHandler);
+      public void start(I iface, invalidateConditionalUpdate_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
+        iface.invalidateConditionalUpdate(args.tinfo, args.sessID,resultHandler);
       }
     }
 
-    public static class applyUpdates<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, applyUpdates_args, Void> {
-      public applyUpdates() {
-        super("applyUpdates");
+    public static class closeConditionalUpdate<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, closeConditionalUpdate_args, Void> {
+      public closeConditionalUpdate() {
+        super("closeConditionalUpdate");
       }
 
-      public applyUpdates_args getEmptyArgsInstance() {
-        return new applyUpdates_args();
+      public closeConditionalUpdate_args getEmptyArgsInstance() {
+        return new closeConditionalUpdate_args();
       }
 
       public org.apache.thrift.async.AsyncMethodCallback<Void> getResultHandler(final org.apache.thrift.server.AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) {
@@ -4568,25 +3746,25 @@ public class TabletClientService {
         return true;
       }
 
-      public void start(I iface, applyUpdates_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
-        iface.applyUpdates(args.tinfo, args.updateID, args.keyExtent, args.mutations,resultHandler);
+      public void start(I iface, closeConditionalUpdate_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
+        iface.closeConditionalUpdate(args.tinfo, args.sessID,resultHandler);
       }
     }
 
-    public static class closeUpdate<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, closeUpdate_args, org.apache.accumulo.core.dataImpl.thrift.UpdateErrors> {
-      public closeUpdate() {
-        super("closeUpdate");
+    public static class bulkImport<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, bulkImport_args, java.util.List<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent>> {
+      public bulkImport() {
+        super("bulkImport");
       }
 
-      public closeUpdate_args getEmptyArgsInstance() {
-        return new closeUpdate_args();
+      public bulkImport_args getEmptyArgsInstance() {
+        return new bulkImport_args();
       }
 
-      public org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.dataImpl.thrift.UpdateErrors> getResultHandler(final org.apache.thrift.server.AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) {
+      public org.apache.thrift.async.AsyncMethodCallback<java.util.List<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent>> getResultHandler(final org.apache.thrift.server.AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) {
         final org.apache.thrift.AsyncProcessFunction fcall = this;
-        return new org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.dataImpl.thrift.UpdateErrors>() { 
-          public void onComplete(org.apache.accumulo.core.dataImpl.thrift.UpdateErrors o) {
-            closeUpdate_result result = new closeUpdate_result();
+        return new org.apache.thrift.async.AsyncMethodCallback<java.util.List<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent>>() { 
+          public void onComplete(java.util.List<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent> o) {
+            bulkImport_result result = new bulkImport_result();
             result.success = o;
             try {
               fcall.sendResponse(fb, result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
@@ -4601,10 +3779,10 @@ public class TabletClientService {
           public void onError(java.lang.Exception e) {
             byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
             org.apache.thrift.TSerializable msg;
-            closeUpdate_result result = new closeUpdate_result();
-            if (e instanceof NoSuchScanIDException) {
-              result.nssi = (NoSuchScanIDException) e;
-              result.setNssiIsSet(true);
+            bulkImport_result result = new bulkImport_result();
+            if (e instanceof org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException) {
+              result.sec = (org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException) e;
+              result.setSecIsSet(true);
               msg = result;
             } else if (e instanceof org.apache.thrift.transport.TTransportException) {
               _LOGGER.error("TTransportException inside handler", e);
@@ -4633,98 +3811,59 @@ public class TabletClientService {
         return false;
       }
 
-      public void start(I iface, closeUpdate_args args, org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.dataImpl.thrift.UpdateErrors> resultHandler) throws org.apache.thrift.TException {
-        iface.closeUpdate(args.tinfo, args.updateID,resultHandler);
+      public void start(I iface, bulkImport_args args, org.apache.thrift.async.AsyncMethodCallback<java.util.List<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent>> resultHandler) throws org.apache.thrift.TException {
+        iface.bulkImport(args.tinfo, args.credentials, args.tid, args.files, args.setTime,resultHandler);
       }
     }
 
-    public static class update<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, update_args, Void> {
-      public update() {
-        super("update");
+    public static class loadFiles<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, loadFiles_args, Void> {
+      public loadFiles() {
+        super("loadFiles");
       }
 
-      public update_args getEmptyArgsInstance() {
-        return new update_args();
+      public loadFiles_args getEmptyArgsInstance() {
+        return new loadFiles_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) {
-            update_result result = new update_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;
-            update_result result = new update_result();
-            if (e instanceof org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException) {
-              result.sec = (org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException) e;
-              result.setSecIsSet(true);
-              msg = result;
-            } else if (e instanceof NotServingTabletException) {
-              result.nste = (NotServingTabletException) e;
-              result.setNsteIsSet(true);
-              msg = result;
-            } else if (e instanceof ConstraintViolationException) {
-              result.cve = (ConstraintViolationException) e;
-              result.setCveIsSet(true);
-              msg = result;
-            } else if (e instanceof org.apache.thrift.transport.TTransportException) {
+            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();
+              _LOGGER.error("Exception inside oneway handler", e);
             }
           }
         };
       }
 
       protected boolean isOneway() {
-        return false;
+        return true;
       }
 
-      public void start(I iface, update_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
-        iface.update(args.tinfo, args.credentials, args.keyExtent, args.mutation, args.durability,resultHandler);
+      public void start(I iface, loadFiles_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
+        iface.loadFiles(args.tinfo, args.credentials, args.tid, args.dir, args.files, args.setTime,resultHandler);
       }
     }
 
-    public static class startConditionalUpdate<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, startConditionalUpdate_args, org.apache.accumulo.core.dataImpl.thrift.TConditionalSession> {
-      public startConditionalUpdate() {
-        super("startConditionalUpdate");
+    public static class splitTablet<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, splitTablet_args, Void> {
+      public splitTablet() {
+        super("splitTablet");
       }
 
-      public startConditionalUpdate_args getEmptyArgsInstance() {
-        return new startConditionalUpdate_args();
+      public splitTablet_args getEmptyArgsInstance() {
+        return new splitTablet_args();
       }
 
-      public org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.dataImpl.thrift.TConditionalSession> 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<org.apache.accumulo.core.dataImpl.thrift.TConditionalSession>() { 
-          public void onComplete(org.apache.accumulo.core.dataImpl.thrift.TConditionalSession o) {
-            startConditionalUpdate_result result = new startConditionalUpdate_result();
-            result.success = o;
+        return new org.apache.thrift.async.AsyncMethodCallback<Void>() { 
+          public void onComplete(Void o) {
+            splitTablet_result result = new splitTablet_result();
             try {
               fcall.sendResponse(fb, result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
             } catch (org.apache.thrift.transport.TTransportException e) {
@@ -4738,11 +3877,15 @@ public class TabletClientService {
           public void onError(java.lang.Exception e) {
             byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
             org.apache.thrift.TSerializable msg;
-            startConditionalUpdate_result result = new startConditionalUpdate_result();
+            splitTablet_result result = new splitTablet_result();
             if (e instanceof org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException) {
               result.sec = (org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException) e;
               result.setSecIsSet(true);
               msg = result;
+            } else if (e instanceof NotServingTabletException) {
+              result.nste = (NotServingTabletException) e;
+              result.setNsteIsSet(true);
+              msg = result;
             } else if (e instanceof org.apache.thrift.transport.TTransportException) {
               _LOGGER.error("TTransportException inside handler", e);
               fb.close();
@@ -4770,143 +3913,86 @@ public class TabletClientService {
         return false;
       }
 
-      public void start(I iface, startConditionalUpdate_args args, org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.dataImpl.thrift.TConditionalSession> resultHandler) throws org.apache.thrift.TException {
-        iface.startConditionalUpdate(args.tinfo, args.credentials, args.authorizations, args.tableID, args.durability, args.classLoaderContext,resultHandler);
+      public void start(I iface, splitTablet_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
+        iface.splitTablet(args.tinfo, args.credentials, args.extent, args.splitPoint,resultHandler);
       }
     }
 
-    public static class conditionalUpdate<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, conditionalUpdate_args, java.util.List<org.apache.accumulo.core.dataImpl.thrift.TCMResult>> {
-      public conditionalUpdate() {
-        super("conditionalUpdate");
+    public static class loadTablet<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, loadTablet_args, Void> {
+      public loadTablet() {
+        super("loadTablet");
       }
 
-      public conditionalUpdate_args getEmptyArgsInstance() {
-        return new conditionalUpdate_args();
+      public loadTablet_args getEmptyArgsInstance() {
+        return new loadTablet_args();
       }
 
-      public org.apache.thrift.async.AsyncMethodCallback<java.util.List<org.apache.accumulo.core.dataImpl.thrift.TCMResult>> 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<java.util.List<org.apache.accumulo.core.dataImpl.thrift.TCMResult>>() { 
-          public void onComplete(java.util.List<org.apache.accumulo.core.dataImpl.thrift.TCMResult> o) {
-            conditionalUpdate_result result = new conditionalUpdate_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);
-            }
+        return new org.apache.thrift.async.AsyncMethodCallback<Void>() { 
+          public void onComplete(Void o) {
           }
           public void onError(java.lang.Exception e) {
-            byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
-            org.apache.thrift.TSerializable msg;
-            conditionalUpdate_result result = new conditionalUpdate_result();
-            if (e instanceof NoSuchScanIDException) {
-              result.nssi = (NoSuchScanIDException) e;
-              result.setNssiIsSet(true);
-              msg = result;
-            } else if (e instanceof org.apache.thrift.transport.TTransportException) {
+            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();
+              _LOGGER.error("Exception inside oneway handler", e);
             }
           }
         };
       }
 
       protected boolean isOneway() {
-        return false;
+        return true;
       }
 
-      public void start(I iface, conditionalUpdate_args args, org.apache.thrift.async.AsyncMethodCallback<java.util.List<org.apache.accumulo.core.dataImpl.thrift.TCMResult>> resultHandler) throws org.apache.thrift.TException {
-        iface.conditionalUpdate(args.tinfo, args.sessID, args.mutations, args.symbols,resultHandler);
+      public void start(I iface, loadTablet_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
+        iface.loadTablet(args.tinfo, args.credentials, args.lock, args.extent,resultHandler);
       }
     }
 
-    public static class invalidateConditionalUpdate<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, invalidateConditionalUpdate_args, Void> {
-      public invalidateConditionalUpdate() {
-        super("invalidateConditionalUpdate");
+    public static class unloadTablet<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, unloadTablet_args, Void> {
+      public unloadTablet() {
+        super("unloadTablet");
       }
 
-      public invalidateConditionalUpdate_args getEmptyArgsInstance() {
-        return new invalidateConditionalUpdate_args();
+      public unloadTablet_args getEmptyArgsInstance() {
+        return new unloadTablet_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) {
-            invalidateConditionalUpdate_result result = new invalidateConditionalUpdate_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;
-            invalidateConditionalUpdate_result result = new invalidateConditionalUpdate_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();
+              _LOGGER.error("Exception inside oneway handler", e);
             }
           }
         };
       }
 
       protected boolean isOneway() {
-        return false;
+        return true;
       }
 
-      public void start(I iface, invalidateConditionalUpdate_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
-        iface.invalidateConditionalUpdate(args.tinfo, args.sessID,resultHandler);
+      public void start(I iface, unloadTablet_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
+        iface.unloadTablet(args.tinfo, args.credentials, args.lock, args.extent, args.goal, args.requestTime,resultHandler);
       }
     }
 
-    public static class closeConditionalUpdate<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, closeConditionalUpdate_args, Void> {
-      public closeConditionalUpdate() {
-        super("closeConditionalUpdate");
+    public static class flush<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, flush_args, Void> {
+      public flush() {
+        super("flush");
       }
 
-      public closeConditionalUpdate_args getEmptyArgsInstance() {
-        return new closeConditionalUpdate_args();
+      public flush_args getEmptyArgsInstance() {
+        return new flush_args();
       }
 
       public org.apache.thrift.async.AsyncMethodCallback<Void> getResultHandler(final org.apache.thrift.server.AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) {
@@ -4929,83 +4015,52 @@ public class TabletClientService {
         return true;
       }
 
-      public void start(I iface, closeConditionalUpdate_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
-        iface.closeConditionalUpdate(args.tinfo, args.sessID,resultHandler);
+      public void start(I iface, flush_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
+        iface.flush(args.tinfo, args.credentials, args.lock, args.tableId, args.startRow, args.endRow,resultHandler);
       }
     }
 
-    public static class bulkImport<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, bulkImport_args, java.util.List<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent>> {
-      public bulkImport() {
-        super("bulkImport");
+    public static class flushTablet<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, flushTablet_args, Void> {
+      public flushTablet() {
+        super("flushTablet");
       }
 
-      public bulkImport_args getEmptyArgsInstance() {
-        return new bulkImport_args();
+      public flushTablet_args getEmptyArgsInstance() {
+        return new flushTablet_args();
       }
 
-      public org.apache.thrift.async.AsyncMethodCallback<java.util.List<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent>> 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<java.util.List<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent>>() { 
-          public void onComplete(java.util.List<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent> o) {
-            bulkImport_result result = new bulkImport_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);
-            }
+        return new org.apache.thrift.async.AsyncMethodCallback<Void>() { 
+          public void onComplete(Void o) {
           }
           public void onError(java.lang.Exception e) {
-            byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
-            org.apache.thrift.TSerializable msg;
-            bulkImport_result result = new bulkImport_result();
-            if (e instanceof org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException) {
-              result.sec = (org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException) e;
-              result.setSecIsSet(true);
-              msg = result;
-            } else if (e instanceof org.apache.thrift.transport.TTransportException) {
+            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();
+              _LOGGER.error("Exception inside oneway handler", e);
             }
           }
         };
       }
 
       protected boolean isOneway() {
-        return false;
+        return true;
       }
 
-      public void start(I iface, bulkImport_args args, org.apache.thrift.async.AsyncMethodCallback<java.util.List<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent>> resultHandler) throws org.apache.thrift.TException {
-        iface.bulkImport(args.tinfo, args.credentials, args.tid, args.files, args.setTime,resultHandler);
+      public void start(I iface, flushTablet_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
+        iface.flushTablet(args.tinfo, args.credentials, args.lock, args.extent,resultHandler);
       }
     }
 
-    public static class loadFiles<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, loadFiles_args, Void> {
-      public loadFiles() {
-        super("loadFiles");
+    public static class chop<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, chop_args, Void> {
+      public chop() {
+        super("chop");
       }
 
-      public loadFiles_args getEmptyArgsInstance() {
-        return new loadFiles_args();
+      public chop_args getEmptyArgsInstance() {
+        return new chop_args();
       }
 
       public org.apache.thrift.async.AsyncMethodCallback<Void> getResultHandler(final org.apache.thrift.server.AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) {
@@ -5028,25 +4083,60 @@ public class TabletClientService {
         return true;
       }
 
-      public void start(I iface, loadFiles_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
-        iface.loadFiles(args.tinfo, args.credentials, args.tid, args.dir, args.files, args.setTime,resultHandler);
+      public void start(I iface, chop_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
+        iface.chop(args.tinfo, args.credentials, args.lock, args.extent,resultHandler);
       }
     }
 
-    public static class splitTablet<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, splitTablet_args, Void> {
-      public splitTablet() {
-        super("splitTablet");
+    public static class compact<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, compact_args, Void> {
+      public compact() {
+        super("compact");
       }
 
-      public splitTablet_args getEmptyArgsInstance() {
-        return new splitTablet_args();
+      public compact_args getEmptyArgsInstance() {
+        return new compact_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) {
-            splitTablet_result result = new splitTablet_result();
+          }
+          public void onError(java.lang.Exception e) {
+            if (e instanceof org.apache.thrift.transport.TTransportException) {
+              _LOGGER.error("TTransportException inside handler", e);
+              fb.close();
+            } else {
+              _LOGGER.error("Exception inside oneway handler", e);
+            }
+          }
+        };
+      }
+
+      protected boolean isOneway() {
+        return true;
+      }
+
+      public void start(I iface, compact_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
+        iface.compact(args.tinfo, args.credentials, args.lock, args.tableId, args.startRow, args.endRow,resultHandler);
+      }
+    }
+
+    public static class getTabletServerStatus<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, getTabletServerStatus_args, org.apache.accumulo.core.master.thrift.TabletServerStatus> {
+      public getTabletServerStatus() {
+        super("getTabletServerStatus");
+      }
+
+      public getTabletServerStatus_args getEmptyArgsInstance() {
+        return new getTabletServerStatus_args();
+      }
+
+      public org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.master.thrift.TabletServerStatus> getResultHandler(final org.apache.thrift.server.AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) {
+        final org.apache.thrift.AsyncProcessFunction fcall = this;
+        return new org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.master.thrift.TabletServerStatus>() { 
+          public void onComplete(org.apache.accumulo.core.master.thrift.TabletServerStatus o) {
+            getTabletServerStatus_result result = new getTabletServerStatus_result();
+            result.success = o;
             try {
               fcall.sendResponse(fb, result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
             } catch (org.apache.thrift.transport.TTransportException e) {
@@ -5060,15 +4150,11 @@ public class TabletClientService {
           public void onError(java.lang.Exception e) {
             byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
             org.apache.thrift.TSerializable msg;
-            splitTablet_result result = new splitTablet_result();
+            getTabletServerStatus_result result = new getTabletServerStatus_result();
             if (e instanceof org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException) {
               result.sec = (org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException) e;
               result.setSecIsSet(true);
               msg = result;
-            } else if (e instanceof NotServingTabletException) {
-              result.nste = (NotServingTabletException) e;
-              result.setNsteIsSet(true);
-              msg = result;
             } else if (e instanceof org.apache.thrift.transport.TTransportException) {
               _LOGGER.error("TTransportException inside handler", e);
               fb.close();
@@ -5096,120 +4182,212 @@ public class TabletClientService {
         return false;
       }
 
-      public void start(I iface, splitTablet_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
-        iface.splitTablet(args.tinfo, args.credentials, args.extent, args.splitPoint,resultHandler);
+      public void start(I iface, getTabletServerStatus_args args, org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.master.thrift.TabletServerStatus> resultHandler) throws org.apache.thrift.TException {
+        iface.getTabletServerStatus(args.tinfo, args.credentials,resultHandler);
       }
     }
 
-    public static class loadTablet<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, loadTablet_args, Void> {
-      public loadTablet() {
-        super("loadTablet");
+    public static class getTabletStats<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, getTabletStats_args, java.util.List<TabletStats>> {
+      public getTabletStats() {
+        super("getTabletStats");
       }
 
-      public loadTablet_args getEmptyArgsInstance() {
-        return new loadTablet_args();
+      public getTabletStats_args getEmptyArgsInstance() {
+        return new getTabletStats_args();
       }
 
-      public org.apache.thrift.async.AsyncMethodCallback<Void> getResultHandler(final org.apache.thrift.server.AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) {
+      public org.apache.thrift.async.AsyncMethodCallback<java.util.List<TabletStats>> 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) {
-          }
-          public void onError(java.lang.Exception e) {
-            if (e instanceof org.apache.thrift.transport.TTransportException) {
-              _LOGGER.error("TTransportException inside handler", e);
+        return new org.apache.thrift.async.AsyncMethodCallback<java.util.List<TabletStats>>() { 
+          public void onComplete(java.util.List<TabletStats> o) {
+            getTabletStats_result result = new getTabletStats_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();
-            } else {
-              _LOGGER.error("Exception inside oneway handler", e);
+            } 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;
+            getTabletStats_result result = new getTabletStats_result();
+            if (e instanceof org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException) {
+              result.sec = (org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException) e;
+              result.setSecIsSet(true);
+              msg = result;
+            } else if (e instanceof org.apache.thrift.transport.TTransportException) {
+              _LOGGER.error("TTransportException inside handler", e);
+              fb.close();
+              return;
+            } else if (e instanceof org.apache.thrift.TApplicationException) {
+              _LOGGER.error("TApplicationException inside handler", e);
+              msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
+              msg = (org.apache.thrift.TApplicationException)e;
+            } else {
+              _LOGGER.error("Exception inside handler", e);
+              msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
+              msg = new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage());
+            }
+            try {
+              fcall.sendResponse(fb,msg,msgType,seqid);
+            } catch (java.lang.Exception ex) {
+              _LOGGER.error("Exception writing to internal frame buffer", ex);
+              fb.close();
+            }
+          }
+        };
       }
 
       protected boolean isOneway() {
-        return true;
+        return false;
       }
 
-      public void start(I iface, loadTablet_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
-        iface.loadTablet(args.tinfo, args.credentials, args.lock, args.extent,resultHandler);
+      public void start(I iface, getTabletStats_args args, org.apache.thrift.async.AsyncMethodCallback<java.util.List<TabletStats>> resultHandler) throws org.apache.thrift.TException {
+        iface.getTabletStats(args.tinfo, args.credentials, args.tableId,resultHandler);
       }
     }
 
-    public static class unloadTablet<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, unloadTablet_args, Void> {
-      public unloadTablet() {
-        super("unloadTablet");
+    public static class getHistoricalStats<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, getHistoricalStats_args, TabletStats> {
+      public getHistoricalStats() {
+        super("getHistoricalStats");
       }
 
-      public unloadTablet_args getEmptyArgsInstance() {
-        return new unloadTablet_args();
+      public getHistoricalStats_args getEmptyArgsInstance() {
+        return new getHistoricalStats_args();
       }
 
-      public org.apache.thrift.async.AsyncMethodCallback<Void> getResultHandler(final org.apache.thrift.server.AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) {
+      public org.apache.thrift.async.AsyncMethodCallback<TabletStats> 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) {
+        return new org.apache.thrift.async.AsyncMethodCallback<TabletStats>() { 
+          public void onComplete(TabletStats o) {
+            getHistoricalStats_result result = new getHistoricalStats_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) {
-            if (e instanceof org.apache.thrift.transport.TTransportException) {
+            byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
+            org.apache.thrift.TSerializable msg;
+            getHistoricalStats_result result = new getHistoricalStats_result();
+            if (e instanceof org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException) {
+              result.sec = (org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException) e;
+              result.setSecIsSet(true);
+              msg = result;
+            } else if (e instanceof org.apache.thrift.transport.TTransportException) {
               _LOGGER.error("TTransportException inside handler", e);
               fb.close();
+              return;
+            } else if (e instanceof org.apache.thrift.TApplicationException) {
+              _LOGGER.error("TApplicationException inside handler", e);
+              msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
+              msg = (org.apache.thrift.TApplicationException)e;
             } else {
-              _LOGGER.error("Exception inside oneway handler", e);
+              _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 true;
+        return false;
       }
 
-      public void start(I iface, unloadTablet_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
-        iface.unloadTablet(args.tinfo, args.credentials, args.lock, args.extent, args.goal, args.requestTime,resultHandler);
+      public void start(I iface, getHistoricalStats_args args, org.apache.thrift.async.AsyncMethodCallback<TabletStats> resultHandler) throws org.apache.thrift.TException {
+        iface.getHistoricalStats(args.tinfo, args.credentials,resultHandler);
       }
     }
 
-    public static class flush<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, flush_args, Void> {
-      public flush() {
-        super("flush");
+    public static class halt<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, halt_args, Void> {
+      public halt() {
+        super("halt");
       }
 
-      public flush_args getEmptyArgsInstance() {
-        return new flush_args();
+      public halt_args getEmptyArgsInstance() {
+        return new halt_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) {
+            halt_result result = new halt_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) {
-            if (e instanceof org.apache.thrift.transport.TTransportException) {
+            byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
+            org.apache.thrift.TSerializable msg;
+            halt_result result = new halt_result();
+            if (e instanceof org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException) {
+              result.sec = (org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException) e;
+              result.setSecIsSet(true);
+              msg = result;
+            } else if (e instanceof org.apache.thrift.transport.TTransportException) {
               _LOGGER.error("TTransportException inside handler", e);
               fb.close();
+              return;
+            } else if (e instanceof org.apache.thrift.TApplicationException) {
+              _LOGGER.error("TApplicationException inside handler", e);
+              msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
+              msg = (org.apache.thrift.TApplicationException)e;
             } else {
-              _LOGGER.error("Exception inside oneway handler", e);
+              _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 true;
+        return false;
       }
 
-      public void start(I iface, flush_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
-        iface.flush(args.tinfo, args.credentials, args.lock, args.tableId, args.startRow, args.endRow,resultHandler);
+      public void start(I iface, halt_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
+        iface.halt(args.tinfo, args.credentials, args.lock,resultHandler);
       }
     }
 
-    public static class flushTablet<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, flushTablet_args, Void> {
-      public flushTablet() {
-        super("flushTablet");
+    public static class fastHalt<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, fastHalt_args, Void> {
+      public fastHalt() {
+        super("fastHalt");
       }
 
-      public flushTablet_args getEmptyArgsInstance() {
-        return new flushTablet_args();
+      public fastHalt_args getEmptyArgsInstance() {
+        return new fastHalt_args();
       }
 
       public org.apache.thrift.async.AsyncMethodCallback<Void> getResultHandler(final org.apache.thrift.server.AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) {
@@ -5232,52 +4410,83 @@ public class TabletClientService {
         return true;
       }
 
-      public void start(I iface, flushTablet_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
-        iface.flushTablet(args.tinfo, args.credentials, args.lock, args.extent,resultHandler);
+      public void start(I iface, fastHalt_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
+        iface.fastHalt(args.tinfo, args.credentials, args.lock,resultHandler);
       }
     }
 
-    public static class chop<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, chop_args, Void> {
-      public chop() {
-        super("chop");
+    public static class getActiveCompactions<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, getActiveCompactions_args, java.util.List<ActiveCompaction>> {
+      public getActiveCompactions() {
+        super("getActiveCompactions");
       }
 
-      public chop_args getEmptyArgsInstance() {
-        return new chop_args();
+      public getActiveCompactions_args getEmptyArgsInstance() {
+        return new getActiveCompactions_args();
       }
 
-      public org.apache.thrift.async.AsyncMethodCallback<Void> getResultHandler(final org.apache.thrift.server.AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) {
+      public org.apache.thrift.async.AsyncMethodCallback<java.util.List<ActiveCompaction>> getResultHandler(final org.apache.thrift.server.AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) {
         final org.apache.thrift.AsyncProcessFunction fcall = this;
-        return new org.apache.thrift.async.AsyncMethodCallback<Void>() { 
-          public void onComplete(Void o) {
+        return new org.apache.thrift.async.AsyncMethodCallback<java.util.List<ActiveCompaction>>() { 
+          public void onComplete(java.util.List<ActiveCompaction> o) {
+            getActiveCompactions_result result = new getActiveCompactions_result();
+            result.success = o;
+            try {
+              fcall.sendResponse(fb, result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
+            } catch (org.apache.thrift.transport.TTransportException e) {
+              _LOGGER.error("TTransportException writing to internal frame buffer", e);
+              fb.close();
+            } catch (java.lang.Exception e) {
+              _LOGGER.error("Exception writing to internal frame buffer", e);
+              onError(e);
+            }
           }
           public void onError(java.lang.Exception e) {
-            if (e instanceof org.apache.thrift.transport.TTransportException) {
+            byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
+            org.apache.thrift.TSerializable msg;
+            getActiveCompactions_result result = new getActiveCompactions_result();
+            if (e instanceof org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException) {
+              result.sec = (org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException) e;
+              result.setSecIsSet(true);
+              msg = result;
+            } else if (e instanceof org.apache.thrift.transport.TTransportException) {
               _LOGGER.error("TTransportException inside handler", e);
               fb.close();
+              return;
+            } else if (e instanceof org.apache.thrift.TApplicationException) {
+              _LOGGER.error("TApplicationException inside handler", e);
+              msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
+              msg = (org.apache.thrift.TApplicationException)e;
             } else {
-              _LOGGER.error("Exception inside oneway handler", e);
+              _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 true;
+        return false;
       }
 
-      public void start(I iface, chop_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
-        iface.chop(args.tinfo, args.credentials, args.lock, args.extent,resultHandler);
+      public void start(I iface, getActiveCompactions_args args, org.apache.thrift.async.AsyncMethodCallback<java.util.List<ActiveCompaction>> resultHandler) throws org.apache.thrift.TException {
+        iface.getActiveCompactions(args.tinfo, args.credentials,resultHandler);
       }
     }
 
-    public static class compact<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, compact_args, Void> {
-      public compact() {
-        super("compact");
+    public static class removeLogs<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, removeLogs_args, Void> {
+      public removeLogs() {
+        super("removeLogs");
       }
 
-      public compact_args getEmptyArgsInstance() {
-        return new compact_args();
+      public removeLogs_args getEmptyArgsInstance() {
+        return new removeLogs_args();
       }
 
       public org.apache.thrift.async.AsyncMethodCallback<Void> getResultHandler(final org.apache.thrift.server.AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) {
@@ -5300,25 +4509,25 @@ public class TabletClientService {
         return true;
       }
 
-      public void start(I iface, compact_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
-        iface.compact(args.tinfo, args.credentials, args.lock, args.tableId, args.startRow, args.endRow,resultHandler);
+      public void start(I iface, removeLogs_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
+        iface.removeLogs(args.tinfo, args.credentials, args.filenames,resultHandler);
       }
     }
 
-    public static class getTabletServerStatus<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, getTabletServerStatus_args, org.apache.accumulo.core.master.thrift.TabletServerStatus> {
-      public getTabletServerStatus() {
-        super("getTabletServerStatus");
+    public static class getActiveLogs<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, getActiveLogs_args, java.util.List<java.lang.String>> {
+      public getActiveLogs() {
+        super("getActiveLogs");
       }
 
-      public getTabletServerStatus_args getEmptyArgsInstance() {
-        return new getTabletServerStatus_args();
+      public getActiveLogs_args getEmptyArgsInstance() {
+        return new getActiveLogs_args();
       }
 
-      public org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.master.thrift.TabletServerStatus> getResultHandler(final org.apache.thrift.server.AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) {
+      public org.apache.thrift.async.AsyncMethodCallback<java.util.List<java.lang.String>> getResultHandler(final org.apache.thrift.server.AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) {
         final org.apache.thrift.AsyncProcessFunction fcall = this;
-        return new org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.master.thrift.TabletServerStatus>() { 
-          public void onComplete(org.apache.accumulo.core.master.thrift.TabletServerStatus o) {
-            getTabletServerStatus_result result = new getTabletServerStatus_result();
+        return new org.apache.thrift.async.AsyncMethodCallback<java.util.List<java.lang.String>>() { 
+          public void onComplete(java.util.List<java.lang.String> o) {
+            getActiveLogs_result result = new getActiveLogs_result();
             result.success = o;
             try {
               fcall.sendResponse(fb, result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
@@ -5333,12 +4542,8 @@ public class TabletClientService {
           public void onError(java.lang.Exception e) {
             byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
             org.apache.thrift.TSerializable msg;
-            getTabletServerStatus_result result = new getTabletServerStatus_result();
-            if (e instanceof org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException) {
-              result.sec = (org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException) e;
-              result.setSecIsSet(true);
-              msg = result;
-            } else if (e instanceof org.apache.thrift.transport.TTransportException) {
+            getActiveLogs_result result = new getActiveLogs_result();
+            if (e instanceof org.apache.thrift.transport.TTransportException) {
               _LOGGER.error("TTransportException inside handler", e);
               fb.close();
               return;
@@ -5365,25 +4570,25 @@ public class TabletClientService {
         return false;
       }
 
-      public void start(I iface, getTabletServerStatus_args args, org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.master.thrift.TabletServerStatus> resultHandler) throws org.apache.thrift.TException {
-        iface.getTabletServerStatus(args.tinfo, args.credentials,resultHandler);
-      }
+      public void start(I iface, getActiveLogs_args args, org.apache.thrift.async.AsyncMethodCallback<java.util.List<java.lang.String>> resultHandler) throws org.apache.thrift.TException {
+        iface.getActiveLogs(args.tinfo, args.credentials,resultHandler);
+      }
     }
 
-    public static class getTabletStats<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, getTabletStats_args, java.util.List<TabletStats>> {
-      public getTabletStats() {
-        super("getTabletStats");
+    public static class startGetSummaries<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, startGetSummaries_args, org.apache.accumulo.core.dataImpl.thrift.TSummaries> {
+      public startGetSummaries() {
+        super("startGetSummaries");
       }
 
-      public getTabletStats_args getEmptyArgsInstance() {
-        return new getTabletStats_args();
+      public startGetSummaries_args getEmptyArgsInstance() {
+        return new startGetSummaries_args();
       }
 
-      public org.apache.thrift.async.AsyncMethodCallback<java.util.List<TabletStats>> getResultHandler(final org.apache.thrift.server.AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) {
+      public org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.dataImpl.thrift.TSummaries> 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<TabletStats>>() { 
-          public void onComplete(java.util.List<TabletStats> o) {
-            getTabletStats_result result = new getTabletStats_result();
+        return new org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.dataImpl.thrift.TSummaries>() { 
+          public void onComplete(org.apache.accumulo.core.dataImpl.thrift.TSummaries o) {
+            startGetSummaries_result result = new startGetSummaries_result();
             result.success = o;
             try {
               fcall.sendResponse(fb, result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
@@ -5398,11 +4603,15 @@ public class TabletClientService {
           public void onError(java.lang.Exception e) {
             byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
             org.apache.thrift.TSerializable msg;
-            getTabletStats_result result = new getTabletStats_result();
+            startGetSummaries_result result = new startGetSummaries_result();
             if (e instanceof org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException) {
               result.sec = (org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException) e;
               result.setSecIsSet(true);
               msg = result;
+            } else if (e instanceof org.apache.accumulo.core.clientImpl.thrift.ThriftTableOperationException) {
+              result.tope = (org.apache.accumulo.core.clientImpl.thrift.ThriftTableOperationException) e;
+              result.setTopeIsSet(true);
+              msg = result;
             } else if (e instanceof org.apache.thrift.transport.TTransportException) {
               _LOGGER.error("TTransportException inside handler", e);
               fb.close();
@@ -5430,25 +4639,25 @@ public class TabletClientService {
         return false;
       }
 
-      public void start(I iface, getTabletStats_args args, org.apache.thrift.async.AsyncMethodCallback<java.util.List<TabletStats>> resultHandler) throws org.apache.thrift.TException {
-        iface.getTabletStats(args.tinfo, args.credentials, args.tableId,resultHandler);
+      public void start(I iface, startGetSummaries_args args, org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.dataImpl.thrift.TSummaries> resultHandler) throws org.apache.thrift.TException {
+        iface.startGetSummaries(args.tinfo, args.credentials, args.request,resultHandler);
       }
     }
 
-    public static class getHistoricalStats<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, getHistoricalStats_args, TabletStats> {
-      public getHistoricalStats() {
-        super("getHistoricalStats");
+    public static class startGetSummariesForPartition<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, startGetSummariesForPartition_args, org.apache.accumulo.core.dataImpl.thrift.TSummaries> {
+      public startGetSummariesForPartition() {
+        super("startGetSummariesForPartition");
       }
 
-      public getHistoricalStats_args getEmptyArgsInstance() {
-        return new getHistoricalStats_args();
+      public startGetSummariesForPartition_args getEmptyArgsInstance() {
+        return new startGetSummariesForPartition_args();
       }
 
-      public org.apache.thrift.async.AsyncMethodCallback<TabletStats> getResultHandler(final org.apache.thrift.server.AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) {
+      public org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.dataImpl.thrift.TSummaries> 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<TabletStats>() { 
-          public void onComplete(TabletStats o) {
-            getHistoricalStats_result result = new getHistoricalStats_result();
+        return new org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.dataImpl.thrift.TSummaries>() { 
+          public void onComplete(org.apache.accumulo.core.dataImpl.thrift.TSummaries o) {
+            startGetSummariesForPartition_result result = new startGetSummariesForPartition_result();
             result.success = o;
             try {
               fcall.sendResponse(fb, result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
@@ -5463,7 +4672,7 @@ public class TabletClientService {
           public void onError(java.lang.Exception e) {
             byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
             org.apache.thrift.TSerializable msg;
-            getHistoricalStats_result result = new getHistoricalStats_result();
+            startGetSummariesForPartition_result result = new startGetSummariesForPartition_result();
             if (e instanceof org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException) {
               result.sec = (org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException) e;
               result.setSecIsSet(true);
@@ -5495,25 +4704,26 @@ public class TabletClientService {
         return false;
       }
 
-      public void start(I iface, getHistoricalStats_args args, org.apache.thrift.async.AsyncMethodCallback<TabletStats> resultHandler) throws org.apache.thrift.TException {
-        iface.getHistoricalStats(args.tinfo, args.credentials,resultHandler);
+      public void start(I iface, startGetSummariesForPartition_args args, org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.dataImpl.thrift.TSummaries> resultHandler) throws org.apache.thrift.TException {
+        iface.startGetSummariesForPartition(args.tinfo, args.credentials, args.request, args.modulus, args.remainder,resultHandler);
       }
     }
 
-    public static class halt<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, halt_args, Void> {
-      public halt() {
-        super("halt");
+    public static class startGetSummariesFromFiles<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, startGetSummariesFromFiles_args, org.apache.accumulo.core.dataImpl.thrift.TSummaries> {
+      public startGetSummariesFromFiles() {
+        super("startGetSummariesFromFiles");
       }
 
-      public halt_args getEmptyArgsInstance() {
-        return new halt_args();
+      public startGetSummariesFromFiles_args getEmptyArgsInstance() {
+        return new startGetSummariesFromFiles_args();
       }
 
-      public org.apache.thrift.async.AsyncMethodCallback<Void> getResultHandler(final org.apache.thrift.server.AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) {
+      public org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.dataImpl.thrift.TSummaries> 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) {
-            halt_result result = new halt_result();
+        return new org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.dataImpl.thrift.TSummaries>() { 
+          public void onComplete(org.apache.accumulo.core.dataImpl.thrift.TSummaries o) {
+            startGetSummariesFromFiles_result result = new startGetSummariesFromFiles_result();
+            result.success = o;
             try {
               fcall.sendResponse(fb, result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
             } catch (org.apache.thrift.transport.TTransportException e) {
@@ -5527,7 +4737,7 @@ public class TabletClientService {
           public void onError(java.lang.Exception e) {
             byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
             org.apache.thrift.TSerializable msg;
-            halt_result result = new halt_result();
+            startGetSummariesFromFiles_result result = new startGetSummariesFromFiles_result();
             if (e instanceof org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException) {
               result.sec = (org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException) e;
               result.setSecIsSet(true);
@@ -5559,59 +4769,90 @@ public class TabletClientService {
         return false;
       }
 
-      public void start(I iface, halt_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
-        iface.halt(args.tinfo, args.credentials, args.lock,resultHandler);
+      public void start(I iface, startGetSummariesFromFiles_args args, org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.dataImpl.thrift.TSummaries> resultHandler) throws org.apache.thrift.TException {
+        iface.startGetSummariesFromFiles(args.tinfo, args.credentials, args.request, args.files,resultHandler);
       }
     }
 
-    public static class fastHalt<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, fastHalt_args, Void> {
-      public fastHalt() {
-        super("fastHalt");
+    public static class contiuneGetSummaries<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, contiuneGetSummaries_args, org.apache.accumulo.core.dataImpl.thrift.TSummaries> {
+      public contiuneGetSummaries() {
+        super("contiuneGetSummaries");
       }
 
-      public fastHalt_args getEmptyArgsInstance() {
-        return new fastHalt_args();
+      public contiuneGetSummaries_args getEmptyArgsInstance() {
+        return new contiuneGetSummaries_args();
       }
 
-      public org.apache.thrift.async.AsyncMethodCallback<Void> getResultHandler(final org.apache.thrift.server.AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) {
+      public org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.dataImpl.thrift.TSummaries> 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) {
+        return new org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.dataImpl.thrift.TSummaries>() { 
+          public void onComplete(org.apache.accumulo.core.dataImpl.thrift.TSummaries o) {
+            contiuneGetSummaries_result result = new contiuneGetSummaries_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) {
-            if (e instanceof org.apache.thrift.transport.TTransportException) {
+            byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
+            org.apache.thrift.TSerializable msg;
+            contiuneGetSummaries_result result = new contiuneGetSummaries_result();
+            if (e instanceof NoSuchScanIDException) {
+              result.nssi = (NoSuchScanIDException) e;
+              result.setNssiIsSet(true);
+              msg = result;
+            } else if (e instanceof org.apache.thrift.transport.TTransportException) {
               _LOGGER.error("TTransportException inside handler", e);
               fb.close();
+              return;
+            } else if (e instanceof org.apache.thrift.TApplicationException) {
+              _LOGGER.error("TApplicationException inside handler", e);
+              msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
+              msg = (org.apache.thrift.TApplicationException)e;
             } else {
-              _LOGGER.error("Exception inside oneway handler", e);
+              _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 true;
+        return false;
       }
 
-      public void start(I iface, fastHalt_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
-        iface.fastHalt(args.tinfo, args.credentials, args.lock,resultHandler);
+      public void start(I iface, contiuneGetSummaries_args args, org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.dataImpl.thrift.TSummaries> resultHandler) throws org.apache.thrift.TException {
+        iface.contiuneGetSummaries(args.tinfo, args.sessionId,resultHandler);
       }
     }
 
-    public static class getActiveScans<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, getActiveScans_args, java.util.List<ActiveScan>> {
-      public getActiveScans() {
-        super("getActiveScans");
+    public static class getCompactionQueueInfo<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, getCompactionQueueInfo_args, java.util.List<TCompactionQueueSummary>> {
+      public getCompactionQueueInfo() {
+        super("getCompactionQueueInfo");
       }
 
-      public getActiveScans_args getEmptyArgsInstance() {
-        return new getActiveScans_args();
+      public getCompactionQueueInfo_args getEmptyArgsInstance() {
+        return new getCompactionQueueInfo_args();
       }
 
-      public org.apache.thrift.async.AsyncMethodCallback<java.util.List<ActiveScan>> getResultHandler(final org.apache.thrift.server.AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) {
+      public org.apache.thrift.async.AsyncMethodCallback<java.util.List<TCompactionQueueSummary>> 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<ActiveScan>>() { 
-          public void onComplete(java.util.List<ActiveScan> o) {
-            getActiveScans_result result = new getActiveScans_result();
+        return new org.apache.thrift.async.AsyncMethodCallback<java.util.List<TCompactionQueueSummary>>() { 
+          public void onComplete(java.util.List<TCompactionQueueSummary> o) {
+            getCompactionQueueInfo_result result = new getCompactionQueueInfo_result();
             result.success = o;
             try {
               fcall.sendResponse(fb, result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
@@ -5626,7 +4867,7 @@ public class TabletClientService {
           public void onError(java.lang.Exception e) {
             byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
             org.apache.thrift.TSerializable msg;
-            getActiveScans_result result = new getActiveScans_result();
+            getCompactionQueueInfo_result result = new getCompactionQueueInfo_result();
             if (e instanceof org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException) {
               result.sec = (org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException) e;
               result.setSecIsSet(true);
@@ -5658,25 +4899,25 @@ public class TabletClientService {
         return false;
       }
 
-      public void start(I iface, getActiveScans_args args, org.apache.thrift.async.AsyncMethodCallback<java.util.List<ActiveScan>> resultHandler) throws org.apache.thrift.TException {
-        iface.getActiveScans(args.tinfo, args.credentials,resultHandler);
+      public void start(I iface, getCompactionQueueInfo_args args, org.apache.thrift.async.AsyncMethodCallback<java.util.List<TCompactionQueueSummary>> resultHandler) throws org.apache.thrift.TException {
+        iface.getCompactionQueueInfo(args.tinfo, args.credentials,resultHandler);
       }
     }
 
-    public static class getActiveCompactions<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, getActiveCompactions_args, java.util.List<ActiveCompaction>> {
-      public getActiveCompactions() {
-        super("getActiveCompactions");
+    public static class reserveCompactionJob<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, reserveCompactionJob_args, TExternalCompactionJob> {
+      public reserveCompactionJob() {
+        super("reserveCompactionJob");
       }
 
-      public getActiveCompactions_args getEmptyArgsInstance() {
-        return new getActiveCompactions_args();
+      public reserveCompactionJob_args getEmptyArgsInstance() {
+        return new reserveCompactionJob_args();
       }
 
-      public org.apache.thrift.async.AsyncMethodCallback<java.util.List<ActiveCompaction>> getResultHandler(final org.apache.thrift.server.AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) {
+      public org.apache.thrift.async.AsyncMethodCallback<TExternalCompactionJob> getResultHandler(final org.apache.thrift.server.AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) {
         final org.apache.thrift.AsyncProcessFunction fcall = this;
-        return new org.apache.thrift.async.AsyncMethodCallback<java.util.List<ActiveCompaction>>() { 
-          public void onComplete(java.util.List<ActiveCompaction> o) {
-            getActiveCompactions_result result = new getActiveCompactions_result();
+        return new org.apache.thrift.async.AsyncMethodCallback<TExternalCompactionJob>() { 
+          public void onComplete(TExternalCompactionJob o) {
+            reserveCompactionJob_result result = new reserveCompactionJob_result();
             result.success = o;
             try {
               fcall.sendResponse(fb, result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
@@ -5691,7 +4932,7 @@ public class TabletClientService {
           public void onError(java.lang.Exception e) {
             byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
             org.apache.thrift.TSerializable msg;
-            getActiveCompactions_result result = new getActiveCompactions_result();
+            reserveCompactionJob_result result = new reserveCompactionJob_result();
             if (e instanceof org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException) {
               result.sec = (org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException) e;
               result.setSecIsSet(true);
@@ -5723,18 +4964,18 @@ public class TabletClientService {
         return false;
       }
 
-      public void start(I iface, getActiveCompactions_args args, org.apache.thrift.async.AsyncMethodCallback<java.util.List<ActiveCompaction>> resultHandler) throws org.apache.thrift.TException {
-        iface.getActiveCompactions(args.tinfo, args.credentials,resultHandler);
+      public void start(I iface, reserveCompactionJob_args args, org.apache.thrift.async.AsyncMethodCallback<TExternalCompactionJob> resultHandler) throws org.apache.thrift.TException {
+        iface.reserveCompactionJob(args.tinfo, args.credentials, args.queueName, args.priority, args.compactor, args.externalCompactionId,resultHandler);
       }
     }
 
-    public static class removeLogs<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, removeLogs_args, Void> {
-      public removeLogs() {
-        super("removeLogs");
+    public static class compactionJobFinished<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, compactionJobFinished_args, Void> {
+      public compactionJobFinished() {
+        super("compactionJobFinished");
       }
 
-      public removeLogs_args getEmptyArgsInstance() {
-        return new removeLogs_args();
+      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) {
@@ -5757,10814 +4998,74 @@ public class TabletClientService {
         return true;
       }
 
-      public void start(I iface, removeLogs_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
-        iface.removeLogs(args.tinfo, args.credentials, args.filenames,resultHandler);
+      public void start(I iface, compactionJobFinished_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
+        iface.compactionJobFinished(args.tinfo, args.credentials, args.externalCompactionId, args.extent, args.fileSize, args.entries,resultHandler);
       }
     }
 
-    public static class getActiveLogs<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, getActiveLogs_args, java.util.List<java.lang.String>> {
-      public getActiveLogs() {
-        super("getActiveLogs");
+    public static class compactionJobFailed<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, compactionJobFailed_args, Void> {
+      public compactionJobFailed() {
+        super("compactionJobFailed");
       }
 
-      public getActiveLogs_args getEmptyArgsInstance() {
-        return new getActiveLogs_args();
+      public compactionJobFailed_args getEmptyArgsInstance() {
+        return new compactionJobFailed_args();
       }
 
-      public org.apache.thrift.async.AsyncMethodCallback<java.util.List<java.lang.String>> 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<java.util.List<java.lang.String>>() { 
-          public void onComplete(java.util.List<java.lang.String> o) {
-            getActiveLogs_result result = new getActiveLogs_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);
-            }
+        return new org.apache.thrift.async.AsyncMethodCallback<Void>() { 
+          public void onComplete(Void o) {
           }
           public void onError(java.lang.Exception e) {
-            byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
-            org.apache.thrift.TSerializable msg;
-            getActiveLogs_result result = new getActiveLogs_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();
+              _LOGGER.error("Exception inside oneway handler", e);
             }
           }
         };
       }
 
       protected boolean isOneway() {
-        return false;
+        return true;
       }
 
-      public void start(I iface, getActiveLogs_args args, org.apache.thrift.async.AsyncMethodCallback<java.util.List<java.lang.String>> resultHandler) throws org.apache.thrift.TException {
-        iface.getActiveLogs(args.tinfo, args.credentials,resultHandler);
+      public void start(I iface, compactionJobFailed_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
+        iface.compactionJobFailed(args.tinfo, args.credentials, args.externalCompactionId, args.extent,resultHandler);
       }
     }
 
-    public static class startGetSummaries<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, startGetSummaries_args, org.apache.accumulo.core.dataImpl.thrift.TSummaries> {
-      public startGetSummaries() {
-        super("startGetSummaries");
-      }
+  }
 
-      public startGetSummaries_args getEmptyArgsInstance() {
-        return new startGetSummaries_args();
-      }
+  public static class startUpdate_args implements org.apache.thrift.TBase<startUpdate_args, startUpdate_args._Fields>, java.io.Serializable, Cloneable, Comparable<startUpdate_args>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("startUpdate_args");
 
-      public org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.dataImpl.thrift.TSummaries> getResultHandler(final org.apache.thrift.server.AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) {
-        final org.apache.thrift.AsyncProcessFunction fcall = this;
-        return new org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.dataImpl.thrift.TSummaries>() { 
-          public void onComplete(org.apache.accumulo.core.dataImpl.thrift.TSummaries o) {
-            startGetSummaries_result result = new startGetSummaries_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;
-            startGetSummaries_result result = new startGetSummaries_result();
-            if (e instanceof org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException) {
-              result.sec = (org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException) e;
-              result.setSecIsSet(true);
-              msg = result;
-            } else if (e instanceof org.apache.accumulo.core.clientImpl.thrift.ThriftTableOperationException) {
-              result.tope = (org.apache.accumulo.core.clientImpl.thrift.ThriftTableOperationException) e;
-              result.setTopeIsSet(true);
-              msg = result;
-            } else if (e instanceof org.apache.thrift.transport.TTransportException) {
-              _LOGGER.error("TTransportException inside handler", e);
-              fb.close();
-              return;
-            } else if (e instanceof org.apache.thrift.TApplicationException) {
-              _LOGGER.error("TApplicationException inside handler", e);
-              msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
-              msg = (org.apache.thrift.TApplicationException)e;
-            } else {
-              _LOGGER.error("Exception inside handler", e);
-              msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
-              msg = new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage());
-            }
-            try {
-              fcall.sendResponse(fb,msg,msgType,seqid);
-            } catch (java.lang.Exception ex) {
-              _LOGGER.error("Exception writing to internal frame buffer", ex);
-              fb.close();
-            }
-          }
-        };
-      }
-
-      protected boolean isOneway() {
-        return false;
-      }
-
-      public void start(I iface, startGetSummaries_args args, org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.dataImpl.thrift.TSummaries> resultHandler) throws org.apache.thrift.TException {
-        iface.startGetSummaries(args.tinfo, args.credentials, args.request,resultHandler);
-      }
-    }
-
-    public static class startGetSummariesForPartition<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, startGetSummariesForPartition_args, org.apache.accumulo.core.dataImpl.thrift.TSummaries> {
-      public startGetSummariesForPartition() {
-        super("startGetSummariesForPartition");
-      }
-
-      public startGetSummariesForPartition_args getEmptyArgsInstance() {
-        return new startGetSummariesForPartition_args();
-      }
-
-      public org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.dataImpl.thrift.TSummaries> getResultHandler(final org.apache.thrift.server.AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) {
-        final org.apache.thrift.AsyncProcessFunction fcall = this;
-        return new org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.dataImpl.thrift.TSummaries>() { 
-          public void onComplete(org.apache.accumulo.core.dataImpl.thrift.TSummaries o) {
-            startGetSummariesForPartition_result result = new startGetSummariesForPartition_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;
-            startGetSummariesForPartition_result result = new startGetSummariesForPartition_result();
-            if (e instanceof org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException) {
-              result.sec = (org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException) e;
-              result.setSecIsSet(true);
-              msg = result;
-            } else if (e instanceof org.apache.thrift.transport.TTransportException) {
-              _LOGGER.error("TTransportException inside handler", e);
-              fb.close();
-              return;
-            } else if (e instanceof org.apache.thrift.TApplicationException) {
-              _LOGGER.error("TApplicationException inside handler", e);
-              msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
-              msg = (org.apache.thrift.TApplicationException)e;
-            } else {
-              _LOGGER.error("Exception inside handler", e);
-              msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
-              msg = new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage());
-            }
-            try {
-              fcall.sendResponse(fb,msg,msgType,seqid);
-            } catch (java.lang.Exception ex) {
-              _LOGGER.error("Exception writing to internal frame buffer", ex);
-              fb.close();
-            }
-          }
-        };
-      }
-
-      protected boolean isOneway() {
-        return false;
-      }
-
-      public void start(I iface, startGetSummariesForPartition_args args, org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.dataImpl.thrift.TSummaries> resultHandler) throws org.apache.thrift.TException {
-        iface.startGetSummariesForPartition(args.tinfo, args.credentials, args.request, args.modulus, args.remainder,resultHandler);
-      }
-    }
-
-    public static class startGetSummariesFromFiles<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, startGetSummariesFromFiles_args, org.apache.accumulo.core.dataImpl.thrift.TSummaries> {
-      public startGetSummariesFromFiles() {
-        super("startGetSummariesFromFiles");
-      }
-
-      public startGetSummariesFromFiles_args getEmptyArgsInstance() {
-        return new startGetSummariesFromFiles_args();
-      }
-
-      public org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.dataImpl.thrift.TSummaries> getResultHandler(final org.apache.thrift.server.AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) {
-        final org.apache.thrift.AsyncProcessFunction fcall = this;
-        return new org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.dataImpl.thrift.TSummaries>() { 
-          public void onComplete(org.apache.accumulo.core.dataImpl.thrift.TSummaries o) {
-            startGetSummariesFromFiles_result result = new startGetSummariesFromFiles_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;
-            startGetSummariesFromFiles_result result = new startGetSummariesFromFiles_result();
-            if (e instanceof org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException) {
-              result.sec = (org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException) e;
-              result.setSecIsSet(true);
-              msg = result;
-            } else if (e instanceof org.apache.thrift.transport.TTransportException) {
-              _LOGGER.error("TTransportException inside handler", e);
-              fb.close();
-              return;
-            } else if (e instanceof org.apache.thrift.TApplicationException) {
-              _LOGGER.error("TApplicationException inside handler", e);
-              msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
-              msg = (org.apache.thrift.TApplicationException)e;
-            } else {
-              _LOGGER.error("Exception inside handler", e);
-              msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
-              msg = new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage());
-            }
-            try {
-              fcall.sendResponse(fb,msg,msgType,seqid);
-            } catch (java.lang.Exception ex) {
-              _LOGGER.error("Exception writing to internal frame buffer", ex);
-              fb.close();
-            }
-          }
-        };
-      }
-
-      protected boolean isOneway() {
-        return false;
-      }
-
-      public void start(I iface, startGetSummariesFromFiles_args args, org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.dataImpl.thrift.TSummaries> resultHandler) throws org.apache.thrift.TException {
-        iface.startGetSummariesFromFiles(args.tinfo, args.credentials, args.request, args.files,resultHandler);
-      }
-    }
-
-    public static class contiuneGetSummaries<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, contiuneGetSummaries_args, org.apache.accumulo.core.dataImpl.thrift.TSummaries> {
-      public contiuneGetSummaries() {
-        super("contiuneGetSummaries");
-      }
-
-      public contiuneGetSummaries_args getEmptyArgsInstance() {
-        return new contiuneGetSummaries_args();
-      }
-
-      public org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.dataImpl.thrift.TSummaries> getResultHandler(final org.apache.thrift.server.AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) {
-        final org.apache.thrift.AsyncProcessFunction fcall = this;
-        return new org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.dataImpl.thrift.TSummaries>() { 
-          public void onComplete(org.apache.accumulo.core.dataImpl.thrift.TSummaries o) {
-            contiuneGetSummaries_result result = new contiuneGetSummaries_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;
-            contiuneGetSummaries_result result = new contiuneGetSummaries_result();
-            if (e instanceof NoSuchScanIDException) {
-              result.nssi = (NoSuchScanIDException) e;
-              result.setNssiIsSet(true);
-              msg = result;
-            } else if (e instanceof org.apache.thrift.transport.TTransportException) {
-              _LOGGER.error("TTransportException inside handler", e);
-              fb.close();
-              return;
-            } else if (e instanceof org.apache.thrift.TApplicationException) {
-              _LOGGER.error("TApplicationException inside handler", e);
-              msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
-              msg = (org.apache.thrift.TApplicationException)e;
-            } else {
-              _LOGGER.error("Exception inside handler", e);
-              msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
-              msg = new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage());
-            }
-            try {
-              fcall.sendResponse(fb,msg,msgType,seqid);
-            } catch (java.lang.Exception ex) {
-              _LOGGER.error("Exception writing to internal frame buffer", ex);
-              fb.close();
-            }
-          }
-        };
-      }
-
-      protected boolean isOneway() {
-        return false;
-      }
-
-      public void start(I iface, contiuneGetSummaries_args args, org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.dataImpl.thrift.TSummaries> resultHandler) throws org.apache.thrift.TException {
-        iface.contiuneGetSummaries(args.tinfo, args.sessionId,resultHandler);
-      }
-    }
-
-    public static class getCompactionQueueInfo<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, getCompactionQueueInfo_args, java.util.List<TCompactionQueueSummary>> {
-      public getCompactionQueueInfo() {
-        super("getCompactionQueueInfo");
-      }
-
-      public getCompactionQueueInfo_args getEmptyArgsInstance() {
-        return new getCompactionQueueInfo_args();
-      }
-
-      public org.apache.thrift.async.AsyncMethodCallback<java.util.List<TCompactionQueueSummary>> 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<TCompactionQueueSummary>>() { 
-          public void onComplete(java.util.List<TCompactionQueueSummary> 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.accumulo.core.clientImpl.thrift.ThriftSecurityException) {
-              result.sec = (org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException) e;
-              result.setSecIsSet(true);
-              msg = result;
-            } else if (e instanceof org.apache.thrift.transport.TTransportException) {
-              _LOGGER.error("TTransportException inside handler", e);
-              fb.close();
-              return;
-            } else if (e instanceof org.apache.thrift.TApplicationException) {
-              _LOGGER.error("TApplicationException inside handler", e);
-              msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
-              msg = (org.apache.thrift.TApplicationException)e;
-            } else {
-              _LOGGER.error("Exception inside handler", e);
-              msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
-              msg = new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage());
-            }
-            try {
-              fcall.sendResponse(fb,msg,msgType,seqid);
-            } catch (java.lang.Exception ex) {
-              _LOGGER.error("Exception writing to internal frame buffer", ex);
-              fb.close();
-            }
-          }
-        };
-      }
-
-      protected boolean isOneway() {
-        return false;
-      }
-
-      public void start(I iface, getCompactionQueueInfo_args args, org.apache.thrift.async.AsyncMethodCallback<java.util.List<TCompactionQueueSummary>> resultHandler) throws org.apache.thrift.TException {
-        iface.getCompactionQueueInfo(args.tinfo, args.credentials,resultHandler);
-      }
-    }
-
-    public static class reserveCompactionJob<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, reserveCompactionJob_args, TExternalCompactionJob> {
-      public reserveCompactionJob() {
-        super("reserveCompactionJob");
-      }
-
-      public reserveCompactionJob_args getEmptyArgsInstance() {
-        return new reserveCompactionJob_args();
-      }
-
-      public org.apache.thrift.async.AsyncMethodCallback<TExternalCompactionJob> getResultHandler(final org.apache.thrift.server.AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) {
-        final org.apache.thrift.AsyncProcessFunction fcall = this;
-        return new org.apache.thrift.async.AsyncMethodCallback<TExternalCompactionJob>() { 
-          public void onComplete(TExternalCompactionJob 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.accumulo.core.clientImpl.thrift.ThriftSecurityException) {
-              result.sec = (org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException) e;
-              result.setSecIsSet(true);
-              msg = result;
-            } else if (e instanceof org.apache.thrift.transport.TTransportException) {
-              _LOGGER.error("TTransportException inside handler", e);
-              fb.close();
-              return;
-            } else if (e instanceof org.apache.thrift.TApplicationException) {
-              _LOGGER.error("TApplicationException inside handler", e);
-              msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
-              msg = (org.apache.thrift.TApplicationException)e;
-            } else {
-              _LOGGER.error("Exception inside handler", e);
-              msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
-              msg = new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage());
-            }
-            try {
-              fcall.sendResponse(fb,msg,msgType,seqid);
-            } catch (java.lang.Exception ex) {
-              _LOGGER.error("Exception writing to internal frame buffer", ex);
-              fb.close();
-            }
-          }
-        };
-      }
-
-      protected boolean isOneway() {
-        return false;
-      }
-
-      public void start(I iface, reserveCompactionJob_args args, org.apache.thrift.async.AsyncMethodCallback<TExternalCompactionJob> resultHandler) throws org.apache.thrift.TException {
-        iface.reserveCompactionJob(args.tinfo, args.credentials, args.queueName, args.priority, args.compactor, args.externalCompactionId,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) {
-          }
-          public void onError(java.lang.Exception e) {
-            if (e instanceof org.apache.thrift.transport.TTransportException) {
-              _LOGGER.error("TTransportException inside handler", e);
-              fb.close();
-            } else {
-              _LOGGER.error("Exception inside oneway handler", e);
-            }
-          }
-        };
-      }
-
-      protected boolean isOneway() {
-        return true;
-      }
-
-      public void start(I iface, compactionJobFinished_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
-        iface.compactionJobFinished(args.tinfo, args.credentials, args.externalCompactionId, args.extent, args.fileSize, args.entries,resultHandler);
-      }
-    }
-
-    public static class compactionJobFailed<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, compactionJobFailed_args, Void> {
-      public compactionJobFailed() {
-        super("compactionJobFailed");
-      }
-
-      public compactionJobFailed_args getEmptyArgsInstance() {
-        return new compactionJobFailed_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) {
-          }
-          public void onError(java.lang.Exception e) {
-            if (e instanceof org.apache.thrift.transport.TTransportException) {
-              _LOGGER.error("TTransportException inside handler", e);
-              fb.close();
-            } else {
-              _LOGGER.error("Exception inside oneway handler", e);
-            }
-          }
-        };
-      }
-
-      protected boolean isOneway() {
-        return true;
-      }
-
-      public void start(I iface, compactionJobFailed_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
-        iface.compactionJobFailed(args.tinfo, args.credentials, args.externalCompactionId, args.extent,resultHandler);
-      }
-    }
-
-  }
-
-  public static class startScan_args implements org.apache.thrift.TBase<startScan_args, startScan_args._Fields>, java.io.Serializable, Cloneable, Comparable<startScan_args>   {
-    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("startScan_args");
-
-    private static final org.apache.thrift.protocol.TField TINFO_FIELD_DESC = new org.apache.thrift.protocol.TField("tinfo", org.apache.thrift.protocol.TType.STRUCT, (short)11);
-    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)1);
-    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)2);
-    private static final org.apache.thrift.protocol.TField RANGE_FIELD_DESC = new org.apache.thrift.protocol.TField("range", org.apache.thrift.protocol.TType.STRUCT, (short)3);
-    private static final org.apache.thrift.protocol.TField COLUMNS_FIELD_DESC = new org.apache.thrift.protocol.TField("columns", org.apache.thrift.protocol.TType.LIST, (short)4);
-    private static final org.apache.thrift.protocol.TField BATCH_SIZE_FIELD_DESC = new org.apache.thrift.protocol.TField("batchSize", org.apache.thrift.protocol.TType.I32, (short)5);
-    private static final org.apache.thrift.protocol.TField SSI_LIST_FIELD_DESC = new org.apache.thrift.protocol.TField("ssiList", org.apache.thrift.protocol.TType.LIST, (short)6);
-    private static final org.apache.thrift.protocol.TField SSIO_FIELD_DESC = new org.apache.thrift.protocol.TField("ssio", org.apache.thrift.protocol.TType.MAP, (short)7);
-    private static final org.apache.thrift.protocol.TField AUTHORIZATIONS_FIELD_DESC = new org.apache.thrift.protocol.TField("authorizations", org.apache.thrift.protocol.TType.LIST, (short)8);
-    private static final org.apache.thrift.protocol.TField WAIT_FOR_WRITES_FIELD_DESC = new org.apache.thrift.protocol.TField("waitForWrites", org.apache.thrift.protocol.TType.BOOL, (short)9);
-    private static final org.apache.thrift.protocol.TField ISOLATED_FIELD_DESC = new org.apache.thrift.protocol.TField("isolated", org.apache.thrift.protocol.TType.BOOL, (short)10);
-    private static final org.apache.thrift.protocol.TField READAHEAD_THRESHOLD_FIELD_DESC = new org.apache.thrift.protocol.TField("readaheadThreshold", org.apache.thrift.protocol.TType.I64, (short)12);
-    private static final org.apache.thrift.protocol.TField SAMPLER_CONFIG_FIELD_DESC = new org.apache.thrift.protocol.TField("samplerConfig", org.apache.thrift.protocol.TType.STRUCT, (short)13);
-    private static final org.apache.thrift.protocol.TField BATCH_TIME_OUT_FIELD_DESC = new org.apache.thrift.protocol.TField("batchTimeOut", org.apache.thrift.protocol.TType.I64, (short)14);
-    private static final org.apache.thrift.protocol.TField CLASS_LOADER_CONTEXT_FIELD_DESC = new org.apache.thrift.protocol.TField("classLoaderContext", org.apache.thrift.protocol.TType.STRING, (short)15);
-    private static final org.apache.thrift.protocol.TField EXECUTION_HINTS_FIELD_DESC = new org.apache.thrift.protocol.TField("executionHints", org.apache.thrift.protocol.TType.MAP, (short)16);
-
-    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new startScan_argsStandardSchemeFactory();
-    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new startScan_argsTupleSchemeFactory();
-
-    public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo; // required
-    public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials; // required
-    public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent; // required
-    public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TRange range; // required
-    public @org.apache.thrift.annotation.Nullable java.util.List<org.apache.accumulo.core.dataImpl.thrift.TColumn> columns; // required
-    public int batchSize; // required
-    public @org.apache.thrift.annotation.Nullable java.util.List<org.apache.accumulo.core.dataImpl.thrift.IterInfo> ssiList; // required
-    public @org.apache.thrift.annotation.Nullable java.util.Map<java.lang.String,java.util.Map<java.lang.String,java.lang.String>> ssio; // required
-    public @org.apache.thrift.annotation.Nullable java.util.List<java.nio.ByteBuffer> authorizations; // required
-    public boolean waitForWrites; // required
-    public boolean isolated; // required
-    public long readaheadThreshold; // required
-    public @org.apache.thrift.annotation.Nullable TSamplerConfiguration samplerConfig; // required
-    public long batchTimeOut; // required
-    public @org.apache.thrift.annotation.Nullable java.lang.String classLoaderContext; // required
-    public @org.apache.thrift.annotation.Nullable java.util.Map<java.lang.String,java.lang.String> executionHints; // required
-
-    /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
-    public enum _Fields implements org.apache.thrift.TFieldIdEnum {
-      TINFO((short)11, "tinfo"),
-      CREDENTIALS((short)1, "credentials"),
-      EXTENT((short)2, "extent"),
-      RANGE((short)3, "range"),
-      COLUMNS((short)4, "columns"),
-      BATCH_SIZE((short)5, "batchSize"),
-      SSI_LIST((short)6, "ssiList"),
-      SSIO((short)7, "ssio"),
-      AUTHORIZATIONS((short)8, "authorizations"),
-      WAIT_FOR_WRITES((short)9, "waitForWrites"),
-      ISOLATED((short)10, "isolated"),
-      READAHEAD_THRESHOLD((short)12, "readaheadThreshold"),
-      SAMPLER_CONFIG((short)13, "samplerConfig"),
-      BATCH_TIME_OUT((short)14, "batchTimeOut"),
-      CLASS_LOADER_CONTEXT((short)15, "classLoaderContext"),
-      EXECUTION_HINTS((short)16, "executionHints");
-
-      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 11: // TINFO
-            return TINFO;
-          case 1: // CREDENTIALS
-            return CREDENTIALS;
-          case 2: // EXTENT
-            return EXTENT;
-          case 3: // RANGE
-            return RANGE;
-          case 4: // COLUMNS
-            return COLUMNS;
-          case 5: // BATCH_SIZE
-            return BATCH_SIZE;
-          case 6: // SSI_LIST
-            return SSI_LIST;
-          case 7: // SSIO
-            return SSIO;
-          case 8: // AUTHORIZATIONS
-            return AUTHORIZATIONS;
-          case 9: // WAIT_FOR_WRITES
-            return WAIT_FOR_WRITES;
-          case 10: // ISOLATED
-            return ISOLATED;
-          case 12: // READAHEAD_THRESHOLD
-            return READAHEAD_THRESHOLD;
-          case 13: // SAMPLER_CONFIG
-            return SAMPLER_CONFIG;
-          case 14: // BATCH_TIME_OUT
-            return BATCH_TIME_OUT;
-          case 15: // CLASS_LOADER_CONTEXT
-            return CLASS_LOADER_CONTEXT;
-          case 16: // EXECUTION_HINTS
-            return EXECUTION_HINTS;
-          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 __BATCHSIZE_ISSET_ID = 0;
-    private static final int __WAITFORWRITES_ISSET_ID = 1;
-    private static final int __ISOLATED_ISSET_ID = 2;
-    private static final int __READAHEADTHRESHOLD_ISSET_ID = 3;
-    private static final int __BATCHTIMEOUT_ISSET_ID = 4;
-    private byte __isset_bitfield = 0;
-    public static final java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
-    static {
-      java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
-      tmpMap.put(_Fields.TINFO, new org.apache.thrift.meta_data.FieldMetaData("tinfo", org.apache.thrift.TFieldRequirementType.DEFAULT, 
-          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.trace.thrift.TInfo.class)));
-      tmpMap.put(_Fields.CREDENTIALS, new org.apache.thrift.meta_data.FieldMetaData("credentials", org.apache.thrift.TFieldRequirementType.DEFAULT, 
-          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.securityImpl.thrift.TCredentials.class)));
-      tmpMap.put(_Fields.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.RANGE, new org.apache.thrift.meta_data.FieldMetaData("range", org.apache.thrift.TFieldRequirementType.DEFAULT, 
-          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.dataImpl.thrift.TRange.class)));
-      tmpMap.put(_Fields.COLUMNS, new org.apache.thrift.meta_data.FieldMetaData("columns", 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, org.apache.accumulo.core.dataImpl.thrift.TColumn.class))));
-      tmpMap.put(_Fields.BATCH_SIZE, new org.apache.thrift.meta_data.FieldMetaData("batchSize", org.apache.thrift.TFieldRequirementType.DEFAULT, 
-          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32)));
-      tmpMap.put(_Fields.SSI_LIST, new org.apache.thrift.meta_data.FieldMetaData("ssiList", 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, org.apache.accumulo.core.dataImpl.thrift.IterInfo.class))));
-      tmpMap.put(_Fields.SSIO, new org.apache.thrift.meta_data.FieldMetaData("ssio", org.apache.thrift.TFieldRequirementType.DEFAULT, 
-          new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP, 
-              new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING), 
-              new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP, 
-                  new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING), 
-                  new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)))));
-      tmpMap.put(_Fields.AUTHORIZATIONS, new org.apache.thrift.meta_data.FieldMetaData("authorizations", 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              , true))));
-      tmpMap.put(_Fields.WAIT_FOR_WRITES, new org.apache.thrift.meta_data.FieldMetaData("waitForWrites", org.apache.thrift.TFieldRequirementType.DEFAULT, 
-          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.BOOL)));
-      tmpMap.put(_Fields.ISOLATED, new org.apache.thrift.meta_data.FieldMetaData("isolated", org.apache.thrift.TFieldRequirementType.DEFAULT, 
-          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.BOOL)));
-      tmpMap.put(_Fields.READAHEAD_THRESHOLD, new org.apache.thrift.meta_data.FieldMetaData("readaheadThreshold", org.apache.thrift.TFieldRequirementType.DEFAULT, 
-          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64)));
-      tmpMap.put(_Fields.SAMPLER_CONFIG, new org.apache.thrift.meta_data.FieldMetaData("samplerConfig", org.apache.thrift.TFieldRequirementType.DEFAULT, 
-          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TSamplerConfiguration.class)));
-      tmpMap.put(_Fields.BATCH_TIME_OUT, new org.apache.thrift.meta_data.FieldMetaData("batchTimeOut", org.apache.thrift.TFieldRequirementType.DEFAULT, 
-          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64)));
-      tmpMap.put(_Fields.CLASS_LOADER_CONTEXT, new org.apache.thrift.meta_data.FieldMetaData("classLoaderContext", org.apache.thrift.TFieldRequirementType.DEFAULT, 
-          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
-      tmpMap.put(_Fields.EXECUTION_HINTS, new org.apache.thrift.meta_data.FieldMetaData("executionHints", org.apache.thrift.TFieldRequirementType.DEFAULT, 
-          new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP, 
-              new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING), 
-              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(startScan_args.class, metaDataMap);
-    }
-
-    public startScan_args() {
-    }
-
-    public startScan_args(
-      org.apache.accumulo.core.trace.thrift.TInfo tinfo,
-      org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials,
-      org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent,
-      org.apache.accumulo.core.dataImpl.thrift.TRange range,
-      java.util.List<org.apache.accumulo.core.dataImpl.thrift.TColumn> columns,
-      int batchSize,
-      java.util.List<org.apache.accumulo.core.dataImpl.thrift.IterInfo> ssiList,
-      java.util.Map<java.lang.String,java.util.Map<java.lang.String,java.lang.String>> ssio,
-      java.util.List<java.nio.ByteBuffer> authorizations,
-      boolean waitForWrites,
-      boolean isolated,
-      long readaheadThreshold,
-      TSamplerConfiguration samplerConfig,
-      long batchTimeOut,
-      java.lang.String classLoaderContext,
-      java.util.Map<java.lang.String,java.lang.String> executionHints)
-    {
-      this();
-      this.tinfo = tinfo;
-      this.credentials = credentials;
-      this.extent = extent;
-      this.range = range;
-      this.columns = columns;
-      this.batchSize = batchSize;
-      setBatchSizeIsSet(true);
-      this.ssiList = ssiList;
-      this.ssio = ssio;
-      this.authorizations = authorizations;
-      this.waitForWrites = waitForWrites;
-      setWaitForWritesIsSet(true);
-      this.isolated = isolated;
-      setIsolatedIsSet(true);
-      this.readaheadThreshold = readaheadThreshold;
-      setReadaheadThresholdIsSet(true);
-      this.samplerConfig = samplerConfig;
-      this.batchTimeOut = batchTimeOut;
-      setBatchTimeOutIsSet(true);
-      this.classLoaderContext = classLoaderContext;
-      this.executionHints = executionHints;
-    }
-
-    /**
-     * Performs a deep copy on <i>other</i>.
-     */
-    public startScan_args(startScan_args other) {
-      __isset_bitfield = other.__isset_bitfield;
-      if (other.isSetTinfo()) {
-        this.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo(other.tinfo);
-      }
-      if (other.isSetCredentials()) {
-        this.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(other.credentials);
-      }
-      if (other.isSetExtent()) {
-        this.extent = new org.apache.accumulo.core.dataImpl.thrift.TKeyExtent(other.extent);
-      }
-      if (other.isSetRange()) {
-        this.range = new org.apache.accumulo.core.dataImpl.thrift.TRange(other.range);
-      }
-      if (other.isSetColumns()) {
-        java.util.List<org.apache.accumulo.core.dataImpl.thrift.TColumn> __this__columns = new java.util.ArrayList<org.apache.accumulo.core.dataImpl.thrift.TColumn>(other.columns.size());
-        for (org.apache.accumulo.core.dataImpl.thrift.TColumn other_element : other.columns) {
-          __this__columns.add(new org.apache.accumulo.core.dataImpl.thrift.TColumn(other_element));
-        }
-        this.columns = __this__columns;
-      }
-      this.batchSize = other.batchSize;
-      if (other.isSetSsiList()) {
-        java.util.List<org.apache.accumulo.core.dataImpl.thrift.IterInfo> __this__ssiList = new java.util.ArrayList<org.apache.accumulo.core.dataImpl.thrift.IterInfo>(other.ssiList.size());
-        for (org.apache.accumulo.core.dataImpl.thrift.IterInfo other_element : other.ssiList) {
-          __this__ssiList.add(new org.apache.accumulo.core.dataImpl.thrift.IterInfo(other_element));
-        }
-        this.ssiList = __this__ssiList;
-      }
-      if (other.isSetSsio()) {
-        java.util.Map<java.lang.String,java.util.Map<java.lang.String,java.lang.String>> __this__ssio = new java.util.HashMap<java.lang.String,java.util.Map<java.lang.String,java.lang.String>>(other.ssio.size());
-        for (java.util.Map.Entry<java.lang.String, java.util.Map<java.lang.String,java.lang.String>> other_element : other.ssio.entrySet()) {
-
-          java.lang.String other_element_key = other_element.getKey();
-          java.util.Map<java.lang.String,java.lang.String> other_element_value = other_element.getValue();
-
-          java.lang.String __this__ssio_copy_key = other_element_key;
-
-          java.util.Map<java.lang.String,java.lang.String> __this__ssio_copy_value = new java.util.HashMap<java.lang.String,java.lang.String>(other_element_value);
-
-          __this__ssio.put(__this__ssio_copy_key, __this__ssio_copy_value);
-        }
-        this.ssio = __this__ssio;
-      }
-      if (other.isSetAuthorizations()) {
-        java.util.List<java.nio.ByteBuffer> __this__authorizations = new java.util.ArrayList<java.nio.ByteBuffer>(other.authorizations);
-        this.authorizations = __this__authorizations;
-      }
-      this.waitForWrites = other.waitForWrites;
-      this.isolated = other.isolated;
-      this.readaheadThreshold = other.readaheadThreshold;
-      if (other.isSetSamplerConfig()) {
-        this.samplerConfig = new TSamplerConfiguration(other.samplerConfig);
-      }
-      this.batchTimeOut = other.batchTimeOut;
-      if (other.isSetClassLoaderContext()) {
-        this.classLoaderContext = other.classLoaderContext;
-      }
-      if (other.isSetExecutionHints()) {
-        java.util.Map<java.lang.String,java.lang.String> __this__executionHints = new java.util.HashMap<java.lang.String,java.lang.String>(other.executionHints);
-        this.executionHints = __this__executionHints;
-      }
-    }
-
-    public startScan_args deepCopy() {
-      return new startScan_args(this);
-    }
-
-    @Override
-    public void clear() {
-      this.tinfo = null;
-      this.credentials = null;
-      this.extent = null;
-      this.range = null;
-      this.columns = null;
-      setBatchSizeIsSet(false);
-      this.batchSize = 0;
-      this.ssiList = null;
-      this.ssio = null;
-      this.authorizations = null;
-      setWaitForWritesIsSet(false);
-      this.waitForWrites = false;
-      setIsolatedIsSet(false);
-      this.isolated = false;
-      setReadaheadThresholdIsSet(false);
-      this.readaheadThreshold = 0;
-      this.samplerConfig = null;
-      setBatchTimeOutIsSet(false);
-      this.batchTimeOut = 0;
-      this.classLoaderContext = null;
-      this.executionHints = null;
-    }
-
-    @org.apache.thrift.annotation.Nullable
-    public org.apache.accumulo.core.trace.thrift.TInfo getTinfo() {
-      return this.tinfo;
-    }
-
-    public startScan_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo) {
-      this.tinfo = tinfo;
-      return this;
-    }
-
-    public void unsetTinfo() {
-      this.tinfo = null;
-    }
-
-    /** Returns true if field tinfo is set (has been assigned a value) and false otherwise */
-    public boolean isSetTinfo() {
-      return this.tinfo != null;
-    }
-
-    public void setTinfoIsSet(boolean value) {
-      if (!value) {
-        this.tinfo = null;
-      }
-    }
-
-    @org.apache.thrift.annotation.Nullable
-    public org.apache.accumulo.core.securityImpl.thrift.TCredentials getCredentials() {
-      return this.credentials;
-    }
-
-    public startScan_args setCredentials(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) {
-      this.credentials = credentials;
-      return this;
-    }
-
-    public void unsetCredentials() {
-      this.credentials = null;
-    }
-
-    /** Returns true if field credentials is set (has been assigned a value) and false otherwise */
-    public boolean isSetCredentials() {
-      return this.credentials != null;
-    }
-
-    public void setCredentialsIsSet(boolean value) {
-      if (!value) {
-        this.credentials = null;
-      }
-    }
-
-    @org.apache.thrift.annotation.Nullable
-    public org.apache.accumulo.core.dataImpl.thrift.TKeyExtent getExtent() {
-      return this.extent;
-    }
-
-    public startScan_args setExtent(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent) {
-      this.extent = extent;
-      return this;
-    }
-
-    public void unsetExtent() {
-      this.extent = null;
-    }
-
-    /** Returns true if field extent is set (has been assigned a value) and false otherwise */
-    public boolean isSetExtent() {
-      return this.extent != null;
-    }
-
-    public void setExtentIsSet(boolean value) {
-      if (!value) {
-        this.extent = null;
-      }
-    }
-
-    @org.apache.thrift.annotation.Nullable
-    public org.apache.accumulo.core.dataImpl.thrift.TRange getRange() {
-      return this.range;
-    }
-
-    public startScan_args setRange(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TRange range) {
-      this.range = range;
-      return this;
-    }
-
-    public void unsetRange() {
-      this.range = null;
-    }
-
-    /** Returns true if field range is set (has been assigned a value) and false otherwise */
-    public boolean isSetRange() {
-      return this.range != null;
-    }
-
-    public void setRangeIsSet(boolean value) {
-      if (!value) {
-        this.range = null;
-      }
-    }
-
-    public int getColumnsSize() {
-      return (this.columns == null) ? 0 : this.columns.size();
-    }
-
-    @org.apache.thrift.annotation.Nullable
-    public java.util.Iterator<org.apache.accumulo.core.dataImpl.thrift.TColumn> getColumnsIterator() {
-      return (this.columns == null) ? null : this.columns.iterator();
-    }
-
-    public void addToColumns(org.apache.accumulo.core.dataImpl.thrift.TColumn elem) {
-      if (this.columns == null) {
-        this.columns = new java.util.ArrayList<org.apache.accumulo.core.dataImpl.thrift.TColumn>();
-      }
-      this.columns.add(elem);
-    }
-
-    @org.apache.thrift.annotation.Nullable
-    public java.util.List<org.apache.accumulo.core.dataImpl.thrift.TColumn> getColumns() {
-      return this.columns;
-    }
-
-    public startScan_args setColumns(@org.apache.thrift.annotation.Nullable java.util.List<org.apache.accumulo.core.dataImpl.thrift.TColumn> columns) {
-      this.columns = columns;
-      return this;
-    }
-
-    public void unsetColumns() {
-      this.columns = null;
-    }
-
-    /** Returns true if field columns is set (has been assigned a value) and false otherwise */
-    public boolean isSetColumns() {
-      return this.columns != null;
-    }
-
-    public void setColumnsIsSet(boolean value) {
-      if (!value) {
-        this.columns = null;
-      }
-    }
-
-    public int getBatchSize() {
-      return this.batchSize;
-    }
-
-    public startScan_args setBatchSize(int batchSize) {
-      this.batchSize = batchSize;
-      setBatchSizeIsSet(true);
-      return this;
-    }
-
-    public void unsetBatchSize() {
-      __isset_bitfield = org.apache.thrift.EncodingUtils.clearBit(__isset_bitfield, __BATCHSIZE_ISSET_ID);
-    }
-
-    /** Returns true if field batchSize is set (has been assigned a value) and false otherwise */
-    public boolean isSetBatchSize() {
-      return org.apache.thrift.EncodingUtils.testBit(__isset_bitfield, __BATCHSIZE_ISSET_ID);
-    }
-
-    public void setBatchSizeIsSet(boolean value) {
-      __isset_bitfield = org.apache.thrift.EncodingUtils.setBit(__isset_bitfield, __BATCHSIZE_ISSET_ID, value);
-    }
-
-    public int getSsiListSize() {
-      return (this.ssiList == null) ? 0 : this.ssiList.size();
-    }
-
-    @org.apache.thrift.annotation.Nullable
-    public java.util.Iterator<org.apache.accumulo.core.dataImpl.thrift.IterInfo> getSsiListIterator() {
-      return (this.ssiList == null) ? null : this.ssiList.iterator();
-    }
-
-    public void addToSsiList(org.apache.accumulo.core.dataImpl.thrift.IterInfo elem) {
-      if (this.ssiList == null) {
-        this.ssiList = new java.util.ArrayList<org.apache.accumulo.core.dataImpl.thrift.IterInfo>();
-      }
-      this.ssiList.add(elem);
-    }
-
-    @org.apache.thrift.annotation.Nullable
-    public java.util.List<org.apache.accumulo.core.dataImpl.thrift.IterInfo> getSsiList() {
-      return this.ssiList;
-    }
-
-    public startScan_args setSsiList(@org.apache.thrift.annotation.Nullable java.util.List<org.apache.accumulo.core.dataImpl.thrift.IterInfo> ssiList) {
-      this.ssiList = ssiList;
-      return this;
-    }
-
-    public void unsetSsiList() {
-      this.ssiList = null;
-    }
-
-    /** Returns true if field ssiList is set (has been assigned a value) and false otherwise */
-    public boolean isSetSsiList() {
-      return this.ssiList != null;
-    }
-
-    public void setSsiListIsSet(boolean value) {
-      if (!value) {
-        this.ssiList = null;
-      }
-    }
-
-    public int getSsioSize() {
-      return (this.ssio == null) ? 0 : this.ssio.size();
-    }
-
-    public void putToSsio(java.lang.String key, java.util.Map<java.lang.String,java.lang.String> val) {
-      if (this.ssio == null) {
-        this.ssio = new java.util.HashMap<java.lang.String,java.util.Map<java.lang.String,java.lang.String>>();
-      }
-      this.ssio.put(key, val);
-    }
-
-    @org.apache.thrift.annotation.Nullable
-    public java.util.Map<java.lang.String,java.util.Map<java.lang.String,java.lang.String>> getSsio() {
-      return this.ssio;
-    }
-
-    public startScan_args setSsio(@org.apache.thrift.annotation.Nullable java.util.Map<java.lang.String,java.util.Map<java.lang.String,java.lang.String>> ssio) {
-      this.ssio = ssio;
-      return this;
-    }
-
-    public void unsetSsio() {
-      this.ssio = null;
-    }
-
-    /** Returns true if field ssio is set (has been assigned a value) and false otherwise */
-    public boolean isSetSsio() {
-      return this.ssio != null;
-    }
-
-    public void setSsioIsSet(boolean value) {
-      if (!value) {
-        this.ssio = null;
-      }
-    }
-
-    public int getAuthorizationsSize() {
-      return (this.authorizations == null) ? 0 : this.authorizations.size();
-    }
-
-    @org.apache.thrift.annotation.Nullable
-    public java.util.Iterator<java.nio.ByteBuffer> getAuthorizationsIterator() {
-      return (this.authorizations == null) ? null : this.authorizations.iterator();
-    }
-
-    public void addToAuthorizations(java.nio.ByteBuffer elem) {
-      if (this.authorizations == null) {
-        this.authorizations = new java.util.ArrayList<java.nio.ByteBuffer>();
-      }
-      this.authorizations.add(elem);
-    }
-
-    @org.apache.thrift.annotation.Nullable
-    public java.util.List<java.nio.ByteBuffer> getAuthorizations() {
-      return this.authorizations;
-    }
-
-    public startScan_args setAuthorizations(@org.apache.thrift.annotation.Nullable java.util.List<java.nio.ByteBuffer> authorizations) {
-      this.authorizations = authorizations;
-      return this;
-    }
-
-    public void unsetAuthorizations() {
-      this.authorizations = null;
-    }
-
-    /** Returns true if field authorizations is set (has been assigned a value) and false otherwise */
-    public boolean isSetAuthorizations() {
-      return this.authorizations != null;
-    }
-
-    public void setAuthorizationsIsSet(boolean value) {
-      if (!value) {
-        this.authorizations = null;
-      }
-    }
-
-    public boolean isWaitForWrites() {
-      return this.waitForWrites;
-    }
-
-    public startScan_args setWaitForWrites(boolean waitForWrites) {
-      this.waitForWrites = waitForWrites;
-      setWaitForWritesIsSet(true);
-      return this;
-    }
-
-    public void unsetWaitForWrites() {
-      __isset_bitfield = org.apache.thrift.EncodingUtils.clearBit(__isset_bitfield, __WAITFORWRITES_ISSET_ID);
-    }
-
-    /** Returns true if field waitForWrites is set (has been assigned a value) and false otherwise */
-    public boolean isSetWaitForWrites() {
-      return org.apache.thrift.EncodingUtils.testBit(__isset_bitfield, __WAITFORWRITES_ISSET_ID);
-    }
-
-    public void setWaitForWritesIsSet(boolean value) {
-      __isset_bitfield = org.apache.thrift.EncodingUtils.setBit(__isset_bitfield, __WAITFORWRITES_ISSET_ID, value);
-    }
-
-    public boolean isIsolated() {
-      return this.isolated;
-    }
-
-    public startScan_args setIsolated(boolean isolated) {
-      this.isolated = isolated;
-      setIsolatedIsSet(true);
-      return this;
-    }
-
-    public void unsetIsolated() {
-      __isset_bitfield = org.apache.thrift.EncodingUtils.clearBit(__isset_bitfield, __ISOLATED_ISSET_ID);
-    }
-
-    /** Returns true if field isolated is set (has been assigned a value) and false otherwise */
-    public boolean isSetIsolated() {
-      return org.apache.thrift.EncodingUtils.testBit(__isset_bitfield, __ISOLATED_ISSET_ID);
-    }
-
-    public void setIsolatedIsSet(boolean value) {
-      __isset_bitfield = org.apache.thrift.EncodingUtils.setBit(__isset_bitfield, __ISOLATED_ISSET_ID, value);
-    }
-
-    public long getReadaheadThreshold() {
-      return this.readaheadThreshold;
-    }
-
-    public startScan_args setReadaheadThreshold(long readaheadThreshold) {
-      this.readaheadThreshold = readaheadThreshold;
-      setReadaheadThresholdIsSet(true);
-      return this;
-    }
-
-    public void unsetReadaheadThreshold() {
-      __isset_bitfield = org.apache.thrift.EncodingUtils.clearBit(__isset_bitfield, __READAHEADTHRESHOLD_ISSET_ID);
-    }
-
-    /** Returns true if field readaheadThreshold is set (has been assigned a value) and false otherwise */
-    public boolean isSetReadaheadThreshold() {
-      return org.apache.thrift.EncodingUtils.testBit(__isset_bitfield, __READAHEADTHRESHOLD_ISSET_ID);
-    }
-
-    public void setReadaheadThresholdIsSet(boolean value) {
-      __isset_bitfield = org.apache.thrift.EncodingUtils.setBit(__isset_bitfield, __READAHEADTHRESHOLD_ISSET_ID, value);
-    }
-
-    @org.apache.thrift.annotation.Nullable
-    public TSamplerConfiguration getSamplerConfig() {
-      return this.samplerConfig;
-    }
-
-    public startScan_args setSamplerConfig(@org.apache.thrift.annotation.Nullable TSamplerConfiguration samplerConfig) {
-      this.samplerConfig = samplerConfig;
-      return this;
-    }
-
-    public void unsetSamplerConfig() {
-      this.samplerConfig = null;
-    }
-
-    /** Returns true if field samplerConfig is set (has been assigned a value) and false otherwise */
-    public boolean isSetSamplerConfig() {
-      return this.samplerConfig != null;
-    }
-
-    public void setSamplerConfigIsSet(boolean value) {
-      if (!value) {
-        this.samplerConfig = null;
-      }
-    }
-
-    public long getBatchTimeOut() {
-      return this.batchTimeOut;
-    }
-
-    public startScan_args setBatchTimeOut(long batchTimeOut) {
-      this.batchTimeOut = batchTimeOut;
-      setBatchTimeOutIsSet(true);
-      return this;
-    }
-
-    public void unsetBatchTimeOut() {
-      __isset_bitfield = org.apache.thrift.EncodingUtils.clearBit(__isset_bitfield, __BATCHTIMEOUT_ISSET_ID);
-    }
-
-    /** Returns true if field batchTimeOut is set (has been assigned a value) and false otherwise */
-    public boolean isSetBatchTimeOut() {
-      return org.apache.thrift.EncodingUtils.testBit(__isset_bitfield, __BATCHTIMEOUT_ISSET_ID);
-    }
-
-    public void setBatchTimeOutIsSet(boolean value) {
-      __isset_bitfield = org.apache.thrift.EncodingUtils.setBit(__isset_bitfield, __BATCHTIMEOUT_ISSET_ID, value);
-    }
-
-    @org.apache.thrift.annotation.Nullable
-    public java.lang.String getClassLoaderContext() {
-      return this.classLoaderContext;
-    }
-
-    public startScan_args setClassLoaderContext(@org.apache.thrift.annotation.Nullable java.lang.String classLoaderContext) {
-      this.classLoaderContext = classLoaderContext;
-      return this;
-    }
-
-    public void unsetClassLoaderContext() {
-      this.classLoaderContext = null;
-    }
-
-    /** Returns true if field classLoaderContext is set (has been assigned a value) and false otherwise */
-    public boolean isSetClassLoaderContext() {
-      return this.classLoaderContext != null;
-    }
-
-    public void setClassLoaderContextIsSet(boolean value) {
-      if (!value) {
-        this.classLoaderContext = null;
-      }
-    }
-
-    public int getExecutionHintsSize() {
-      return (this.executionHints == null) ? 0 : this.executionHints.size();
-    }
-
-    public void putToExecutionHints(java.lang.String key, java.lang.String val) {
-      if (this.executionHints == null) {
-        this.executionHints = new java.util.HashMap<java.lang.String,java.lang.String>();
-      }
-      this.executionHints.put(key, val);
-    }
-
-    @org.apache.thrift.annotation.Nullable
-    public java.util.Map<java.lang.String,java.lang.String> getExecutionHints() {
-      return this.executionHints;
-    }
-
-    public startScan_args setExecutionHints(@org.apache.thrift.annotation.Nullable java.util.Map<java.lang.String,java.lang.String> executionHints) {
-      this.executionHints = executionHints;
-      return this;
-    }
-
-    public void unsetExecutionHints() {
-      this.executionHints = null;
-    }
-
-    /** Returns true if field executionHints is set (has been assigned a value) and false otherwise */
-    public boolean isSetExecutionHints() {
-      return this.executionHints != null;
-    }
-
-    public void setExecutionHintsIsSet(boolean value) {
-      if (!value) {
-        this.executionHints = null;
-      }
-    }
-
-    public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable java.lang.Object value) {
-      switch (field) {
-      case TINFO:
-        if (value == null) {
-          unsetTinfo();
-        } else {
-          setTinfo((org.apache.accumulo.core.trace.thrift.TInfo)value);
-        }
-        break;
-
-      case CREDENTIALS:
-        if (value == null) {
-          unsetCredentials();
-        } else {
-          setCredentials((org.apache.accumulo.core.securityImpl.thrift.TCredentials)value);
-        }
-        break;
-
-      case EXTENT:
-        if (value == null) {
-          unsetExtent();
-        } else {
-          setExtent((org.apache.accumulo.core.dataImpl.thrift.TKeyExtent)value);
-        }
-        break;
-
-      case RANGE:
-        if (value == null) {
-          unsetRange();
-        } else {
-          setRange((org.apache.accumulo.core.dataImpl.thrift.TRange)value);
-        }
-        break;
-
-      case COLUMNS:
-        if (value == null) {
-          unsetColumns();
-        } else {
-          setColumns((java.util.List<org.apache.accumulo.core.dataImpl.thrift.TColumn>)value);
-        }
-        break;
-
-      case BATCH_SIZE:
-        if (value == null) {
-          unsetBatchSize();
-        } else {
-          setBatchSize((java.lang.Integer)value);
-        }
-        break;
-
-      case SSI_LIST:
-        if (value == null) {
-          unsetSsiList();
-        } else {
-          setSsiList((java.util.List<org.apache.accumulo.core.dataImpl.thrift.IterInfo>)value);
-        }
-        break;
-
-      case SSIO:
-        if (value == null) {
-          unsetSsio();
-        } else {
-          setSsio((java.util.Map<java.lang.String,java.util.Map<java.lang.String,java.lang.String>>)value);
-        }
-        break;
-
-      case AUTHORIZATIONS:
-        if (value == null) {
-          unsetAuthorizations();
-        } else {
-          setAuthorizations((java.util.List<java.nio.ByteBuffer>)value);
-        }
-        break;
-
-      case WAIT_FOR_WRITES:
-        if (value == null) {
-          unsetWaitForWrites();
-        } else {
-          setWaitForWrites((java.lang.Boolean)value);
-        }
-        break;
-
-      case ISOLATED:
-        if (value == null) {
-          unsetIsolated();
-        } else {
-          setIsolated((java.lang.Boolean)value);
-        }
-        break;
-
-      case READAHEAD_THRESHOLD:
-        if (value == null) {
-          unsetReadaheadThreshold();
-        } else {
-          setReadaheadThreshold((java.lang.Long)value);
-        }
-        break;
-
-      case SAMPLER_CONFIG:
-        if (value == null) {
-          unsetSamplerConfig();
-        } else {
-          setSamplerConfig((TSamplerConfiguration)value);
-        }
-        break;
-
-      case BATCH_TIME_OUT:
-        if (value == null) {
-          unsetBatchTimeOut();
-        } else {
-          setBatchTimeOut((java.lang.Long)value);
-        }
-        break;
-
-      case CLASS_LOADER_CONTEXT:
-        if (value == null) {
-          unsetClassLoaderContext();
-        } else {
-          setClassLoaderContext((java.lang.String)value);
-        }
-        break;
-
-      case EXECUTION_HINTS:
-        if (value == null) {
-          unsetExecutionHints();
-        } else {
-          setExecutionHints((java.util.Map<java.lang.String,java.lang.String>)value);
-        }
-        break;
-
-      }
-    }
-
-    @org.apache.thrift.annotation.Nullable
-    public java.lang.Object getFieldValue(_Fields field) {
-      switch (field) {
-      case TINFO:
-        return getTinfo();
-
-      case CREDENTIALS:
-        return getCredentials();
-
-      case EXTENT:
-        return getExtent();
-
-      case RANGE:
-        return getRange();
-
-      case COLUMNS:
-        return getColumns();
-
-      case BATCH_SIZE:
-        return getBatchSize();
-
-      case SSI_LIST:
-        return getSsiList();
-
-      case SSIO:
-        return getSsio();
-
-      case AUTHORIZATIONS:
-        return getAuthorizations();
-
-      case WAIT_FOR_WRITES:
-        return isWaitForWrites();
-
-      case ISOLATED:
-        return isIsolated();
-
-      case READAHEAD_THRESHOLD:
-        return getReadaheadThreshold();
-
-      case SAMPLER_CONFIG:
-        return getSamplerConfig();
-
-      case BATCH_TIME_OUT:
-        return getBatchTimeOut();
-
-      case CLASS_LOADER_CONTEXT:
-        return getClassLoaderContext();
-
-      case EXECUTION_HINTS:
-        return getExecutionHints();
-
-      }
-      throw new java.lang.IllegalStateException();
-    }
-
-    /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
-    public boolean isSet(_Fields field) {
-      if (field == null) {
-        throw new java.lang.IllegalArgumentException();
-      }
-
-      switch (field) {
-      case TINFO:
-        return isSetTinfo();
-      case CREDENTIALS:
-        return isSetCredentials();
-      case EXTENT:
-        return isSetExtent();
-      case RANGE:
-        return isSetRange();
-      case COLUMNS:
-        return isSetColumns();
-      case BATCH_SIZE:
-        return isSetBatchSize();
-      case SSI_LIST:
-        return isSetSsiList();
-      case SSIO:
-        return isSetSsio();
-      case AUTHORIZATIONS:
-        return isSetAuthorizations();
-      case WAIT_FOR_WRITES:
-        return isSetWaitForWrites();
-      case ISOLATED:
-        return isSetIsolated();
-      case READAHEAD_THRESHOLD:
-        return isSetReadaheadThreshold();
-      case SAMPLER_CONFIG:
-        return isSetSamplerConfig();
-      case BATCH_TIME_OUT:
-        return isSetBatchTimeOut();
-      case CLASS_LOADER_CONTEXT:
-        return isSetClassLoaderContext();
-      case EXECUTION_HINTS:
-        return isSetExecutionHints();
-      }
-      throw new java.lang.IllegalStateException();
-    }
-
-    @Override
-    public boolean equals(java.lang.Object that) {
-      if (that instanceof startScan_args)
-        return this.equals((startScan_args)that);
-      return false;
-    }
-
-    public boolean equals(startScan_args that) {
-      if (that == null)
-        return false;
-      if (this == that)
-        return true;
-
-      boolean this_present_tinfo = true && this.isSetTinfo();
-      boolean that_present_tinfo = true && that.isSetTinfo();
-      if (this_present_tinfo || that_present_tinfo) {
-        if (!(this_present_tinfo && that_present_tinfo))
-          return false;
-        if (!this.tinfo.equals(that.tinfo))
-          return false;
-      }
-
-      boolean this_present_credentials = true && this.isSetCredentials();
-      boolean that_present_credentials = true && that.isSetCredentials();
-      if (this_present_credentials || that_present_credentials) {
-        if (!(this_present_credentials && that_present_credentials))
-          return false;
-        if (!this.credentials.equals(that.credentials))
-          return false;
-      }
-
-      boolean this_present_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_range = true && this.isSetRange();
-      boolean that_present_range = true && that.isSetRange();
-      if (this_present_range || that_present_range) {
-        if (!(this_present_range && that_present_range))
-          return false;
-        if (!this.range.equals(that.range))
-          return false;
-      }
-
-      boolean this_present_columns = true && this.isSetColumns();
-      boolean that_present_columns = true && that.isSetColumns();
-      if (this_present_columns || that_present_columns) {
-        if (!(this_present_columns && that_present_columns))
-          return false;
-        if (!this.columns.equals(that.columns))
-          return false;
-      }
-
-      boolean this_present_batchSize = true;
-      boolean that_present_batchSize = true;
-      if (this_present_batchSize || that_present_batchSize) {
-        if (!(this_present_batchSize && that_present_batchSize))
-          return false;
-        if (this.batchSize != that.batchSize)
-          return false;
-      }
-
-      boolean this_present_ssiList = true && this.isSetSsiList();
-      boolean that_present_ssiList = true && that.isSetSsiList();
-      if (this_present_ssiList || that_present_ssiList) {
-        if (!(this_present_ssiList && that_present_ssiList))
-          return false;
-        if (!this.ssiList.equals(that.ssiList))
-          return false;
-      }
-
-      boolean this_present_ssio = true && this.isSetSsio();
-      boolean that_present_ssio = true && that.isSetSsio();
-      if (this_present_ssio || that_present_ssio) {
-        if (!(this_present_ssio && that_present_ssio))
-          return false;
-        if (!this.ssio.equals(that.ssio))
-          return false;
-      }
-
-      boolean this_present_authorizations = true && this.isSetAuthorizations();
-      boolean that_present_authorizations = true && that.isSetAuthorizations();
-      if (this_present_authorizations || that_present_authorizations) {
-        if (!(this_present_authorizations && that_present_authorizations))
-          return false;
-        if (!this.authorizations.equals(that.authorizations))
-          return false;
-      }
-
-      boolean this_present_waitForWrites = true;
-      boolean that_present_waitForWrites = true;
-      if (this_present_waitForWrites || that_present_waitForWrites) {
-        if (!(this_present_waitForWrites && that_present_waitForWrites))
-          return false;
-        if (this.waitForWrites != that.waitForWrites)
-          return false;
-      }
-
-      boolean this_present_isolated = true;
-      boolean that_present_isolated = true;
-      if (this_present_isolated || that_present_isolated) {
-        if (!(this_present_isolated && that_present_isolated))
-          return false;
-        if (this.isolated != that.isolated)
-          return false;
-      }
-
-      boolean this_present_readaheadThreshold = true;
-      boolean that_present_readaheadThreshold = true;
-      if (this_present_readaheadThreshold || that_present_readaheadThreshold) {
-        if (!(this_present_readaheadThreshold && that_present_readaheadThreshold))
-          return false;
-        if (this.readaheadThreshold != that.readaheadThreshold)
-          return false;
-      }
-
-      boolean this_present_samplerConfig = true && this.isSetSamplerConfig();
-      boolean that_present_samplerConfig = true && that.isSetSamplerConfig();
-      if (this_present_samplerConfig || that_present_samplerConfig) {
-        if (!(this_present_samplerConfig && that_present_samplerConfig))
-          return false;
-        if (!this.samplerConfig.equals(that.samplerConfig))
-          return false;
-      }
-
-      boolean this_present_batchTimeOut = true;
-      boolean that_present_batchTimeOut = true;
-      if (this_present_batchTimeOut || that_present_batchTimeOut) {
-        if (!(this_present_batchTimeOut && that_present_batchTimeOut))
-          return false;
-        if (this.batchTimeOut != that.batchTimeOut)
-          return false;
-      }
-
-      boolean this_present_classLoaderContext = true && this.isSetClassLoaderContext();
-      boolean that_present_classLoaderContext = true && that.isSetClassLoaderContext();
-      if (this_present_classLoaderContext || that_present_classLoaderContext) {
-        if (!(this_present_classLoaderContext && that_present_classLoaderContext))
-          return false;
-        if (!this.classLoaderContext.equals(that.classLoaderContext))
-          return false;
-      }
-
-      boolean this_present_executionHints = true && this.isSetExecutionHints();
-      boolean that_present_executionHints = true && that.isSetExecutionHints();
-      if (this_present_executionHints || that_present_executionHints) {
-        if (!(this_present_executionHints && that_present_executionHints))
-          return false;
-        if (!this.executionHints.equals(that.executionHints))
-          return false;
-      }
-
-      return true;
-    }
-
-    @Override
-    public int hashCode() {
-      int hashCode = 1;
-
-      hashCode = hashCode * 8191 + ((isSetTinfo()) ? 131071 : 524287);
-      if (isSetTinfo())
-        hashCode = hashCode * 8191 + tinfo.hashCode();
-
-      hashCode = hashCode * 8191 + ((isSetCredentials()) ? 131071 : 524287);
-      if (isSetCredentials())
-        hashCode = hashCode * 8191 + credentials.hashCode();
-
-      hashCode = hashCode * 8191 + ((isSetExtent()) ? 131071 : 524287);
-      if (isSetExtent())
-        hashCode = hashCode * 8191 + extent.hashCode();
-
-      hashCode = hashCode * 8191 + ((isSetRange()) ? 131071 : 524287);
-      if (isSetRange())
-        hashCode = hashCode * 8191 + range.hashCode();
-
-      hashCode = hashCode * 8191 + ((isSetColumns()) ? 131071 : 524287);
-      if (isSetColumns())
-        hashCode = hashCode * 8191 + columns.hashCode();
-
-      hashCode = hashCode * 8191 + batchSize;
-
-      hashCode = hashCode * 8191 + ((isSetSsiList()) ? 131071 : 524287);
-      if (isSetSsiList())
-        hashCode = hashCode * 8191 + ssiList.hashCode();
-
-      hashCode = hashCode * 8191 + ((isSetSsio()) ? 131071 : 524287);
-      if (isSetSsio())
-        hashCode = hashCode * 8191 + ssio.hashCode();
-
-      hashCode = hashCode * 8191 + ((isSetAuthorizations()) ? 131071 : 524287);
-      if (isSetAuthorizations())
-        hashCode = hashCode * 8191 + authorizations.hashCode();
-
-      hashCode = hashCode * 8191 + ((waitForWrites) ? 131071 : 524287);
-
-      hashCode = hashCode * 8191 + ((isolated) ? 131071 : 524287);
-
-      hashCode = hashCode * 8191 + org.apache.thrift.TBaseHelper.hashCode(readaheadThreshold);
-
-      hashCode = hashCode * 8191 + ((isSetSamplerConfig()) ? 131071 : 524287);
-      if (isSetSamplerConfig())
-        hashCode = hashCode * 8191 + samplerConfig.hashCode();
-
-      hashCode = hashCode * 8191 + org.apache.thrift.TBaseHelper.hashCode(batchTimeOut);
-
-      hashCode = hashCode * 8191 + ((isSetClassLoaderContext()) ? 131071 : 524287);
-      if (isSetClassLoaderContext())
-        hashCode = hashCode * 8191 + classLoaderContext.hashCode();
-
-      hashCode = hashCode * 8191 + ((isSetExecutionHints()) ? 131071 : 524287);
-      if (isSetExecutionHints())
-        hashCode = hashCode * 8191 + executionHints.hashCode();
-
-      return hashCode;
-    }
-
-    @Override
-    public int compareTo(startScan_args other) {
-      if (!getClass().equals(other.getClass())) {
-        return getClass().getName().compareTo(other.getClass().getName());
-      }
-
-      int lastComparison = 0;
-
-      lastComparison = java.lang.Boolean.compare(isSetTinfo(), other.isSetTinfo());
-      if (lastComparison != 0) {
-        return lastComparison;
-      }
-      if (isSetTinfo()) {
-        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tinfo, other.tinfo);
-        if (lastComparison != 0) {
-          return lastComparison;
-        }
-      }
-      lastComparison = java.lang.Boolean.compare(isSetCredentials(), 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.compare(isSetExtent(), 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.compare(isSetRange(), other.isSetRange());
-      if (lastComparison != 0) {
-        return lastComparison;
-      }
-      if (isSetRange()) {
-        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.range, other.range);
-        if (lastComparison != 0) {
-          return lastComparison;
-        }
-      }
-      lastComparison = java.lang.Boolean.compare(isSetColumns(), other.isSetColumns());
-      if (lastComparison != 0) {
-        return lastComparison;
-      }
-      if (isSetColumns()) {
-        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.columns, other.columns);
-        if (lastComparison != 0) {
-          return lastComparison;
-        }
-      }
-      lastComparison = java.lang.Boolean.compare(isSetBatchSize(), other.isSetBatchSize());
-      if (lastComparison != 0) {
-        return lastComparison;
-      }
-      if (isSetBatchSize()) {
-        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.batchSize, other.batchSize);
-        if (lastComparison != 0) {
-          return lastComparison;
-        }
-      }
-      lastComparison = java.lang.Boolean.compare(isSetSsiList(), other.isSetSsiList());
-      if (lastComparison != 0) {
-        return lastComparison;
-      }
-      if (isSetSsiList()) {
-        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.ssiList, other.ssiList);
-        if (lastComparison != 0) {
-          return lastComparison;
-        }
-      }
-      lastComparison = java.lang.Boolean.compare(isSetSsio(), other.isSetSsio());
-      if (lastComparison != 0) {
-        return lastComparison;
-      }
-      if (isSetSsio()) {
-        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.ssio, other.ssio);
-        if (lastComparison != 0) {
-          return lastComparison;
-        }
-      }
-      lastComparison = java.lang.Boolean.compare(isSetAuthorizations(), other.isSetAuthorizations());
-      if (lastComparison != 0) {
-        return lastComparison;
-      }
-      if (isSetAuthorizations()) {
-        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.authorizations, other.authorizations);
-        if (lastComparison != 0) {
-          return lastComparison;
-        }
-      }
-      lastComparison = java.lang.Boolean.compare(isSetWaitForWrites(), other.isSetWaitForWrites());
-      if (lastComparison != 0) {
-        return lastComparison;
-      }
-      if (isSetWaitForWrites()) {
-        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.waitForWrites, other.waitForWrites);
-        if (lastComparison != 0) {
-          return lastComparison;
-        }
-      }
-      lastComparison = java.lang.Boolean.compare(isSetIsolated(), other.isSetIsolated());
-      if (lastComparison != 0) {
-        return lastComparison;
-      }
-      if (isSetIsolated()) {
-        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.isolated, other.isolated);
-        if (lastComparison != 0) {
-          return lastComparison;
-        }
-      }
-      lastComparison = java.lang.Boolean.compare(isSetReadaheadThreshold(), other.isSetReadaheadThreshold());
-      if (lastComparison != 0) {
-        return lastComparison;
-      }
-      if (isSetReadaheadThreshold()) {
-        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.readaheadThreshold, other.readaheadThreshold);
-        if (lastComparison != 0) {
-          return lastComparison;
-        }
-      }
-      lastComparison = java.lang.Boolean.compare(isSetSamplerConfig(), other.isSetSamplerConfig());
-      if (lastComparison != 0) {
-        return lastComparison;
-      }
-      if (isSetSamplerConfig()) {
-        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.samplerConfig, other.samplerConfig);
-        if (lastComparison != 0) {
-          return lastComparison;
-        }
-      }
-      lastComparison = java.lang.Boolean.compare(isSetBatchTimeOut(), other.isSetBatchTimeOut());
-      if (lastComparison != 0) {
-        return lastComparison;
-      }
-      if (isSetBatchTimeOut()) {
-        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.batchTimeOut, other.batchTimeOut);
-        if (lastComparison != 0) {
-          return lastComparison;
-        }
-      }
-      lastComparison = java.lang.Boolean.compare(isSetClassLoaderContext(), other.isSetClassLoaderContext());
-      if (lastComparison != 0) {
-        return lastComparison;
-      }
-      if (isSetClassLoaderContext()) {
-        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.classLoaderContext, other.classLoaderContext);
-        if (lastComparison != 0) {
-          return lastComparison;
-        }
-      }
-      lastComparison = java.lang.Boolean.compare(isSetExecutionHints(), other.isSetExecutionHints());
-      if (lastComparison != 0) {
-        return lastComparison;
-      }
-      if (isSetExecutionHints()) {
-        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.executionHints, other.executionHints);
-        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("startScan_args(");
-      boolean first = true;
-
-      sb.append("tinfo:");
-      if (this.tinfo == null) {
-        sb.append("null");
-      } else {
-        sb.append(this.tinfo);
-      }
-      first = false;
-      if (!first) sb.append(", ");
-      sb.append("credentials:");
-      if (this.credentials == null) {
-        sb.append("null");
-      } else {
-        sb.append(this.credentials);
-      }
-      first = false;
-      if (!first) sb.append(", ");
-      sb.append("extent:");
-      if (this.extent == null) {
-        sb.append("null");
-      } else {
-        sb.append(this.extent);
-      }
-      first = false;
-      if (!first) sb.append(", ");
-      sb.append("range:");
-      if (this.range == null) {
-        sb.append("null");
-      } else {
-        sb.append(this.range);
-      }
-      first = false;
-      if (!first) sb.append(", ");
-      sb.append("columns:");
-      if (this.columns == null) {
-        sb.append("null");
-      } else {
-        sb.append(this.columns);
-      }
-      first = false;
-      if (!first) sb.append(", ");
-      sb.append("batchSize:");
-      sb.append(this.batchSize);
-      first = false;
-      if (!first) sb.append(", ");
-      sb.append("ssiList:");
-      if (this.ssiList == null) {
-        sb.append("null");
-      } else {
-        sb.append(this.ssiList);
-      }
-      first = false;
-      if (!first) sb.append(", ");
-      sb.append("ssio:");
-      if (this.ssio == null) {
-        sb.append("null");
-      } else {
-        sb.append(this.ssio);
-      }
-      first = false;
-      if (!first) sb.append(", ");
-      sb.append("authorizations:");
-      if (this.authorizations == null) {
-        sb.append("null");
-      } else {
-        org.apache.thrift.TBaseHelper.toString(this.authorizations, sb);
-      }
-      first = false;
-      if (!first) sb.append(", ");
-      sb.append("waitForWrites:");
-      sb.append(this.waitForWrites);
-      first = false;
-      if (!first) sb.append(", ");
-      sb.append("isolated:");
-      sb.append(this.isolated);
-      first = false;
-      if (!first) sb.append(", ");
-      sb.append("readaheadThreshold:");
-      sb.append(this.readaheadThreshold);
-      first = false;
-      if (!first) sb.append(", ");
-      sb.append("samplerConfig:");
-      if (this.samplerConfig == null) {
-        sb.append("null");
-      } else {
-        sb.append(this.samplerConfig);
-      }
-      first = false;
-      if (!first) sb.append(", ");
-      sb.append("batchTimeOut:");
-      sb.append(this.batchTimeOut);
-      first = false;
-      if (!first) sb.append(", ");
-      sb.append("classLoaderContext:");
-      if (this.classLoaderContext == null) {
-        sb.append("null");
-      } else {
-        sb.append(this.classLoaderContext);
-      }
-      first = false;
-      if (!first) sb.append(", ");
-      sb.append("executionHints:");
-      if (this.executionHints == null) {
-        sb.append("null");
-      } else {
-        sb.append(this.executionHints);
-      }
-      first = false;
-      sb.append(")");
-      return sb.toString();
-    }
-
-    public void validate() throws org.apache.thrift.TException {
-      // check for required fields
-      // check for sub-struct validity
-      if (tinfo != null) {
-        tinfo.validate();
-      }
-      if (credentials != null) {
-        credentials.validate();
-      }
-      if (extent != null) {
-        extent.validate();
-      }
-      if (range != null) {
-        range.validate();
-      }
-      if (samplerConfig != null) {
-        samplerConfig.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 startScan_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      public startScan_argsStandardScheme getScheme() {
-        return new startScan_argsStandardScheme();
-      }
-    }
-
-    private static class startScan_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme<startScan_args> {
-
-      public void read(org.apache.thrift.protocol.TProtocol iprot, startScan_args struct) throws org.apache.thrift.TException {
-        org.apache.thrift.protocol.TField schemeField;
-        iprot.readStructBegin();
-        while (true)
-        {
-          schemeField = iprot.readFieldBegin();
-          if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
-            break;
-          }
-          switch (schemeField.id) {
-            case 11: // TINFO
-              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
-                struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
-                struct.tinfo.read(iprot);
-                struct.setTinfoIsSet(true);
-              } else { 
-                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
-              }
-              break;
-            case 1: // 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 2: // 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 3: // RANGE
-              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
-                struct.range = new org.apache.accumulo.core.dataImpl.thrift.TRange();
-                struct.range.read(iprot);
-                struct.setRangeIsSet(true);
-              } else { 
-                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
-              }
-              break;
-            case 4: // COLUMNS
-              if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
-                {
-                  org.apache.thrift.protocol.TList _list134 = iprot.readListBegin();
-                  struct.columns = new java.util.ArrayList<org.apache.accumulo.core.dataImpl.thrift.TColumn>(_list134.size);
-                  @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TColumn _elem135;
-                  for (int _i136 = 0; _i136 < _list134.size; ++_i136)
-                  {
-                    _elem135 = new org.apache.accumulo.core.dataImpl.thrift.TColumn();
-                    _elem135.read(iprot);
-                    struct.columns.add(_elem135);
-                  }
-                  iprot.readListEnd();
-                }
-                struct.setColumnsIsSet(true);
-              } else { 
-                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
-              }
-              break;
-            case 5: // BATCH_SIZE
-              if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
-                struct.batchSize = iprot.readI32();
-                struct.setBatchSizeIsSet(true);
-              } else { 
-                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
-              }
-              break;
-            case 6: // SSI_LIST
-              if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
-                {
-                  org.apache.thrift.protocol.TList _list137 = iprot.readListBegin();
-                  struct.ssiList = new java.util.ArrayList<org.apache.accumulo.core.dataImpl.thrift.IterInfo>(_list137.size);
-                  @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.IterInfo _elem138;
-                  for (int _i139 = 0; _i139 < _list137.size; ++_i139)
-                  {
-                    _elem138 = new org.apache.accumulo.core.dataImpl.thrift.IterInfo();
-                    _elem138.read(iprot);
-                    struct.ssiList.add(_elem138);
-                  }
-                  iprot.readListEnd();
-                }
-                struct.setSsiListIsSet(true);
-              } else { 
-                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
-              }
-              break;
-            case 7: // SSIO
-              if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
-                {
-                  org.apache.thrift.protocol.TMap _map140 = iprot.readMapBegin();
-                  struct.ssio = new java.util.HashMap<java.lang.String,java.util.Map<java.lang.String,java.lang.String>>(2*_map140.size);
-                  @org.apache.thrift.annotation.Nullable java.lang.String _key141;
-                  @org.apache.thrift.annotation.Nullable java.util.Map<java.lang.String,java.lang.String> _val142;
-                  for (int _i143 = 0; _i143 < _map140.size; ++_i143)
-                  {
-                    _key141 = iprot.readString();
-                    {
-                      org.apache.thrift.protocol.TMap _map144 = iprot.readMapBegin();
-                      _val142 = new java.util.HashMap<java.lang.String,java.lang.String>(2*_map144.size);
-                      @org.apache.thrift.annotation.Nullable java.lang.String _key145;
-                      @org.apache.thrift.annotation.Nullable java.lang.String _val146;
-                      for (int _i147 = 0; _i147 < _map144.size; ++_i147)
-                      {
-                        _key145 = iprot.readString();
-                        _val146 = iprot.readString();
-                        _val142.put(_key145, _val146);
-                      }
-                      iprot.readMapEnd();
-                    }
-                    struct.ssio.put(_key141, _val142);
-                  }
-                  iprot.readMapEnd();
-                }
-                struct.setSsioIsSet(true);
-              } else { 
-                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
-              }
-              break;
-            case 8: // AUTHORIZATIONS
-              if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
-                {
-                  org.apache.thrift.protocol.TList _list148 = iprot.readListBegin();
-                  struct.authorizations = new java.util.ArrayList<java.nio.ByteBuffer>(_list148.size);
-                  @org.apache.thrift.annotation.Nullable java.nio.ByteBuffer _elem149;
-                  for (int _i150 = 0; _i150 < _list148.size; ++_i150)
-                  {
-                    _elem149 = iprot.readBinary();
-                    struct.authorizations.add(_elem149);
-                  }
-                  iprot.readListEnd();
-                }
-                struct.setAuthorizationsIsSet(true);
-              } else { 
-                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
-              }
-              break;
-            case 9: // WAIT_FOR_WRITES
-              if (schemeField.type == org.apache.thrift.protocol.TType.BOOL) {
-                struct.waitForWrites = iprot.readBool();
-                struct.setWaitForWritesIsSet(true);
-              } else { 
-                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
-              }
-              break;
-            case 10: // ISOLATED
-              if (schemeField.type == org.apache.thrift.protocol.TType.BOOL) {
-                struct.isolated = iprot.readBool();
-                struct.setIsolatedIsSet(true);
-              } else { 
-                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
-              }
-              break;
-            case 12: // READAHEAD_THRESHOLD
-              if (schemeField.type == org.apache.thrift.protocol.TType.I64) {
-                struct.readaheadThreshold = iprot.readI64();
-                struct.setReadaheadThresholdIsSet(true);
-              } else { 
-                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
-              }
-              break;
-            case 13: // SAMPLER_CONFIG
-              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
-                struct.samplerConfig = new TSamplerConfiguration();
-                struct.samplerConfig.read(iprot);
-                struct.setSamplerConfigIsSet(true);
-              } else { 
-                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
-              }
-              break;
-            case 14: // BATCH_TIME_OUT
-              if (schemeField.type == org.apache.thrift.protocol.TType.I64) {
-                struct.batchTimeOut = iprot.readI64();
-                struct.setBatchTimeOutIsSet(true);
-              } else { 
-                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
-              }
-              break;
-            case 15: // CLASS_LOADER_CONTEXT
-              if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
-                struct.classLoaderContext = iprot.readString();
-                struct.setClassLoaderContextIsSet(true);
-              } else { 
-                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
-              }
-              break;
-            case 16: // EXECUTION_HINTS
-              if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
-                {
-                  org.apache.thrift.protocol.TMap _map151 = iprot.readMapBegin();
-                  struct.executionHints = new java.util.HashMap<java.lang.String,java.lang.String>(2*_map151.size);
-                  @org.apache.thrift.annotation.Nullable java.lang.String _key152;
-                  @org.apache.thrift.annotation.Nullable java.lang.String _val153;
-                  for (int _i154 = 0; _i154 < _map151.size; ++_i154)
-                  {
-                    _key152 = iprot.readString();
-                    _val153 = iprot.readString();
-                    struct.executionHints.put(_key152, _val153);
-                  }
-                  iprot.readMapEnd();
-                }
-                struct.setExecutionHintsIsSet(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, startScan_args struct) throws org.apache.thrift.TException {
-        struct.validate();
-
-        oprot.writeStructBegin(STRUCT_DESC);
-        if (struct.credentials != null) {
-          oprot.writeFieldBegin(CREDENTIALS_FIELD_DESC);
-          struct.credentials.write(oprot);
-          oprot.writeFieldEnd();
-        }
-        if (struct.extent != null) {
-          oprot.writeFieldBegin(EXTENT_FIELD_DESC);
-          struct.extent.write(oprot);
-          oprot.writeFieldEnd();
-        }
-        if (struct.range != null) {
-          oprot.writeFieldBegin(RANGE_FIELD_DESC);
-          struct.range.write(oprot);
-          oprot.writeFieldEnd();
-        }
-        if (struct.columns != null) {
-          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 _iter155 : struct.columns)
-            {
-              _iter155.write(oprot);
-            }
-            oprot.writeListEnd();
-          }
-          oprot.writeFieldEnd();
-        }
-        oprot.writeFieldBegin(BATCH_SIZE_FIELD_DESC);
-        oprot.writeI32(struct.batchSize);
-        oprot.writeFieldEnd();
-        if (struct.ssiList != null) {
-          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 _iter156 : struct.ssiList)
-            {
-              _iter156.write(oprot);
-            }
-            oprot.writeListEnd();
-          }
-          oprot.writeFieldEnd();
-        }
-        if (struct.ssio != null) {
-          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>> _iter157 : struct.ssio.entrySet())
-            {
-              oprot.writeString(_iter157.getKey());
-              {
-                oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, _iter157.getValue().size()));
-                for (java.util.Map.Entry<java.lang.String, java.lang.String> _iter158 : _iter157.getValue().entrySet())
-                {
-                  oprot.writeString(_iter158.getKey());
-                  oprot.writeString(_iter158.getValue());
-                }
-                oprot.writeMapEnd();
-              }
-            }
-            oprot.writeMapEnd();
-          }
-          oprot.writeFieldEnd();
-        }
-        if (struct.authorizations != null) {
-          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 _iter159 : struct.authorizations)
-            {
-              oprot.writeBinary(_iter159);
-            }
-            oprot.writeListEnd();
-          }
-          oprot.writeFieldEnd();
-        }
-        oprot.writeFieldBegin(WAIT_FOR_WRITES_FIELD_DESC);
-        oprot.writeBool(struct.waitForWrites);
-        oprot.writeFieldEnd();
-        oprot.writeFieldBegin(ISOLATED_FIELD_DESC);
-        oprot.writeBool(struct.isolated);
-        oprot.writeFieldEnd();
-        if (struct.tinfo != null) {
-          oprot.writeFieldBegin(TINFO_FIELD_DESC);
-          struct.tinfo.write(oprot);
-          oprot.writeFieldEnd();
-        }
-        oprot.writeFieldBegin(READAHEAD_THRESHOLD_FIELD_DESC);
-        oprot.writeI64(struct.readaheadThreshold);
-        oprot.writeFieldEnd();
-        if (struct.samplerConfig != null) {
-          oprot.writeFieldBegin(SAMPLER_CONFIG_FIELD_DESC);
-          struct.samplerConfig.write(oprot);
-          oprot.writeFieldEnd();
-        }
-        oprot.writeFieldBegin(BATCH_TIME_OUT_FIELD_DESC);
-        oprot.writeI64(struct.batchTimeOut);
-        oprot.writeFieldEnd();
-        if (struct.classLoaderContext != null) {
-          oprot.writeFieldBegin(CLASS_LOADER_CONTEXT_FIELD_DESC);
-          oprot.writeString(struct.classLoaderContext);
-          oprot.writeFieldEnd();
-        }
-        if (struct.executionHints != null) {
-          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> _iter160 : struct.executionHints.entrySet())
-            {
-              oprot.writeString(_iter160.getKey());
-              oprot.writeString(_iter160.getValue());
-            }
-            oprot.writeMapEnd();
-          }
-          oprot.writeFieldEnd();
-        }
-        oprot.writeFieldStop();
-        oprot.writeStructEnd();
-      }
-
-    }
-
-    private static class startScan_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      public startScan_argsTupleScheme getScheme() {
-        return new startScan_argsTupleScheme();
-      }
-    }
-
-    private static class startScan_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme<startScan_args> {
-
-      @Override
-      public void write(org.apache.thrift.protocol.TProtocol prot, startScan_args struct) throws org.apache.thrift.TException {
-        org.apache.thrift.protocol.TTupleProtocol oprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
-        java.util.BitSet optionals = new java.util.BitSet();
-        if (struct.isSetTinfo()) {
-          optionals.set(0);
-        }
-        if (struct.isSetCredentials()) {
-          optionals.set(1);
-        }
-        if (struct.isSetExtent()) {
-          optionals.set(2);
-        }
-        if (struct.isSetRange()) {
-          optionals.set(3);
-        }
-        if (struct.isSetColumns()) {
-          optionals.set(4);
-        }
-        if (struct.isSetBatchSize()) {
-          optionals.set(5);
-        }
-        if (struct.isSetSsiList()) {
-          optionals.set(6);
-        }
-        if (struct.isSetSsio()) {
-          optionals.set(7);
-        }
-        if (struct.isSetAuthorizations()) {
-          optionals.set(8);
-        }
-        if (struct.isSetWaitForWrites()) {
-          optionals.set(9);
-        }
-        if (struct.isSetIsolated()) {
-          optionals.set(10);
-        }
-        if (struct.isSetReadaheadThreshold()) {
-          optionals.set(11);
-        }
-        if (struct.isSetSamplerConfig()) {
-          optionals.set(12);
-        }
-        if (struct.isSetBatchTimeOut()) {
-          optionals.set(13);
-        }
-        if (struct.isSetClassLoaderContext()) {
-          optionals.set(14);
-        }
-        if (struct.isSetExecutionHints()) {
-          optionals.set(15);
-        }
-        oprot.writeBitSet(optionals, 16);
-        if (struct.isSetTinfo()) {
-          struct.tinfo.write(oprot);
-        }
-        if (struct.isSetCredentials()) {
-          struct.credentials.write(oprot);
-        }
-        if (struct.isSetExtent()) {
-          struct.extent.write(oprot);
-        }
-        if (struct.isSetRange()) {
-          struct.range.write(oprot);
-        }
-        if (struct.isSetColumns()) {
-          {
-            oprot.writeI32(struct.columns.size());
-            for (org.apache.accumulo.core.dataImpl.thrift.TColumn _iter161 : struct.columns)
-            {
-              _iter161.write(oprot);
-            }
-          }
-        }
-        if (struct.isSetBatchSize()) {
-          oprot.writeI32(struct.batchSize);
-        }
-        if (struct.isSetSsiList()) {
-          {
-            oprot.writeI32(struct.ssiList.size());
-            for (org.apache.accumulo.core.dataImpl.thrift.IterInfo _iter162 : struct.ssiList)
-            {
-              _iter162.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>> _iter163 : struct.ssio.entrySet())
-            {
-              oprot.writeString(_iter163.getKey());
-              {
-                oprot.writeI32(_iter163.getValue().size());
-                for (java.util.Map.Entry<java.lang.String, java.lang.String> _iter164 : _iter163.getValue().entrySet())
-                {
-                  oprot.writeString(_iter164.getKey());
-                  oprot.writeString(_iter164.getValue());
-                }
-              }
-            }
-          }
-        }
-        if (struct.isSetAuthorizations()) {
-          {
-            oprot.writeI32(struct.authorizations.size());
-            for (java.nio.ByteBuffer _iter165 : struct.authorizations)
-            {
-              oprot.writeBinary(_iter165);
-            }
-          }
-        }
-        if (struct.isSetWaitForWrites()) {
-          oprot.writeBool(struct.waitForWrites);
-        }
-        if (struct.isSetIsolated()) {
-          oprot.writeBool(struct.isolated);
-        }
-        if (struct.isSetReadaheadThreshold()) {
-          oprot.writeI64(struct.readaheadThreshold);
-        }
-        if (struct.isSetSamplerConfig()) {
-          struct.samplerConfig.write(oprot);
-        }
-        if (struct.isSetBatchTimeOut()) {
-          oprot.writeI64(struct.batchTimeOut);
-        }
-        if (struct.isSetClassLoaderContext()) {
-          oprot.writeString(struct.classLoaderContext);
-        }
-        if (struct.isSetExecutionHints()) {
-          {
-            oprot.writeI32(struct.executionHints.size());
-            for (java.util.Map.Entry<java.lang.String, java.lang.String> _iter166 : struct.executionHints.entrySet())
-            {
-              oprot.writeString(_iter166.getKey());
-              oprot.writeString(_iter166.getValue());
-            }
-          }
-        }
-      }
-
-      @Override
-      public void read(org.apache.thrift.protocol.TProtocol prot, startScan_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(16);
-        if (incoming.get(0)) {
-          struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
-          struct.tinfo.read(iprot);
-          struct.setTinfoIsSet(true);
-        }
-        if (incoming.get(1)) {
-          struct.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials();
-          struct.credentials.read(iprot);
-          struct.setCredentialsIsSet(true);
-        }
-        if (incoming.get(2)) {
-          struct.extent = new org.apache.accumulo.core.dataImpl.thrift.TKeyExtent();
-          struct.extent.read(iprot);
-          struct.setExtentIsSet(true);
-        }
-        if (incoming.get(3)) {
-          struct.range = new org.apache.accumulo.core.dataImpl.thrift.TRange();
-          struct.range.read(iprot);
-          struct.setRangeIsSet(true);
-        }
-        if (incoming.get(4)) {
-          {
-            org.apache.thrift.protocol.TList _list167 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT);
-            struct.columns = new java.util.ArrayList<org.apache.accumulo.core.dataImpl.thrift.TColumn>(_list167.size);
-            @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TColumn _elem168;
-            for (int _i169 = 0; _i169 < _list167.size; ++_i169)
-            {
-              _elem168 = new org.apache.accumulo.core.dataImpl.thrift.TColumn();
-              _elem168.read(iprot);
-              struct.columns.add(_elem168);
-            }
-          }
-          struct.setColumnsIsSet(true);
-        }
-        if (incoming.get(5)) {
-          struct.batchSize = iprot.readI32();
-          struct.setBatchSizeIsSet(true);
-        }
-        if (incoming.get(6)) {
-          {
-            org.apache.thrift.protocol.TList _list170 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT);
-            struct.ssiList = new java.util.ArrayList<org.apache.accumulo.core.dataImpl.thrift.IterInfo>(_list170.size);
-            @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.IterInfo _elem171;
-            for (int _i172 = 0; _i172 < _list170.size; ++_i172)
-            {
-              _elem171 = new org.apache.accumulo.core.dataImpl.thrift.IterInfo();
-              _elem171.read(iprot);
-              struct.ssiList.add(_elem171);
-            }
-          }
-          struct.setSsiListIsSet(true);
-        }
-        if (incoming.get(7)) {
-          {
-            org.apache.thrift.protocol.TMap _map173 = iprot.readMapBegin(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.MAP); 
-            struct.ssio = new java.util.HashMap<java.lang.String,java.util.Map<java.lang.String,java.lang.String>>(2*_map173.size);
-            @org.apache.thrift.annotation.Nullable java.lang.String _key174;
-            @org.apache.thrift.annotation.Nullable java.util.Map<java.lang.String,java.lang.String> _val175;
-            for (int _i176 = 0; _i176 < _map173.size; ++_i176)
-            {
-              _key174 = iprot.readString();
-              {
-                org.apache.thrift.protocol.TMap _map177 = iprot.readMapBegin(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING); 
-                _val175 = new java.util.HashMap<java.lang.String,java.lang.String>(2*_map177.size);
-                @org.apache.thrift.annotation.Nullable java.lang.String _key178;
-                @org.apache.thrift.annotation.Nullable java.lang.String _val179;
-                for (int _i180 = 0; _i180 < _map177.size; ++_i180)
-                {
-                  _key178 = iprot.readString();
-                  _val179 = iprot.readString();
-                  _val175.put(_key178, _val179);
-                }
-              }
-              struct.ssio.put(_key174, _val175);
-            }
-          }
-          struct.setSsioIsSet(true);
-        }
-        if (incoming.get(8)) {
-          {
-            org.apache.thrift.protocol.TList _list181 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING);
-            struct.authorizations = new java.util.ArrayList<java.nio.ByteBuffer>(_list181.size);
-            @org.apache.thrift.annotation.Nullable java.nio.ByteBuffer _elem182;
-            for (int _i183 = 0; _i183 < _list181.size; ++_i183)
-            {
-              _elem182 = iprot.readBinary();
-              struct.authorizations.add(_elem182);
-            }
-          }
-          struct.setAuthorizationsIsSet(true);
-        }
-        if (incoming.get(9)) {
-          struct.waitForWrites = iprot.readBool();
-          struct.setWaitForWritesIsSet(true);
-        }
-        if (incoming.get(10)) {
-          struct.isolated = iprot.readBool();
-          struct.setIsolatedIsSet(true);
-        }
-        if (incoming.get(11)) {
-          struct.readaheadThreshold = iprot.readI64();
-          struct.setReadaheadThresholdIsSet(true);
-        }
-        if (incoming.get(12)) {
-          struct.samplerConfig = new TSamplerConfiguration();
-          struct.samplerConfig.read(iprot);
-          struct.setSamplerConfigIsSet(true);
-        }
-        if (incoming.get(13)) {
-          struct.batchTimeOut = iprot.readI64();
-          struct.setBatchTimeOutIsSet(true);
-        }
-        if (incoming.get(14)) {
-          struct.classLoaderContext = iprot.readString();
-          struct.setClassLoaderContextIsSet(true);
-        }
-        if (incoming.get(15)) {
-          {
-            org.apache.thrift.protocol.TMap _map184 = iprot.readMapBegin(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING); 
-            struct.executionHints = new java.util.HashMap<java.lang.String,java.lang.String>(2*_map184.size);
-            @org.apache.thrift.annotation.Nullable java.lang.String _key185;
-            @org.apache.thrift.annotation.Nullable java.lang.String _val186;
-            for (int _i187 = 0; _i187 < _map184.size; ++_i187)
-            {
-              _key185 = iprot.readString();
-              _val186 = iprot.readString();
-              struct.executionHints.put(_key185, _val186);
-            }
-          }
-          struct.setExecutionHintsIsSet(true);
-        }
-      }
-    }
-
-    private static <S extends org.apache.thrift.scheme.IScheme> S scheme(org.apache.thrift.protocol.TProtocol proto) {
-      return (org.apache.thrift.scheme.StandardScheme.class.equals(proto.getScheme()) ? STANDARD_SCHEME_FACTORY : TUPLE_SCHEME_FACTORY).getScheme();
-    }
-  }
-
-  public static class startScan_result implements org.apache.thrift.TBase<startScan_result, startScan_result._Fields>, java.io.Serializable, Cloneable, Comparable<startScan_result>   {
-    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("startScan_result");
-
-    private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.STRUCT, (short)0);
-    private static final org.apache.thrift.protocol.TField SEC_FIELD_DESC = new org.apache.thrift.protocol.TField("sec", org.apache.thrift.protocol.TType.STRUCT, (short)1);
-    private static final org.apache.thrift.protocol.TField NSTE_FIELD_DESC = new org.apache.thrift.protocol.TField("nste", org.apache.thrift.protocol.TType.STRUCT, (short)2);
-    private static final org.apache.thrift.protocol.TField TMFE_FIELD_DESC = new org.apache.thrift.protocol.TField("tmfe", org.apache.thrift.protocol.TType.STRUCT, (short)3);
-    private static final org.apache.thrift.protocol.TField TSNPE_FIELD_DESC = new org.apache.thrift.protocol.TField("tsnpe", org.apache.thrift.protocol.TType.STRUCT, (short)4);
-
-    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new startScan_resultStandardSchemeFactory();
-    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new startScan_resultTupleSchemeFactory();
-
-    public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.InitialScan success; // required
-    public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException sec; // required
-    public @org.apache.thrift.annotation.Nullable NotServingTabletException nste; // required
-    public @org.apache.thrift.annotation.Nullable TooManyFilesException tmfe; // required
-    public @org.apache.thrift.annotation.Nullable TSampleNotPresentException tsnpe; // required
-
-    /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
-    public enum _Fields implements org.apache.thrift.TFieldIdEnum {
-      SUCCESS((short)0, "success"),
-      SEC((short)1, "sec"),
-      NSTE((short)2, "nste"),
-      TMFE((short)3, "tmfe"),
-      TSNPE((short)4, "tsnpe");
-
-      private static final java.util.Map<java.lang.String, _Fields> byName = new java.util.HashMap<java.lang.String, _Fields>();
-
-      static {
-        for (_Fields field : java.util.EnumSet.allOf(_Fields.class)) {
-          byName.put(field.getFieldName(), field);
-        }
-      }
-
-      /**
-       * Find the _Fields constant that matches fieldId, or null if its not found.
-       */
-      @org.apache.thrift.annotation.Nullable
-      public static _Fields findByThriftId(int fieldId) {
-        switch(fieldId) {
-          case 0: // SUCCESS
-            return SUCCESS;
-          case 1: // SEC
-            return SEC;
-          case 2: // NSTE
-            return NSTE;
-          case 3: // TMFE
-            return TMFE;
-          case 4: // TSNPE
-            return TSNPE;
-          default:
-            return null;
-        }
-      }
-
-      /**
-       * Find the _Fields constant that matches fieldId, throwing an exception
-       * if it is not found.
-       */
-      public static _Fields findByThriftIdOrThrow(int fieldId) {
-        _Fields fields = findByThriftId(fieldId);
-        if (fields == null) throw new java.lang.IllegalArgumentException("Field " + fieldId + " doesn't exist!");
-        return fields;
-      }
-
-      /**
-       * Find the _Fields constant that matches name, or null if its not found.
-       */
-      @org.apache.thrift.annotation.Nullable
-      public static _Fields findByName(java.lang.String name) {
-        return byName.get(name);
-      }
-
-      private final short _thriftId;
-      private final java.lang.String _fieldName;
-
-      _Fields(short thriftId, java.lang.String fieldName) {
-        _thriftId = thriftId;
-        _fieldName = fieldName;
-      }
-
-      public short getThriftFieldId() {
-        return _thriftId;
-      }
-
-      public java.lang.String getFieldName() {
-        return _fieldName;
-      }
-    }
-
-    // isset id assignments
-    public static final java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
-    static {
-      java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
-      tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, 
-          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.dataImpl.thrift.InitialScan.class)));
-      tmpMap.put(_Fields.SEC, new org.apache.thrift.meta_data.FieldMetaData("sec", org.apache.thrift.TFieldRequirementType.DEFAULT, 
-          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException.class)));
-      tmpMap.put(_Fields.NSTE, new org.apache.thrift.meta_data.FieldMetaData("nste", org.apache.thrift.TFieldRequirementType.DEFAULT, 
-          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, NotServingTabletException.class)));
-      tmpMap.put(_Fields.TMFE, new org.apache.thrift.meta_data.FieldMetaData("tmfe", org.apache.thrift.TFieldRequirementType.DEFAULT, 
-          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TooManyFilesException.class)));
-      tmpMap.put(_Fields.TSNPE, new org.apache.thrift.meta_data.FieldMetaData("tsnpe", org.apache.thrift.TFieldRequirementType.DEFAULT, 
-          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TSampleNotPresentException.class)));
-      metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
-      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(startScan_result.class, metaDataMap);
-    }
-
-    public startScan_result() {
-    }
-
-    public startScan_result(
-      org.apache.accumulo.core.dataImpl.thrift.InitialScan success,
-      org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException sec,
-      NotServingTabletException nste,
-      TooManyFilesException tmfe,
-      TSampleNotPresentException tsnpe)
-    {
-      this();
-      this.success = success;
-      this.sec = sec;
-      this.nste = nste;
-      this.tmfe = tmfe;
-      this.tsnpe = tsnpe;
-    }
-
-    /**
-     * Performs a deep copy on <i>other</i>.
-     */
-    public startScan_result(startScan_result other) {
-      if (other.isSetSuccess()) {
-        this.success = new org.apache.accumulo.core.dataImpl.thrift.InitialScan(other.success);
-      }
-      if (other.isSetSec()) {
-        this.sec = new org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException(other.sec);
-      }
-      if (other.isSetNste()) {
-        this.nste = new NotServingTabletException(other.nste);
-      }
-      if (other.isSetTmfe()) {
-        this.tmfe = new TooManyFilesException(other.tmfe);
-      }
-      if (other.isSetTsnpe()) {
-        this.tsnpe = new TSampleNotPresentException(other.tsnpe);
-      }
-    }
-
-    public startScan_result deepCopy() {
-      return new startScan_result(this);
-    }
-
-    @Override
-    public void clear() {
-      this.success = null;
-      this.sec = null;
-      this.nste = null;
-      this.tmfe = null;
-      this.tsnpe = null;
-    }
-
-    @org.apache.thrift.annotation.Nullable
-    public org.apache.accumulo.core.dataImpl.thrift.InitialScan getSuccess() {
-      return this.success;
-    }
-
-    public startScan_result setSuccess(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.InitialScan success) {
-      this.success = success;
-      return this;
-    }
-
-    public void unsetSuccess() {
-      this.success = null;
-    }
-
-    /** Returns true if field success is set (has been assigned a value) and false otherwise */
-    public boolean isSetSuccess() {
-      return this.success != null;
-    }
-
-    public void setSuccessIsSet(boolean value) {
-      if (!value) {
-        this.success = null;
-      }
-    }
-
-    @org.apache.thrift.annotation.Nullable
-    public org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException getSec() {
-      return this.sec;
-    }
-
-    public startScan_result setSec(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException sec) {
-      this.sec = sec;
-      return this;
-    }
-
-    public void unsetSec() {
-      this.sec = null;
-    }
-
-    /** Returns true if field sec is set (has been assigned a value) and false otherwise */
-    public boolean isSetSec() {
-      return this.sec != null;
-    }
-
-    public void setSecIsSet(boolean value) {
-      if (!value) {
-        this.sec = null;
-      }
-    }
-
-    @org.apache.thrift.annotation.Nullable
-    public NotServingTabletException getNste() {
-      return this.nste;
-    }
-
-    public startScan_result setNste(@org.apache.thrift.annotation.Nullable NotServingTabletException nste) {
-      this.nste = nste;
-      return this;
-    }
-
-    public void unsetNste() {
-      this.nste = null;
-    }
-
-    /** Returns true if field nste is set (has been assigned a value) and false otherwise */
-    public boolean isSetNste() {
-      return this.nste != null;
-    }
-
-    public void setNsteIsSet(boolean value) {
-      if (!value) {
-        this.nste = null;
-      }
-    }
-
-    @org.apache.thrift.annotation.Nullable
-    public TooManyFilesException getTmfe() {
-      return this.tmfe;
-    }
-
-    public startScan_result setTmfe(@org.apache.thrift.annotation.Nullable TooManyFilesException tmfe) {
-      this.tmfe = tmfe;
-      return this;
-    }
-
-    public void unsetTmfe() {
-      this.tmfe = null;
-    }
-
-    /** Returns true if field tmfe is set (has been assigned a value) and false otherwise */
-    public boolean isSetTmfe() {
-      return this.tmfe != null;
-    }
-
-    public void setTmfeIsSet(boolean value) {
-      if (!value) {
-        this.tmfe = null;
-      }
-    }
-
-    @org.apache.thrift.annotation.Nullable
-    public TSampleNotPresentException getTsnpe() {
-      return this.tsnpe;
-    }
-
-    public startScan_result setTsnpe(@org.apache.thrift.annotation.Nullable TSampleNotPresentException tsnpe) {
-      this.tsnpe = tsnpe;
-      return this;
-    }
-
-    public void unsetTsnpe() {
-      this.tsnpe = null;
-    }
-
-    /** Returns true if field tsnpe is set (has been assigned a value) and false otherwise */
-    public boolean isSetTsnpe() {
-      return this.tsnpe != null;
-    }
-
-    public void setTsnpeIsSet(boolean value) {
-      if (!value) {
-        this.tsnpe = null;
-      }
-    }
-
-    public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable java.lang.Object value) {
-      switch (field) {
-      case SUCCESS:
-        if (value == null) {
-          unsetSuccess();
-        } else {
-          setSuccess((org.apache.accumulo.core.dataImpl.thrift.InitialScan)value);
-        }
-        break;
-
-      case SEC:
-        if (value == null) {
-          unsetSec();
-        } else {
-          setSec((org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException)value);
-        }
-        break;
-
-      case NSTE:
-        if (value == null) {
-          unsetNste();
-        } else {
-          setNste((NotServingTabletException)value);
-        }
-        break;
-
-      case TMFE:
-        if (value == null) {
-          unsetTmfe();
-        } else {
-          setTmfe((TooManyFilesException)value);
-        }
-        break;
-
-      case TSNPE:
-        if (value == null) {
-          unsetTsnpe();
-        } else {
-          setTsnpe((TSampleNotPresentException)value);
-        }
-        break;
-
-      }
-    }
-
-    @org.apache.thrift.annotation.Nullable
-    public java.lang.Object getFieldValue(_Fields field) {
-      switch (field) {
-      case SUCCESS:
-        return getSuccess();
-
-      case SEC:
-        return getSec();
-
-      case NSTE:
-        return getNste();
-
-      case TMFE:
-        return getTmfe();
-
-      case TSNPE:
-        return getTsnpe();
-
-      }
-      throw new java.lang.IllegalStateException();
-    }
-
-    /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
-    public boolean isSet(_Fields field) {
-      if (field == null) {
-        throw new java.lang.IllegalArgumentException();
-      }
-
-      switch (field) {
-      case SUCCESS:
-        return isSetSuccess();
-      case SEC:
-        return isSetSec();
-      case NSTE:
-        return isSetNste();
-      case TMFE:
-        return isSetTmfe();
-      case TSNPE:
-        return isSetTsnpe();
-      }
-      throw new java.lang.IllegalStateException();
-    }
-
-    @Override
-    public boolean equals(java.lang.Object that) {
-      if (that instanceof startScan_result)
-        return this.equals((startScan_result)that);
-      return false;
-    }
-
-    public boolean equals(startScan_result that) {
-      if (that == null)
-        return false;
-      if (this == that)
-        return true;
-
-      boolean this_present_success = true && this.isSetSuccess();
-      boolean that_present_success = true && that.isSetSuccess();
-      if (this_present_success || that_present_success) {
-        if (!(this_present_success && that_present_success))
-          return false;
-        if (!this.success.equals(that.success))
-          return false;
-      }
-
-      boolean this_present_sec = true && this.isSetSec();
-      boolean that_present_sec = true && that.isSetSec();
-      if (this_present_sec || that_present_sec) {
-        if (!(this_present_sec && that_present_sec))
-          return false;
-        if (!this.sec.equals(that.sec))
-          return false;
-      }
-
-      boolean this_present_nste = true && this.isSetNste();
-      boolean that_present_nste = true && that.isSetNste();
-      if (this_present_nste || that_present_nste) {
-        if (!(this_present_nste && that_present_nste))
-          return false;
-        if (!this.nste.equals(that.nste))
-          return false;
-      }
-
-      boolean this_present_tmfe = true && this.isSetTmfe();
-      boolean that_present_tmfe = true && that.isSetTmfe();
-      if (this_present_tmfe || that_present_tmfe) {
-        if (!(this_present_tmfe && that_present_tmfe))
-          return false;
-        if (!this.tmfe.equals(that.tmfe))
-          return false;
-      }
-
-      boolean this_present_tsnpe = true && this.isSetTsnpe();
-      boolean that_present_tsnpe = true && that.isSetTsnpe();
-      if (this_present_tsnpe || that_present_tsnpe) {
-        if (!(this_present_tsnpe && that_present_tsnpe))
-          return false;
-        if (!this.tsnpe.equals(that.tsnpe))
-          return false;
-      }
-
-      return true;
-    }
-
-    @Override
-    public int hashCode() {
-      int hashCode = 1;
-
-      hashCode = hashCode * 8191 + ((isSetSuccess()) ? 131071 : 524287);
-      if (isSetSuccess())
-        hashCode = hashCode * 8191 + success.hashCode();
-
-      hashCode = hashCode * 8191 + ((isSetSec()) ? 131071 : 524287);
-      if (isSetSec())
-        hashCode = hashCode * 8191 + sec.hashCode();
-
-      hashCode = hashCode * 8191 + ((isSetNste()) ? 131071 : 524287);
-      if (isSetNste())
-        hashCode = hashCode * 8191 + nste.hashCode();
-
-      hashCode = hashCode * 8191 + ((isSetTmfe()) ? 131071 : 524287);
-      if (isSetTmfe())
-        hashCode = hashCode * 8191 + tmfe.hashCode();
-
-      hashCode = hashCode * 8191 + ((isSetTsnpe()) ? 131071 : 524287);
-      if (isSetTsnpe())
-        hashCode = hashCode * 8191 + tsnpe.hashCode();
-
-      return hashCode;
-    }
-
-    @Override
-    public int compareTo(startScan_result other) {
-      if (!getClass().equals(other.getClass())) {
-        return getClass().getName().compareTo(other.getClass().getName());
-      }
-
-      int lastComparison = 0;
-
-      lastComparison = java.lang.Boolean.compare(isSetSuccess(), other.isSetSuccess());
-      if (lastComparison != 0) {
-        return lastComparison;
-      }
-      if (isSetSuccess()) {
-        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, other.success);
-        if (lastComparison != 0) {
-          return lastComparison;
-        }
-      }
-      lastComparison = java.lang.Boolean.compare(isSetSec(), other.isSetSec());
-      if (lastComparison != 0) {
-        return lastComparison;
-      }
-      if (isSetSec()) {
-        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.sec, other.sec);
-        if (lastComparison != 0) {
-          return lastComparison;
-        }
-      }
-      lastComparison = java.lang.Boolean.compare(isSetNste(), other.isSetNste());
-      if (lastComparison != 0) {
-        return lastComparison;
-      }
-      if (isSetNste()) {
-        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.nste, other.nste);
-        if (lastComparison != 0) {
-          return lastComparison;
-        }
-      }
-      lastComparison = java.lang.Boolean.compare(isSetTmfe(), other.isSetTmfe());
-      if (lastComparison != 0) {
-        return lastComparison;
-      }
-      if (isSetTmfe()) {
-        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tmfe, other.tmfe);
-        if (lastComparison != 0) {
-          return lastComparison;
-        }
-      }
-      lastComparison = java.lang.Boolean.compare(isSetTsnpe(), other.isSetTsnpe());
-      if (lastComparison != 0) {
-        return lastComparison;
-      }
-      if (isSetTsnpe()) {
-        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tsnpe, other.tsnpe);
-        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("startScan_result(");
-      boolean first = true;
-
-      sb.append("success:");
-      if (this.success == null) {
-        sb.append("null");
-      } else {
-        sb.append(this.success);
-      }
-      first = false;
-      if (!first) sb.append(", ");
-      sb.append("sec:");
-      if (this.sec == null) {
-        sb.append("null");
-      } else {
-        sb.append(this.sec);
-      }
-      first = false;
-      if (!first) sb.append(", ");
-      sb.append("nste:");
-      if (this.nste == null) {
-        sb.append("null");
-      } else {
-        sb.append(this.nste);
-      }
-      first = false;
-      if (!first) sb.append(", ");
-      sb.append("tmfe:");
-      if (this.tmfe == null) {
-        sb.append("null");
-      } else {
-        sb.append(this.tmfe);
-      }
-      first = false;
-      if (!first) sb.append(", ");
-      sb.append("tsnpe:");
-      if (this.tsnpe == null) {
-        sb.append("null");
-      } else {
-        sb.append(this.tsnpe);
-      }
-      first = false;
-      sb.append(")");
-      return sb.toString();
-    }
-
-    public void validate() throws org.apache.thrift.TException {
-      // check for required fields
-      // check for sub-struct validity
-      if (success != null) {
-        success.validate();
-      }
-    }
-
-    private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
-      try {
-        write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
-      } catch (org.apache.thrift.TException te) {
-        throw new java.io.IOException(te);
-      }
-    }
-
-    private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, java.lang.ClassNotFoundException {
-      try {
-        read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
-      } catch (org.apache.thrift.TException te) {
-        throw new java.io.IOException(te);
-      }
-    }
-
-    private static class startScan_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      public startScan_resultStandardScheme getScheme() {
-        return new startScan_resultStandardScheme();
-      }
-    }
-
-    private static class startScan_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme<startScan_result> {
-
-      public void read(org.apache.thrift.protocol.TProtocol iprot, startScan_result struct) throws org.apache.thrift.TException {
-        org.apache.thrift.protocol.TField schemeField;
-        iprot.readStructBegin();
-        while (true)
-        {
-          schemeField = iprot.readFieldBegin();
-          if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
-            break;
-          }
-          switch (schemeField.id) {
-            case 0: // SUCCESS
-              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
-                struct.success = new org.apache.accumulo.core.dataImpl.thrift.InitialScan();
-                struct.success.read(iprot);
-                struct.setSuccessIsSet(true);
-              } else { 
-                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
-              }
-              break;
-            case 1: // SEC
-              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
-                struct.sec = new org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException();
-                struct.sec.read(iprot);
-                struct.setSecIsSet(true);
-              } else { 
-                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
-              }
-              break;
-            case 2: // NSTE
-              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
-                struct.nste = new NotServingTabletException();
-                struct.nste.read(iprot);
-                struct.setNsteIsSet(true);
-              } else { 
-                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
-              }
-              break;
-            case 3: // TMFE
-              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
-                struct.tmfe = new TooManyFilesException();
-                struct.tmfe.read(iprot);
-                struct.setTmfeIsSet(true);
-              } else { 
-                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
-              }
-              break;
-            case 4: // TSNPE
-              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
-                struct.tsnpe = new TSampleNotPresentException();
-                struct.tsnpe.read(iprot);
-                struct.setTsnpeIsSet(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, startScan_result struct) throws org.apache.thrift.TException {
-        struct.validate();
-
-        oprot.writeStructBegin(STRUCT_DESC);
-        if (struct.success != null) {
-          oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
-          struct.success.write(oprot);
-          oprot.writeFieldEnd();
-        }
-        if (struct.sec != null) {
-          oprot.writeFieldBegin(SEC_FIELD_DESC);
-          struct.sec.write(oprot);
-          oprot.writeFieldEnd();
-        }
-        if (struct.nste != null) {
-          oprot.writeFieldBegin(NSTE_FIELD_DESC);
-          struct.nste.write(oprot);
-          oprot.writeFieldEnd();
-        }
-        if (struct.tmfe != null) {
-          oprot.writeFieldBegin(TMFE_FIELD_DESC);
-          struct.tmfe.write(oprot);
-          oprot.writeFieldEnd();
-        }
-        if (struct.tsnpe != null) {
-          oprot.writeFieldBegin(TSNPE_FIELD_DESC);
-          struct.tsnpe.write(oprot);
-          oprot.writeFieldEnd();
-        }
-        oprot.writeFieldStop();
-        oprot.writeStructEnd();
-      }
-
-    }
-
-    private static class startScan_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      public startScan_resultTupleScheme getScheme() {
-        return new startScan_resultTupleScheme();
-      }
-    }
-
-    private static class startScan_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme<startScan_result> {
-
-      @Override
-      public void write(org.apache.thrift.protocol.TProtocol prot, startScan_result struct) throws org.apache.thrift.TException {
-        org.apache.thrift.protocol.TTupleProtocol oprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
-        java.util.BitSet optionals = new java.util.BitSet();
-        if (struct.isSetSuccess()) {
-          optionals.set(0);
-        }
-        if (struct.isSetSec()) {
-          optionals.set(1);
-        }
-        if (struct.isSetNste()) {
-          optionals.set(2);
-        }
-        if (struct.isSetTmfe()) {
-          optionals.set(3);
-        }
-        if (struct.isSetTsnpe()) {
-          optionals.set(4);
-        }
-        oprot.writeBitSet(optionals, 5);
-        if (struct.isSetSuccess()) {
-          struct.success.write(oprot);
-        }
-        if (struct.isSetSec()) {
-          struct.sec.write(oprot);
-        }
-        if (struct.isSetNste()) {
-          struct.nste.write(oprot);
-        }
-        if (struct.isSetTmfe()) {
-          struct.tmfe.write(oprot);
-        }
-        if (struct.isSetTsnpe()) {
-          struct.tsnpe.write(oprot);
-        }
-      }
-
-      @Override
-      public void read(org.apache.thrift.protocol.TProtocol prot, startScan_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(5);
-        if (incoming.get(0)) {
-          struct.success = new org.apache.accumulo.core.dataImpl.thrift.InitialScan();
-          struct.success.read(iprot);
-          struct.setSuccessIsSet(true);
-        }
-        if (incoming.get(1)) {
-          struct.sec = new org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException();
-          struct.sec.read(iprot);
-          struct.setSecIsSet(true);
-        }
-        if (incoming.get(2)) {
-          struct.nste = new NotServingTabletException();
-          struct.nste.read(iprot);
-          struct.setNsteIsSet(true);
-        }
-        if (incoming.get(3)) {
-          struct.tmfe = new TooManyFilesException();
-          struct.tmfe.read(iprot);
-          struct.setTmfeIsSet(true);
-        }
-        if (incoming.get(4)) {
-          struct.tsnpe = new TSampleNotPresentException();
-          struct.tsnpe.read(iprot);
-          struct.setTsnpeIsSet(true);
-        }
-      }
-    }
-
-    private static <S extends org.apache.thrift.scheme.IScheme> S scheme(org.apache.thrift.protocol.TProtocol proto) {
-      return (org.apache.thrift.scheme.StandardScheme.class.equals(proto.getScheme()) ? STANDARD_SCHEME_FACTORY : TUPLE_SCHEME_FACTORY).getScheme();
-    }
-  }
-
-  public static class continueScan_args implements org.apache.thrift.TBase<continueScan_args, continueScan_args._Fields>, java.io.Serializable, Cloneable, Comparable<continueScan_args>   {
-    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("continueScan_args");
-
-    private static final org.apache.thrift.protocol.TField TINFO_FIELD_DESC = new org.apache.thrift.protocol.TField("tinfo", org.apache.thrift.protocol.TType.STRUCT, (short)2);
-    private static final org.apache.thrift.protocol.TField SCAN_ID_FIELD_DESC = new org.apache.thrift.protocol.TField("scanID", org.apache.thrift.protocol.TType.I64, (short)1);
-
-    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new continueScan_argsStandardSchemeFactory();
-    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new continueScan_argsTupleSchemeFactory();
-
-    public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo; // required
-    public long scanID; // required
-
-    /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
-    public enum _Fields implements org.apache.thrift.TFieldIdEnum {
-      TINFO((short)2, "tinfo"),
-      SCAN_ID((short)1, "scanID");
-
-      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 2: // TINFO
-            return TINFO;
-          case 1: // SCAN_ID
-            return SCAN_ID;
-          default:
-            return null;
-        }
-      }
-
-      /**
-       * Find the _Fields constant that matches fieldId, throwing an exception
-       * if it is not found.
-       */
-      public static _Fields findByThriftIdOrThrow(int fieldId) {
-        _Fields fields = findByThriftId(fieldId);
-        if (fields == null) throw new java.lang.IllegalArgumentException("Field " + fieldId + " doesn't exist!");
-        return fields;
-      }
-
-      /**
-       * Find the _Fields constant that matches name, or null if its not found.
-       */
-      @org.apache.thrift.annotation.Nullable
-      public static _Fields findByName(java.lang.String name) {
-        return byName.get(name);
-      }
-
-      private final short _thriftId;
-      private final java.lang.String _fieldName;
-
-      _Fields(short thriftId, java.lang.String fieldName) {
-        _thriftId = thriftId;
-        _fieldName = fieldName;
-      }
-
-      public short getThriftFieldId() {
-        return _thriftId;
-      }
-
-      public java.lang.String getFieldName() {
-        return _fieldName;
-      }
-    }
-
-    // isset id assignments
-    private static final int __SCANID_ISSET_ID = 0;
-    private byte __isset_bitfield = 0;
-    public static final java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
-    static {
-      java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
-      tmpMap.put(_Fields.TINFO, new org.apache.thrift.meta_data.FieldMetaData("tinfo", org.apache.thrift.TFieldRequirementType.DEFAULT, 
-          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.trace.thrift.TInfo.class)));
-      tmpMap.put(_Fields.SCAN_ID, new org.apache.thrift.meta_data.FieldMetaData("scanID", org.apache.thrift.TFieldRequirementType.DEFAULT, 
-          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64          , "ScanID")));
-      metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
-      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(continueScan_args.class, metaDataMap);
-    }
-
-    public continueScan_args() {
-    }
-
-    public continueScan_args(
-      org.apache.accumulo.core.trace.thrift.TInfo tinfo,
-      long scanID)
-    {
-      this();
-      this.tinfo = tinfo;
-      this.scanID = scanID;
-      setScanIDIsSet(true);
-    }
-
-    /**
-     * Performs a deep copy on <i>other</i>.
-     */
-    public continueScan_args(continueScan_args other) {
-      __isset_bitfield = other.__isset_bitfield;
-      if (other.isSetTinfo()) {
-        this.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo(other.tinfo);
-      }
-      this.scanID = other.scanID;
-    }
-
-    public continueScan_args deepCopy() {
-      return new continueScan_args(this);
-    }
-
-    @Override
-    public void clear() {
-      this.tinfo = null;
-      setScanIDIsSet(false);
-      this.scanID = 0;
-    }
-
-    @org.apache.thrift.annotation.Nullable
-    public org.apache.accumulo.core.trace.thrift.TInfo getTinfo() {
-      return this.tinfo;
-    }
-
-    public continueScan_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo) {
-      this.tinfo = tinfo;
-      return this;
-    }
-
-    public void unsetTinfo() {
-      this.tinfo = null;
-    }
-
-    /** Returns true if field tinfo is set (has been assigned a value) and false otherwise */
-    public boolean isSetTinfo() {
-      return this.tinfo != null;
-    }
-
-    public void setTinfoIsSet(boolean value) {
-      if (!value) {
-        this.tinfo = null;
-      }
-    }
-
-    public long getScanID() {
-      return this.scanID;
-    }
-
-    public continueScan_args setScanID(long scanID) {
-      this.scanID = scanID;
-      setScanIDIsSet(true);
-      return this;
-    }
-
-    public void unsetScanID() {
-      __isset_bitfield = org.apache.thrift.EncodingUtils.clearBit(__isset_bitfield, __SCANID_ISSET_ID);
-    }
-
-    /** Returns true if field scanID is set (has been assigned a value) and false otherwise */
-    public boolean isSetScanID() {
-      return org.apache.thrift.EncodingUtils.testBit(__isset_bitfield, __SCANID_ISSET_ID);
-    }
-
-    public void setScanIDIsSet(boolean value) {
-      __isset_bitfield = org.apache.thrift.EncodingUtils.setBit(__isset_bitfield, __SCANID_ISSET_ID, value);
-    }
-
-    public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable java.lang.Object value) {
-      switch (field) {
-      case TINFO:
-        if (value == null) {
-          unsetTinfo();
-        } else {
-          setTinfo((org.apache.accumulo.core.trace.thrift.TInfo)value);
-        }
-        break;
-
-      case SCAN_ID:
-        if (value == null) {
-          unsetScanID();
-        } else {
-          setScanID((java.lang.Long)value);
-        }
-        break;
-
-      }
-    }
-
-    @org.apache.thrift.annotation.Nullable
-    public java.lang.Object getFieldValue(_Fields field) {
-      switch (field) {
-      case TINFO:
-        return getTinfo();
-
-      case SCAN_ID:
-        return getScanID();
-
-      }
-      throw new java.lang.IllegalStateException();
-    }
-
-    /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
-    public boolean isSet(_Fields field) {
-      if (field == null) {
-        throw new java.lang.IllegalArgumentException();
-      }
-
-      switch (field) {
-      case TINFO:
-        return isSetTinfo();
-      case SCAN_ID:
-        return isSetScanID();
-      }
-      throw new java.lang.IllegalStateException();
-    }
-
-    @Override
-    public boolean equals(java.lang.Object that) {
-      if (that instanceof continueScan_args)
-        return this.equals((continueScan_args)that);
-      return false;
-    }
-
-    public boolean equals(continueScan_args that) {
-      if (that == null)
-        return false;
-      if (this == that)
-        return true;
-
-      boolean this_present_tinfo = true && this.isSetTinfo();
-      boolean that_present_tinfo = true && that.isSetTinfo();
-      if (this_present_tinfo || that_present_tinfo) {
-        if (!(this_present_tinfo && that_present_tinfo))
-          return false;
-        if (!this.tinfo.equals(that.tinfo))
-          return false;
-      }
-
-      boolean this_present_scanID = true;
-      boolean that_present_scanID = true;
-      if (this_present_scanID || that_present_scanID) {
-        if (!(this_present_scanID && that_present_scanID))
-          return false;
-        if (this.scanID != that.scanID)
-          return false;
-      }
-
-      return true;
-    }
-
-    @Override
-    public int hashCode() {
-      int hashCode = 1;
-
-      hashCode = hashCode * 8191 + ((isSetTinfo()) ? 131071 : 524287);
-      if (isSetTinfo())
-        hashCode = hashCode * 8191 + tinfo.hashCode();
-
-      hashCode = hashCode * 8191 + org.apache.thrift.TBaseHelper.hashCode(scanID);
-
-      return hashCode;
-    }
-
-    @Override
-    public int compareTo(continueScan_args other) {
-      if (!getClass().equals(other.getClass())) {
-        return getClass().getName().compareTo(other.getClass().getName());
-      }
-
-      int lastComparison = 0;
-
-      lastComparison = java.lang.Boolean.compare(isSetTinfo(), other.isSetTinfo());
-      if (lastComparison != 0) {
-        return lastComparison;
-      }
-      if (isSetTinfo()) {
-        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tinfo, other.tinfo);
-        if (lastComparison != 0) {
-          return lastComparison;
-        }
-      }
-      lastComparison = java.lang.Boolean.compare(isSetScanID(), other.isSetScanID());
-      if (lastComparison != 0) {
-        return lastComparison;
-      }
-      if (isSetScanID()) {
-        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.scanID, other.scanID);
-        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("continueScan_args(");
-      boolean first = true;
-
-      sb.append("tinfo:");
-      if (this.tinfo == null) {
-        sb.append("null");
-      } else {
-        sb.append(this.tinfo);
-      }
-      first = false;
-      if (!first) sb.append(", ");
-      sb.append("scanID:");
-      sb.append(this.scanID);
-      first = false;
-      sb.append(")");
-      return sb.toString();
-    }
-
-    public void validate() throws org.apache.thrift.TException {
-      // check for required fields
-      // check for sub-struct validity
-      if (tinfo != null) {
-        tinfo.validate();
-      }
-    }
-
-    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 continueScan_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      public continueScan_argsStandardScheme getScheme() {
-        return new continueScan_argsStandardScheme();
-      }
-    }
-
-    private static class continueScan_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme<continueScan_args> {
-
-      public void read(org.apache.thrift.protocol.TProtocol iprot, continueScan_args struct) throws org.apache.thrift.TException {
-        org.apache.thrift.protocol.TField schemeField;
-        iprot.readStructBegin();
-        while (true)
-        {
-          schemeField = iprot.readFieldBegin();
-          if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
-            break;
-          }
-          switch (schemeField.id) {
-            case 2: // TINFO
-              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
-                struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
-                struct.tinfo.read(iprot);
-                struct.setTinfoIsSet(true);
-              } else { 
-                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
-              }
-              break;
-            case 1: // SCAN_ID
-              if (schemeField.type == org.apache.thrift.protocol.TType.I64) {
-                struct.scanID = iprot.readI64();
-                struct.setScanIDIsSet(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, continueScan_args struct) throws org.apache.thrift.TException {
-        struct.validate();
-
-        oprot.writeStructBegin(STRUCT_DESC);
-        oprot.writeFieldBegin(SCAN_ID_FIELD_DESC);
-        oprot.writeI64(struct.scanID);
-        oprot.writeFieldEnd();
-        if (struct.tinfo != null) {
-          oprot.writeFieldBegin(TINFO_FIELD_DESC);
-          struct.tinfo.write(oprot);
-          oprot.writeFieldEnd();
-        }
-        oprot.writeFieldStop();
-        oprot.writeStructEnd();
-      }
-
-    }
-
-    private static class continueScan_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      public continueScan_argsTupleScheme getScheme() {
-        return new continueScan_argsTupleScheme();
-      }
-    }
-
-    private static class continueScan_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme<continueScan_args> {
-
-      @Override
-      public void write(org.apache.thrift.protocol.TProtocol prot, continueScan_args struct) throws org.apache.thrift.TException {
-        org.apache.thrift.protocol.TTupleProtocol oprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
-        java.util.BitSet optionals = new java.util.BitSet();
-        if (struct.isSetTinfo()) {
-          optionals.set(0);
-        }
-        if (struct.isSetScanID()) {
-          optionals.set(1);
-        }
-        oprot.writeBitSet(optionals, 2);
-        if (struct.isSetTinfo()) {
-          struct.tinfo.write(oprot);
-        }
-        if (struct.isSetScanID()) {
-          oprot.writeI64(struct.scanID);
-        }
-      }
-
-      @Override
-      public void read(org.apache.thrift.protocol.TProtocol prot, continueScan_args struct) throws org.apache.thrift.TException {
-        org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
-        java.util.BitSet incoming = iprot.readBitSet(2);
-        if (incoming.get(0)) {
-          struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
-          struct.tinfo.read(iprot);
-          struct.setTinfoIsSet(true);
-        }
-        if (incoming.get(1)) {
-          struct.scanID = iprot.readI64();
-          struct.setScanIDIsSet(true);
-        }
-      }
-    }
-
-    private static <S extends org.apache.thrift.scheme.IScheme> S scheme(org.apache.thrift.protocol.TProtocol proto) {
-      return (org.apache.thrift.scheme.StandardScheme.class.equals(proto.getScheme()) ? STANDARD_SCHEME_FACTORY : TUPLE_SCHEME_FACTORY).getScheme();
-    }
-  }
-
-  public static class continueScan_result implements org.apache.thrift.TBase<continueScan_result, continueScan_result._Fields>, java.io.Serializable, Cloneable, Comparable<continueScan_result>   {
-    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("continueScan_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 NSSI_FIELD_DESC = new org.apache.thrift.protocol.TField("nssi", org.apache.thrift.protocol.TType.STRUCT, (short)1);
-    private static final org.apache.thrift.protocol.TField NSTE_FIELD_DESC = new org.apache.thrift.protocol.TField("nste", org.apache.thrift.protocol.TType.STRUCT, (short)2);
-    private static final org.apache.thrift.protocol.TField TMFE_FIELD_DESC = new org.apache.thrift.protocol.TField("tmfe", org.apache.thrift.protocol.TType.STRUCT, (short)3);
-    private static final org.apache.thrift.protocol.TField TSNPE_FIELD_DESC = new org.apache.thrift.protocol.TField("tsnpe", org.apache.thrift.protocol.TType.STRUCT, (short)4);
-
-    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new continueScan_resultStandardSchemeFactory();
-    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new continueScan_resultTupleSchemeFactory();
-
-    public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.ScanResult success; // required
-    public @org.apache.thrift.annotation.Nullable NoSuchScanIDException nssi; // required
-    public @org.apache.thrift.annotation.Nullable NotServingTabletException nste; // required
-    public @org.apache.thrift.annotation.Nullable TooManyFilesException tmfe; // required
-    public @org.apache.thrift.annotation.Nullable TSampleNotPresentException tsnpe; // 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"),
-      NSSI((short)1, "nssi"),
-      NSTE((short)2, "nste"),
-      TMFE((short)3, "tmfe"),
-      TSNPE((short)4, "tsnpe");
-
-      private static final java.util.Map<java.lang.String, _Fields> byName = new java.util.HashMap<java.lang.String, _Fields>();
-
-      static {
-        for (_Fields field : java.util.EnumSet.allOf(_Fields.class)) {
-          byName.put(field.getFieldName(), field);
-        }
-      }
-
-      /**
-       * Find the _Fields constant that matches fieldId, or null if its not found.
-       */
-      @org.apache.thrift.annotation.Nullable
-      public static _Fields findByThriftId(int fieldId) {
-        switch(fieldId) {
-          case 0: // SUCCESS
-            return SUCCESS;
-          case 1: // NSSI
-            return NSSI;
-          case 2: // NSTE
-            return NSTE;
-          case 3: // TMFE
-            return TMFE;
-          case 4: // TSNPE
-            return TSNPE;
-          default:
-            return null;
-        }
-      }
-
-      /**
-       * Find the _Fields constant that matches fieldId, throwing an exception
-       * if it is not found.
-       */
-      public static _Fields findByThriftIdOrThrow(int fieldId) {
-        _Fields fields = findByThriftId(fieldId);
-        if (fields == null) throw new java.lang.IllegalArgumentException("Field " + fieldId + " doesn't exist!");
-        return fields;
-      }
-
-      /**
-       * Find the _Fields constant that matches name, or null if its not found.
-       */
-      @org.apache.thrift.annotation.Nullable
-      public static _Fields findByName(java.lang.String name) {
-        return byName.get(name);
-      }
-
-      private final short _thriftId;
-      private final java.lang.String _fieldName;
-
-      _Fields(short thriftId, java.lang.String fieldName) {
-        _thriftId = thriftId;
-        _fieldName = fieldName;
-      }
-
-      public short getThriftFieldId() {
-        return _thriftId;
-      }
-
-      public java.lang.String getFieldName() {
-        return _fieldName;
-      }
-    }
-
-    // isset id assignments
-    public static final java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
-    static {
-      java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
-      tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, 
-          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.dataImpl.thrift.ScanResult.class)));
-      tmpMap.put(_Fields.NSSI, new org.apache.thrift.meta_data.FieldMetaData("nssi", org.apache.thrift.TFieldRequirementType.DEFAULT, 
-          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, NoSuchScanIDException.class)));
-      tmpMap.put(_Fields.NSTE, new org.apache.thrift.meta_data.FieldMetaData("nste", org.apache.thrift.TFieldRequirementType.DEFAULT, 
-          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, NotServingTabletException.class)));
-      tmpMap.put(_Fields.TMFE, new org.apache.thrift.meta_data.FieldMetaData("tmfe", org.apache.thrift.TFieldRequirementType.DEFAULT, 
-          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TooManyFilesException.class)));
-      tmpMap.put(_Fields.TSNPE, new org.apache.thrift.meta_data.FieldMetaData("tsnpe", org.apache.thrift.TFieldRequirementType.DEFAULT, 
-          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TSampleNotPresentException.class)));
-      metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
-      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(continueScan_result.class, metaDataMap);
-    }
-
-    public continueScan_result() {
-    }
-
-    public continueScan_result(
-      org.apache.accumulo.core.dataImpl.thrift.ScanResult success,
-      NoSuchScanIDException nssi,
-      NotServingTabletException nste,
-      TooManyFilesException tmfe,
-      TSampleNotPresentException tsnpe)
-    {
-      this();
-      this.success = success;
-      this.nssi = nssi;
-      this.nste = nste;
-      this.tmfe = tmfe;
-      this.tsnpe = tsnpe;
-    }
-
-    /**
-     * Performs a deep copy on <i>other</i>.
-     */
-    public continueScan_result(continueScan_result other) {
-      if (other.isSetSuccess()) {
-        this.success = new org.apache.accumulo.core.dataImpl.thrift.ScanResult(other.success);
-      }
-      if (other.isSetNssi()) {
-        this.nssi = new NoSuchScanIDException(other.nssi);
-      }
-      if (other.isSetNste()) {
-        this.nste = new NotServingTabletException(other.nste);
-      }
-      if (other.isSetTmfe()) {
-        this.tmfe = new TooManyFilesException(other.tmfe);
-      }
-      if (other.isSetTsnpe()) {
-        this.tsnpe = new TSampleNotPresentException(other.tsnpe);
-      }
-    }
-
-    public continueScan_result deepCopy() {
-      return new continueScan_result(this);
-    }
-
-    @Override
-    public void clear() {
-      this.success = null;
-      this.nssi = null;
-      this.nste = null;
-      this.tmfe = null;
-      this.tsnpe = null;
-    }
-
-    @org.apache.thrift.annotation.Nullable
-    public org.apache.accumulo.core.dataImpl.thrift.ScanResult getSuccess() {
-      return this.success;
-    }
-
-    public continueScan_result setSuccess(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.ScanResult success) {
-      this.success = success;
-      return this;
-    }
-
-    public void unsetSuccess() {
-      this.success = null;
-    }
-
-    /** Returns true if field success is set (has been assigned a value) and false otherwise */
-    public boolean isSetSuccess() {
-      return this.success != null;
-    }
-
-    public void setSuccessIsSet(boolean value) {
-      if (!value) {
-        this.success = null;
-      }
-    }
-
-    @org.apache.thrift.annotation.Nullable
-    public NoSuchScanIDException getNssi() {
-      return this.nssi;
-    }
-
-    public continueScan_result setNssi(@org.apache.thrift.annotation.Nullable NoSuchScanIDException nssi) {
-      this.nssi = nssi;
-      return this;
-    }
-
-    public void unsetNssi() {
-      this.nssi = null;
-    }
-
-    /** Returns true if field nssi is set (has been assigned a value) and false otherwise */
-    public boolean isSetNssi() {
-      return this.nssi != null;
-    }
-
-    public void setNssiIsSet(boolean value) {
-      if (!value) {
-        this.nssi = null;
-      }
-    }
-
-    @org.apache.thrift.annotation.Nullable
-    public NotServingTabletException getNste() {
-      return this.nste;
-    }
-
-    public continueScan_result setNste(@org.apache.thrift.annotation.Nullable NotServingTabletException nste) {
-      this.nste = nste;
-      return this;
-    }
-
-    public void unsetNste() {
-      this.nste = null;
-    }
-
-    /** Returns true if field nste is set (has been assigned a value) and false otherwise */
-    public boolean isSetNste() {
-      return this.nste != null;
-    }
-
-    public void setNsteIsSet(boolean value) {
-      if (!value) {
-        this.nste = null;
-      }
-    }
-
-    @org.apache.thrift.annotation.Nullable
-    public TooManyFilesException getTmfe() {
-      return this.tmfe;
-    }
-
-    public continueScan_result setTmfe(@org.apache.thrift.annotation.Nullable TooManyFilesException tmfe) {
-      this.tmfe = tmfe;
-      return this;
-    }
-
-    public void unsetTmfe() {
-      this.tmfe = null;
-    }
-
-    /** Returns true if field tmfe is set (has been assigned a value) and false otherwise */
-    public boolean isSetTmfe() {
-      return this.tmfe != null;
-    }
-
-    public void setTmfeIsSet(boolean value) {
-      if (!value) {
-        this.tmfe = null;
-      }
-    }
-
-    @org.apache.thrift.annotation.Nullable
-    public TSampleNotPresentException getTsnpe() {
-      return this.tsnpe;
-    }
-
-    public continueScan_result setTsnpe(@org.apache.thrift.annotation.Nullable TSampleNotPresentException tsnpe) {
-      this.tsnpe = tsnpe;
-      return this;
-    }
-
-    public void unsetTsnpe() {
-      this.tsnpe = null;
-    }
-
-    /** Returns true if field tsnpe is set (has been assigned a value) and false otherwise */
-    public boolean isSetTsnpe() {
-      return this.tsnpe != null;
-    }
-
-    public void setTsnpeIsSet(boolean value) {
-      if (!value) {
-        this.tsnpe = null;
-      }
-    }
-
-    public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable java.lang.Object value) {
-      switch (field) {
-      case SUCCESS:
-        if (value == null) {
-          unsetSuccess();
-        } else {
-          setSuccess((org.apache.accumulo.core.dataImpl.thrift.ScanResult)value);
-        }
-        break;
-
-      case NSSI:
-        if (value == null) {
-          unsetNssi();
-        } else {
-          setNssi((NoSuchScanIDException)value);
-        }
-        break;
-
-      case NSTE:
-        if (value == null) {
-          unsetNste();
-        } else {
-          setNste((NotServingTabletException)value);
-        }
-        break;
-
-      case TMFE:
-        if (value == null) {
-          unsetTmfe();
-        } else {
-          setTmfe((TooManyFilesException)value);
-        }
-        break;
-
-      case TSNPE:
-        if (value == null) {
-          unsetTsnpe();
-        } else {
-          setTsnpe((TSampleNotPresentException)value);
-        }
-        break;
-
-      }
-    }
-
-    @org.apache.thrift.annotation.Nullable
-    public java.lang.Object getFieldValue(_Fields field) {
-      switch (field) {
-      case SUCCESS:
-        return getSuccess();
-
-      case NSSI:
-        return getNssi();
-
-      case NSTE:
-        return getNste();
-
-      case TMFE:
-        return getTmfe();
-
-      case TSNPE:
-        return getTsnpe();
-
-      }
-      throw new java.lang.IllegalStateException();
-    }
-
-    /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
-    public boolean isSet(_Fields field) {
-      if (field == null) {
-        throw new java.lang.IllegalArgumentException();
-      }
-
-      switch (field) {
-      case SUCCESS:
-        return isSetSuccess();
-      case NSSI:
-        return isSetNssi();
-      case NSTE:
-        return isSetNste();
-      case TMFE:
-        return isSetTmfe();
-      case TSNPE:
-        return isSetTsnpe();
-      }
-      throw new java.lang.IllegalStateException();
-    }
-
-    @Override
-    public boolean equals(java.lang.Object that) {
-      if (that instanceof continueScan_result)
-        return this.equals((continueScan_result)that);
-      return false;
-    }
-
-    public boolean equals(continueScan_result that) {
-      if (that == null)
-        return false;
-      if (this == that)
-        return true;
-
-      boolean this_present_success = true && this.isSetSuccess();
-      boolean that_present_success = true && that.isSetSuccess();
-      if (this_present_success || that_present_success) {
-        if (!(this_present_success && that_present_success))
-          return false;
-        if (!this.success.equals(that.success))
-          return false;
-      }
-
-      boolean this_present_nssi = true && this.isSetNssi();
-      boolean that_present_nssi = true && that.isSetNssi();
-      if (this_present_nssi || that_present_nssi) {
-        if (!(this_present_nssi && that_present_nssi))
-          return false;
-        if (!this.nssi.equals(that.nssi))
-          return false;
-      }
-
-      boolean this_present_nste = true && this.isSetNste();
-      boolean that_present_nste = true && that.isSetNste();
-      if (this_present_nste || that_present_nste) {
-        if (!(this_present_nste && that_present_nste))
-          return false;
-        if (!this.nste.equals(that.nste))
-          return false;
-      }
-
-      boolean this_present_tmfe = true && this.isSetTmfe();
-      boolean that_present_tmfe = true && that.isSetTmfe();
-      if (this_present_tmfe || that_present_tmfe) {
-        if (!(this_present_tmfe && that_present_tmfe))
-          return false;
-        if (!this.tmfe.equals(that.tmfe))
-          return false;
-      }
-
-      boolean this_present_tsnpe = true && this.isSetTsnpe();
-      boolean that_present_tsnpe = true && that.isSetTsnpe();
-      if (this_present_tsnpe || that_present_tsnpe) {
-        if (!(this_present_tsnpe && that_present_tsnpe))
-          return false;
-        if (!this.tsnpe.equals(that.tsnpe))
-          return false;
-      }
-
-      return true;
-    }
-
-    @Override
-    public int hashCode() {
-      int hashCode = 1;
-
-      hashCode = hashCode * 8191 + ((isSetSuccess()) ? 131071 : 524287);
-      if (isSetSuccess())
-        hashCode = hashCode * 8191 + success.hashCode();
-
-      hashCode = hashCode * 8191 + ((isSetNssi()) ? 131071 : 524287);
-      if (isSetNssi())
-        hashCode = hashCode * 8191 + nssi.hashCode();
-
-      hashCode = hashCode * 8191 + ((isSetNste()) ? 131071 : 524287);
-      if (isSetNste())
-        hashCode = hashCode * 8191 + nste.hashCode();
-
-      hashCode = hashCode * 8191 + ((isSetTmfe()) ? 131071 : 524287);
-      if (isSetTmfe())
-        hashCode = hashCode * 8191 + tmfe.hashCode();
-
-      hashCode = hashCode * 8191 + ((isSetTsnpe()) ? 131071 : 524287);
-      if (isSetTsnpe())
-        hashCode = hashCode * 8191 + tsnpe.hashCode();
-
-      return hashCode;
-    }
-
-    @Override
-    public int compareTo(continueScan_result other) {
-      if (!getClass().equals(other.getClass())) {
-        return getClass().getName().compareTo(other.getClass().getName());
-      }
-
-      int lastComparison = 0;
-
-      lastComparison = java.lang.Boolean.compare(isSetSuccess(), other.isSetSuccess());
-      if (lastComparison != 0) {
-        return lastComparison;
-      }
-      if (isSetSuccess()) {
-        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, other.success);
-        if (lastComparison != 0) {
-          return lastComparison;
-        }
-      }
-      lastComparison = java.lang.Boolean.compare(isSetNssi(), other.isSetNssi());
-      if (lastComparison != 0) {
-        return lastComparison;
-      }
-      if (isSetNssi()) {
-        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.nssi, other.nssi);
-        if (lastComparison != 0) {
-          return lastComparison;
-        }
-      }
-      lastComparison = java.lang.Boolean.compare(isSetNste(), other.isSetNste());
-      if (lastComparison != 0) {
-        return lastComparison;
-      }
-      if (isSetNste()) {
-        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.nste, other.nste);
-        if (lastComparison != 0) {
-          return lastComparison;
-        }
-      }
-      lastComparison = java.lang.Boolean.compare(isSetTmfe(), other.isSetTmfe());
-      if (lastComparison != 0) {
-        return lastComparison;
-      }
-      if (isSetTmfe()) {
-        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tmfe, other.tmfe);
-        if (lastComparison != 0) {
-          return lastComparison;
-        }
-      }
-      lastComparison = java.lang.Boolean.compare(isSetTsnpe(), other.isSetTsnpe());
-      if (lastComparison != 0) {
-        return lastComparison;
-      }
-      if (isSetTsnpe()) {
-        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tsnpe, other.tsnpe);
-        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("continueScan_result(");
-      boolean first = true;
-
-      sb.append("success:");
-      if (this.success == null) {
-        sb.append("null");
-      } else {
-        sb.append(this.success);
-      }
-      first = false;
-      if (!first) sb.append(", ");
-      sb.append("nssi:");
-      if (this.nssi == null) {
-        sb.append("null");
-      } else {
-        sb.append(this.nssi);
-      }
-      first = false;
-      if (!first) sb.append(", ");
-      sb.append("nste:");
-      if (this.nste == null) {
-        sb.append("null");
-      } else {
-        sb.append(this.nste);
-      }
-      first = false;
-      if (!first) sb.append(", ");
-      sb.append("tmfe:");
-      if (this.tmfe == null) {
-        sb.append("null");
-      } else {
-        sb.append(this.tmfe);
-      }
-      first = false;
-      if (!first) sb.append(", ");
-      sb.append("tsnpe:");
-      if (this.tsnpe == null) {
-        sb.append("null");
-      } else {
-        sb.append(this.tsnpe);
-      }
-      first = false;
-      sb.append(")");
-      return sb.toString();
-    }
-
-    public void validate() throws org.apache.thrift.TException {
-      // check for required fields
-      // check for sub-struct validity
-      if (success != null) {
-        success.validate();
-      }
-    }
-
-    private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
-      try {
-        write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
-      } catch (org.apache.thrift.TException te) {
-        throw new java.io.IOException(te);
-      }
-    }
-
-    private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, java.lang.ClassNotFoundException {
-      try {
-        read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
-      } catch (org.apache.thrift.TException te) {
-        throw new java.io.IOException(te);
-      }
-    }
-
-    private static class continueScan_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      public continueScan_resultStandardScheme getScheme() {
-        return new continueScan_resultStandardScheme();
-      }
-    }
-
-    private static class continueScan_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme<continueScan_result> {
-
-      public void read(org.apache.thrift.protocol.TProtocol iprot, continueScan_result struct) throws org.apache.thrift.TException {
-        org.apache.thrift.protocol.TField schemeField;
-        iprot.readStructBegin();
-        while (true)
-        {
-          schemeField = iprot.readFieldBegin();
-          if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
-            break;
-          }
-          switch (schemeField.id) {
-            case 0: // SUCCESS
-              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
-                struct.success = new org.apache.accumulo.core.dataImpl.thrift.ScanResult();
-                struct.success.read(iprot);
-                struct.setSuccessIsSet(true);
-              } else { 
-                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
-              }
-              break;
-            case 1: // NSSI
-              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
-                struct.nssi = new NoSuchScanIDException();
-                struct.nssi.read(iprot);
-                struct.setNssiIsSet(true);
-              } else { 
-                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
-              }
-              break;
-            case 2: // NSTE
-              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
-                struct.nste = new NotServingTabletException();
-                struct.nste.read(iprot);
-                struct.setNsteIsSet(true);
-              } else { 
-                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
-              }
-              break;
-            case 3: // TMFE
-              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
-                struct.tmfe = new TooManyFilesException();
-                struct.tmfe.read(iprot);
-                struct.setTmfeIsSet(true);
-              } else { 
-                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
-              }
-              break;
-            case 4: // TSNPE
-              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
-                struct.tsnpe = new TSampleNotPresentException();
-                struct.tsnpe.read(iprot);
-                struct.setTsnpeIsSet(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, continueScan_result struct) throws org.apache.thrift.TException {
-        struct.validate();
-
-        oprot.writeStructBegin(STRUCT_DESC);
-        if (struct.success != null) {
-          oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
-          struct.success.write(oprot);
-          oprot.writeFieldEnd();
-        }
-        if (struct.nssi != null) {
-          oprot.writeFieldBegin(NSSI_FIELD_DESC);
-          struct.nssi.write(oprot);
-          oprot.writeFieldEnd();
-        }
-        if (struct.nste != null) {
-          oprot.writeFieldBegin(NSTE_FIELD_DESC);
-          struct.nste.write(oprot);
-          oprot.writeFieldEnd();
-        }
-        if (struct.tmfe != null) {
-          oprot.writeFieldBegin(TMFE_FIELD_DESC);
-          struct.tmfe.write(oprot);
-          oprot.writeFieldEnd();
-        }
-        if (struct.tsnpe != null) {
-          oprot.writeFieldBegin(TSNPE_FIELD_DESC);
-          struct.tsnpe.write(oprot);
-          oprot.writeFieldEnd();
-        }
-        oprot.writeFieldStop();
-        oprot.writeStructEnd();
-      }
-
-    }
-
-    private static class continueScan_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      public continueScan_resultTupleScheme getScheme() {
-        return new continueScan_resultTupleScheme();
-      }
-    }
-
-    private static class continueScan_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme<continueScan_result> {
-
-      @Override
-      public void write(org.apache.thrift.protocol.TProtocol prot, continueScan_result struct) throws org.apache.thrift.TException {
-        org.apache.thrift.protocol.TTupleProtocol oprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
-        java.util.BitSet optionals = new java.util.BitSet();
-        if (struct.isSetSuccess()) {
-          optionals.set(0);
-        }
-        if (struct.isSetNssi()) {
-          optionals.set(1);
-        }
-        if (struct.isSetNste()) {
-          optionals.set(2);
-        }
-        if (struct.isSetTmfe()) {
-          optionals.set(3);
-        }
-        if (struct.isSetTsnpe()) {
-          optionals.set(4);
-        }
-        oprot.writeBitSet(optionals, 5);
-        if (struct.isSetSuccess()) {
-          struct.success.write(oprot);
-        }
-        if (struct.isSetNssi()) {
-          struct.nssi.write(oprot);
-        }
-        if (struct.isSetNste()) {
-          struct.nste.write(oprot);
-        }
-        if (struct.isSetTmfe()) {
-          struct.tmfe.write(oprot);
-        }
-        if (struct.isSetTsnpe()) {
-          struct.tsnpe.write(oprot);
-        }
-      }
-
-      @Override
-      public void read(org.apache.thrift.protocol.TProtocol prot, continueScan_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(5);
-        if (incoming.get(0)) {
-          struct.success = new org.apache.accumulo.core.dataImpl.thrift.ScanResult();
-          struct.success.read(iprot);
-          struct.setSuccessIsSet(true);
-        }
-        if (incoming.get(1)) {
-          struct.nssi = new NoSuchScanIDException();
-          struct.nssi.read(iprot);
-          struct.setNssiIsSet(true);
-        }
-        if (incoming.get(2)) {
-          struct.nste = new NotServingTabletException();
-          struct.nste.read(iprot);
-          struct.setNsteIsSet(true);
-        }
-        if (incoming.get(3)) {
-          struct.tmfe = new TooManyFilesException();
-          struct.tmfe.read(iprot);
-          struct.setTmfeIsSet(true);
-        }
-        if (incoming.get(4)) {
-          struct.tsnpe = new TSampleNotPresentException();
-          struct.tsnpe.read(iprot);
-          struct.setTsnpeIsSet(true);
-        }
-      }
-    }
-
-    private static <S extends org.apache.thrift.scheme.IScheme> S scheme(org.apache.thrift.protocol.TProtocol proto) {
-      return (org.apache.thrift.scheme.StandardScheme.class.equals(proto.getScheme()) ? STANDARD_SCHEME_FACTORY : TUPLE_SCHEME_FACTORY).getScheme();
-    }
-  }
-
-  public static class closeScan_args implements org.apache.thrift.TBase<closeScan_args, closeScan_args._Fields>, java.io.Serializable, Cloneable, Comparable<closeScan_args>   {
-    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("closeScan_args");
-
-    private static final org.apache.thrift.protocol.TField TINFO_FIELD_DESC = new org.apache.thrift.protocol.TField("tinfo", org.apache.thrift.protocol.TType.STRUCT, (short)2);
-    private static final org.apache.thrift.protocol.TField SCAN_ID_FIELD_DESC = new org.apache.thrift.protocol.TField("scanID", org.apache.thrift.protocol.TType.I64, (short)1);
-
-    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new closeScan_argsStandardSchemeFactory();
-    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new closeScan_argsTupleSchemeFactory();
-
-    public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo; // required
-    public long scanID; // required
-
-    /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
-    public enum _Fields implements org.apache.thrift.TFieldIdEnum {
-      TINFO((short)2, "tinfo"),
-      SCAN_ID((short)1, "scanID");
-
-      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 2: // TINFO
-            return TINFO;
-          case 1: // SCAN_ID
-            return SCAN_ID;
-          default:
-            return null;
-        }
-      }
-
-      /**
-       * Find the _Fields constant that matches fieldId, throwing an exception
-       * if it is not found.
-       */
-      public static _Fields findByThriftIdOrThrow(int fieldId) {
-        _Fields fields = findByThriftId(fieldId);
-        if (fields == null) throw new java.lang.IllegalArgumentException("Field " + fieldId + " doesn't exist!");
-        return fields;
-      }
-
-      /**
-       * Find the _Fields constant that matches name, or null if its not found.
-       */
-      @org.apache.thrift.annotation.Nullable
-      public static _Fields findByName(java.lang.String name) {
-        return byName.get(name);
-      }
-
-      private final short _thriftId;
-      private final java.lang.String _fieldName;
-
-      _Fields(short thriftId, java.lang.String fieldName) {
-        _thriftId = thriftId;
-        _fieldName = fieldName;
-      }
-
-      public short getThriftFieldId() {
-        return _thriftId;
-      }
-
-      public java.lang.String getFieldName() {
-        return _fieldName;
-      }
-    }
-
-    // isset id assignments
-    private static final int __SCANID_ISSET_ID = 0;
-    private byte __isset_bitfield = 0;
-    public static final java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
-    static {
-      java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
-      tmpMap.put(_Fields.TINFO, new org.apache.thrift.meta_data.FieldMetaData("tinfo", org.apache.thrift.TFieldRequirementType.DEFAULT, 
-          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.trace.thrift.TInfo.class)));
-      tmpMap.put(_Fields.SCAN_ID, new org.apache.thrift.meta_data.FieldMetaData("scanID", org.apache.thrift.TFieldRequirementType.DEFAULT, 
-          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64          , "ScanID")));
-      metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
-      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(closeScan_args.class, metaDataMap);
-    }
-
-    public closeScan_args() {
-    }
-
-    public closeScan_args(
-      org.apache.accumulo.core.trace.thrift.TInfo tinfo,
-      long scanID)
-    {
-      this();
-      this.tinfo = tinfo;
-      this.scanID = scanID;
-      setScanIDIsSet(true);
-    }
-
-    /**
-     * Performs a deep copy on <i>other</i>.
-     */
-    public closeScan_args(closeScan_args other) {
-      __isset_bitfield = other.__isset_bitfield;
-      if (other.isSetTinfo()) {
-        this.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo(other.tinfo);
-      }
-      this.scanID = other.scanID;
-    }
-
-    public closeScan_args deepCopy() {
-      return new closeScan_args(this);
-    }
-
-    @Override
-    public void clear() {
-      this.tinfo = null;
-      setScanIDIsSet(false);
-      this.scanID = 0;
-    }
-
-    @org.apache.thrift.annotation.Nullable
-    public org.apache.accumulo.core.trace.thrift.TInfo getTinfo() {
-      return this.tinfo;
-    }
-
-    public closeScan_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo) {
-      this.tinfo = tinfo;
-      return this;
-    }
-
-    public void unsetTinfo() {
-      this.tinfo = null;
-    }
-
-    /** Returns true if field tinfo is set (has been assigned a value) and false otherwise */
-    public boolean isSetTinfo() {
-      return this.tinfo != null;
-    }
-
-    public void setTinfoIsSet(boolean value) {
-      if (!value) {
-        this.tinfo = null;
-      }
-    }
-
-    public long getScanID() {
-      return this.scanID;
-    }
-
-    public closeScan_args setScanID(long scanID) {
-      this.scanID = scanID;
-      setScanIDIsSet(true);
-      return this;
-    }
-
-    public void unsetScanID() {
-      __isset_bitfield = org.apache.thrift.EncodingUtils.clearBit(__isset_bitfield, __SCANID_ISSET_ID);
-    }
-
-    /** Returns true if field scanID is set (has been assigned a value) and false otherwise */
-    public boolean isSetScanID() {
-      return org.apache.thrift.EncodingUtils.testBit(__isset_bitfield, __SCANID_ISSET_ID);
-    }
-
-    public void setScanIDIsSet(boolean value) {
-      __isset_bitfield = org.apache.thrift.EncodingUtils.setBit(__isset_bitfield, __SCANID_ISSET_ID, value);
-    }
-
-    public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable java.lang.Object value) {
-      switch (field) {
-      case TINFO:
-        if (value == null) {
-          unsetTinfo();
-        } else {
-          setTinfo((org.apache.accumulo.core.trace.thrift.TInfo)value);
-        }
-        break;
-
-      case SCAN_ID:
-        if (value == null) {
-          unsetScanID();
-        } else {
-          setScanID((java.lang.Long)value);
-        }
-        break;
-
-      }
-    }
-
-    @org.apache.thrift.annotation.Nullable
-    public java.lang.Object getFieldValue(_Fields field) {
-      switch (field) {
-      case TINFO:
-        return getTinfo();
-
-      case SCAN_ID:
-        return getScanID();
-
-      }
-      throw new java.lang.IllegalStateException();
-    }
-
-    /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
-    public boolean isSet(_Fields field) {
-      if (field == null) {
-        throw new java.lang.IllegalArgumentException();
-      }
-
-      switch (field) {
-      case TINFO:
-        return isSetTinfo();
-      case SCAN_ID:
-        return isSetScanID();
-      }
-      throw new java.lang.IllegalStateException();
-    }
-
-    @Override
-    public boolean equals(java.lang.Object that) {
-      if (that instanceof closeScan_args)
-        return this.equals((closeScan_args)that);
-      return false;
-    }
-
-    public boolean equals(closeScan_args that) {
-      if (that == null)
-        return false;
-      if (this == that)
-        return true;
-
-      boolean this_present_tinfo = true && this.isSetTinfo();
-      boolean that_present_tinfo = true && that.isSetTinfo();
-      if (this_present_tinfo || that_present_tinfo) {
-        if (!(this_present_tinfo && that_present_tinfo))
-          return false;
-        if (!this.tinfo.equals(that.tinfo))
-          return false;
-      }
-
-      boolean this_present_scanID = true;
-      boolean that_present_scanID = true;
-      if (this_present_scanID || that_present_scanID) {
-        if (!(this_present_scanID && that_present_scanID))
-          return false;
-        if (this.scanID != that.scanID)
-          return false;
-      }
-
-      return true;
-    }
-
-    @Override
-    public int hashCode() {
-      int hashCode = 1;
-
-      hashCode = hashCode * 8191 + ((isSetTinfo()) ? 131071 : 524287);
-      if (isSetTinfo())
-        hashCode = hashCode * 8191 + tinfo.hashCode();
-
-      hashCode = hashCode * 8191 + org.apache.thrift.TBaseHelper.hashCode(scanID);
-
-      return hashCode;
-    }
-
-    @Override
-    public int compareTo(closeScan_args other) {
-      if (!getClass().equals(other.getClass())) {
-        return getClass().getName().compareTo(other.getClass().getName());
-      }
-
-      int lastComparison = 0;
-
-      lastComparison = java.lang.Boolean.compare(isSetTinfo(), other.isSetTinfo());
-      if (lastComparison != 0) {
-        return lastComparison;
-      }
-      if (isSetTinfo()) {
-        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tinfo, other.tinfo);
-        if (lastComparison != 0) {
-          return lastComparison;
-        }
-      }
-      lastComparison = java.lang.Boolean.compare(isSetScanID(), other.isSetScanID());
-      if (lastComparison != 0) {
-        return lastComparison;
-      }
-      if (isSetScanID()) {
-        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.scanID, other.scanID);
-        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("closeScan_args(");
-      boolean first = true;
-
-      sb.append("tinfo:");
-      if (this.tinfo == null) {
-        sb.append("null");
-      } else {
-        sb.append(this.tinfo);
-      }
-      first = false;
-      if (!first) sb.append(", ");
-      sb.append("scanID:");
-      sb.append(this.scanID);
-      first = false;
-      sb.append(")");
-      return sb.toString();
-    }
-
-    public void validate() throws org.apache.thrift.TException {
-      // check for required fields
-      // check for sub-struct validity
-      if (tinfo != null) {
-        tinfo.validate();
-      }
-    }
-
-    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 closeScan_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      public closeScan_argsStandardScheme getScheme() {
-        return new closeScan_argsStandardScheme();
-      }
-    }
-
-    private static class closeScan_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme<closeScan_args> {
-
-      public void read(org.apache.thrift.protocol.TProtocol iprot, closeScan_args struct) throws org.apache.thrift.TException {
-        org.apache.thrift.protocol.TField schemeField;
-        iprot.readStructBegin();
-        while (true)
-        {
-          schemeField = iprot.readFieldBegin();
-          if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
-            break;
-          }
-          switch (schemeField.id) {
-            case 2: // TINFO
-              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
-                struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
-                struct.tinfo.read(iprot);
-                struct.setTinfoIsSet(true);
-              } else { 
-                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
-              }
-              break;
-            case 1: // SCAN_ID
-              if (schemeField.type == org.apache.thrift.protocol.TType.I64) {
-                struct.scanID = iprot.readI64();
-                struct.setScanIDIsSet(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, closeScan_args struct) throws org.apache.thrift.TException {
-        struct.validate();
-
-        oprot.writeStructBegin(STRUCT_DESC);
-        oprot.writeFieldBegin(SCAN_ID_FIELD_DESC);
-        oprot.writeI64(struct.scanID);
-        oprot.writeFieldEnd();
-        if (struct.tinfo != null) {
-          oprot.writeFieldBegin(TINFO_FIELD_DESC);
-          struct.tinfo.write(oprot);
-          oprot.writeFieldEnd();
-        }
-        oprot.writeFieldStop();
-        oprot.writeStructEnd();
-      }
-
-    }
-
-    private static class closeScan_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      public closeScan_argsTupleScheme getScheme() {
-        return new closeScan_argsTupleScheme();
-      }
-    }
-
-    private static class closeScan_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme<closeScan_args> {
-
-      @Override
-      public void write(org.apache.thrift.protocol.TProtocol prot, closeScan_args struct) throws org.apache.thrift.TException {
-        org.apache.thrift.protocol.TTupleProtocol oprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
-        java.util.BitSet optionals = new java.util.BitSet();
-        if (struct.isSetTinfo()) {
-          optionals.set(0);
-        }
-        if (struct.isSetScanID()) {
-          optionals.set(1);
-        }
-        oprot.writeBitSet(optionals, 2);
-        if (struct.isSetTinfo()) {
-          struct.tinfo.write(oprot);
-        }
-        if (struct.isSetScanID()) {
-          oprot.writeI64(struct.scanID);
-        }
-      }
-
-      @Override
-      public void read(org.apache.thrift.protocol.TProtocol prot, closeScan_args struct) throws org.apache.thrift.TException {
-        org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
-        java.util.BitSet incoming = iprot.readBitSet(2);
-        if (incoming.get(0)) {
-          struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
-          struct.tinfo.read(iprot);
-          struct.setTinfoIsSet(true);
-        }
-        if (incoming.get(1)) {
-          struct.scanID = iprot.readI64();
-          struct.setScanIDIsSet(true);
-        }
-      }
-    }
-
-    private static <S extends org.apache.thrift.scheme.IScheme> S scheme(org.apache.thrift.protocol.TProtocol proto) {
-      return (org.apache.thrift.scheme.StandardScheme.class.equals(proto.getScheme()) ? STANDARD_SCHEME_FACTORY : TUPLE_SCHEME_FACTORY).getScheme();
-    }
-  }
-
-  public static class startMultiScan_args implements org.apache.thrift.TBase<startMultiScan_args, startMultiScan_args._Fields>, java.io.Serializable, Cloneable, Comparable<startMultiScan_args>   {
-    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("startMultiScan_args");
-
-    private static final org.apache.thrift.protocol.TField TINFO_FIELD_DESC = new org.apache.thrift.protocol.TField("tinfo", org.apache.thrift.protocol.TType.STRUCT, (short)8);
-    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)1);
-    private static final org.apache.thrift.protocol.TField BATCH_FIELD_DESC = new org.apache.thrift.protocol.TField("batch", org.apache.thrift.protocol.TType.MAP, (short)2);
-    private static final org.apache.thrift.protocol.TField COLUMNS_FIELD_DESC = new org.apache.thrift.protocol.TField("columns", org.apache.thrift.protocol.TType.LIST, (short)3);
-    private static final org.apache.thrift.protocol.TField SSI_LIST_FIELD_DESC = new org.apache.thrift.protocol.TField("ssiList", org.apache.thrift.protocol.TType.LIST, (short)4);
-    private static final org.apache.thrift.protocol.TField SSIO_FIELD_DESC = new org.apache.thrift.protocol.TField("ssio", org.apache.thrift.protocol.TType.MAP, (short)5);
-    private static final org.apache.thrift.protocol.TField AUTHORIZATIONS_FIELD_DESC = new org.apache.thrift.protocol.TField("authorizations", org.apache.thrift.protocol.TType.LIST, (short)6);
-    private static final org.apache.thrift.protocol.TField WAIT_FOR_WRITES_FIELD_DESC = new org.apache.thrift.protocol.TField("waitForWrites", org.apache.thrift.protocol.TType.BOOL, (short)7);
-    private static final org.apache.thrift.protocol.TField SAMPLER_CONFIG_FIELD_DESC = new org.apache.thrift.protocol.TField("samplerConfig", org.apache.thrift.protocol.TType.STRUCT, (short)9);
-    private static final org.apache.thrift.protocol.TField BATCH_TIME_OUT_FIELD_DESC = new org.apache.thrift.protocol.TField("batchTimeOut", org.apache.thrift.protocol.TType.I64, (short)10);
-    private static final org.apache.thrift.protocol.TField CLASS_LOADER_CONTEXT_FIELD_DESC = new org.apache.thrift.protocol.TField("classLoaderContext", org.apache.thrift.protocol.TType.STRING, (short)11);
-    private static final org.apache.thrift.protocol.TField EXECUTION_HINTS_FIELD_DESC = new org.apache.thrift.protocol.TField("executionHints", org.apache.thrift.protocol.TType.MAP, (short)12);
-
-    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new startMultiScan_argsStandardSchemeFactory();
-    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new startMultiScan_argsTupleSchemeFactory();
-
-    public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo; // required
-    public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials; // required
-    public @org.apache.thrift.annotation.Nullable java.util.Map<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent,java.util.List<org.apache.accumulo.core.dataImpl.thrift.TRange>> batch; // required
-    public @org.apache.thrift.annotation.Nullable java.util.List<org.apache.accumulo.core.dataImpl.thrift.TColumn> columns; // required
-    public @org.apache.thrift.annotation.Nullable java.util.List<org.apache.accumulo.core.dataImpl.thrift.IterInfo> ssiList; // required
-    public @org.apache.thrift.annotation.Nullable java.util.Map<java.lang.String,java.util.Map<java.lang.String,java.lang.String>> ssio; // required
-    public @org.apache.thrift.annotation.Nullable java.util.List<java.nio.ByteBuffer> authorizations; // required
-    public boolean waitForWrites; // required
-    public @org.apache.thrift.annotation.Nullable TSamplerConfiguration samplerConfig; // required
-    public long batchTimeOut; // required
-    public @org.apache.thrift.annotation.Nullable java.lang.String classLoaderContext; // required
-    public @org.apache.thrift.annotation.Nullable java.util.Map<java.lang.String,java.lang.String> executionHints; // required
-
-    /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
-    public enum _Fields implements org.apache.thrift.TFieldIdEnum {
-      TINFO((short)8, "tinfo"),
-      CREDENTIALS((short)1, "credentials"),
-      BATCH((short)2, "batch"),
-      COLUMNS((short)3, "columns"),
-      SSI_LIST((short)4, "ssiList"),
-      SSIO((short)5, "ssio"),
-      AUTHORIZATIONS((short)6, "authorizations"),
-      WAIT_FOR_WRITES((short)7, "waitForWrites"),
-      SAMPLER_CONFIG((short)9, "samplerConfig"),
-      BATCH_TIME_OUT((short)10, "batchTimeOut"),
-      CLASS_LOADER_CONTEXT((short)11, "classLoaderContext"),
-      EXECUTION_HINTS((short)12, "executionHints");
-
-      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 8: // TINFO
-            return TINFO;
-          case 1: // CREDENTIALS
-            return CREDENTIALS;
-          case 2: // BATCH
-            return BATCH;
-          case 3: // COLUMNS
-            return COLUMNS;
-          case 4: // SSI_LIST
-            return SSI_LIST;
-          case 5: // SSIO
-            return SSIO;
-          case 6: // AUTHORIZATIONS
-            return AUTHORIZATIONS;
-          case 7: // WAIT_FOR_WRITES
-            return WAIT_FOR_WRITES;
-          case 9: // SAMPLER_CONFIG
-            return SAMPLER_CONFIG;
-          case 10: // BATCH_TIME_OUT
-            return BATCH_TIME_OUT;
-          case 11: // CLASS_LOADER_CONTEXT
-            return CLASS_LOADER_CONTEXT;
-          case 12: // EXECUTION_HINTS
-            return EXECUTION_HINTS;
-          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 __WAITFORWRITES_ISSET_ID = 0;
-    private static final int __BATCHTIMEOUT_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.TINFO, new org.apache.thrift.meta_data.FieldMetaData("tinfo", org.apache.thrift.TFieldRequirementType.DEFAULT, 
-          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.trace.thrift.TInfo.class)));
-      tmpMap.put(_Fields.CREDENTIALS, new org.apache.thrift.meta_data.FieldMetaData("credentials", org.apache.thrift.TFieldRequirementType.DEFAULT, 
-          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.securityImpl.thrift.TCredentials.class)));
-      tmpMap.put(_Fields.BATCH, new org.apache.thrift.meta_data.FieldMetaData("batch", org.apache.thrift.TFieldRequirementType.DEFAULT, 
-          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.MAP          , "ScanBatch")));
-      tmpMap.put(_Fields.COLUMNS, new org.apache.thrift.meta_data.FieldMetaData("columns", 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, org.apache.accumulo.core.dataImpl.thrift.TColumn.class))));
-      tmpMap.put(_Fields.SSI_LIST, new org.apache.thrift.meta_data.FieldMetaData("ssiList", 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, org.apache.accumulo.core.dataImpl.thrift.IterInfo.class))));
-      tmpMap.put(_Fields.SSIO, new org.apache.thrift.meta_data.FieldMetaData("ssio", org.apache.thrift.TFieldRequirementType.DEFAULT, 
-          new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP, 
-              new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING), 
-              new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP, 
-                  new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING), 
-                  new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)))));
-      tmpMap.put(_Fields.AUTHORIZATIONS, new org.apache.thrift.meta_data.FieldMetaData("authorizations", 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              , true))));
-      tmpMap.put(_Fields.WAIT_FOR_WRITES, new org.apache.thrift.meta_data.FieldMetaData("waitForWrites", org.apache.thrift.TFieldRequirementType.DEFAULT, 
-          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.BOOL)));
-      tmpMap.put(_Fields.SAMPLER_CONFIG, new org.apache.thrift.meta_data.FieldMetaData("samplerConfig", org.apache.thrift.TFieldRequirementType.DEFAULT, 
-          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TSamplerConfiguration.class)));
-      tmpMap.put(_Fields.BATCH_TIME_OUT, new org.apache.thrift.meta_data.FieldMetaData("batchTimeOut", org.apache.thrift.TFieldRequirementType.DEFAULT, 
-          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64)));
-      tmpMap.put(_Fields.CLASS_LOADER_CONTEXT, new org.apache.thrift.meta_data.FieldMetaData("classLoaderContext", org.apache.thrift.TFieldRequirementType.DEFAULT, 
-          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
-      tmpMap.put(_Fields.EXECUTION_HINTS, new org.apache.thrift.meta_data.FieldMetaData("executionHints", org.apache.thrift.TFieldRequirementType.DEFAULT, 
-          new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP, 
-              new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING), 
-              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(startMultiScan_args.class, metaDataMap);
-    }
-
-    public startMultiScan_args() {
-    }
-
-    public startMultiScan_args(
-      org.apache.accumulo.core.trace.thrift.TInfo tinfo,
-      org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials,
-      java.util.Map<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent,java.util.List<org.apache.accumulo.core.dataImpl.thrift.TRange>> batch,
-      java.util.List<org.apache.accumulo.core.dataImpl.thrift.TColumn> columns,
-      java.util.List<org.apache.accumulo.core.dataImpl.thrift.IterInfo> ssiList,
-      java.util.Map<java.lang.String,java.util.Map<java.lang.String,java.lang.String>> ssio,
-      java.util.List<java.nio.ByteBuffer> authorizations,
-      boolean waitForWrites,
-      TSamplerConfiguration samplerConfig,
-      long batchTimeOut,
-      java.lang.String classLoaderContext,
-      java.util.Map<java.lang.String,java.lang.String> executionHints)
-    {
-      this();
-      this.tinfo = tinfo;
-      this.credentials = credentials;
-      this.batch = batch;
-      this.columns = columns;
-      this.ssiList = ssiList;
-      this.ssio = ssio;
-      this.authorizations = authorizations;
-      this.waitForWrites = waitForWrites;
-      setWaitForWritesIsSet(true);
-      this.samplerConfig = samplerConfig;
-      this.batchTimeOut = batchTimeOut;
-      setBatchTimeOutIsSet(true);
-      this.classLoaderContext = classLoaderContext;
-      this.executionHints = executionHints;
-    }
-
-    /**
-     * Performs a deep copy on <i>other</i>.
-     */
-    public startMultiScan_args(startMultiScan_args other) {
-      __isset_bitfield = other.__isset_bitfield;
-      if (other.isSetTinfo()) {
-        this.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo(other.tinfo);
-      }
-      if (other.isSetCredentials()) {
-        this.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(other.credentials);
-      }
-      if (other.isSetBatch()) {
-        java.util.Map<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent,java.util.List<org.apache.accumulo.core.dataImpl.thrift.TRange>> __this__batch = new java.util.HashMap<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent,java.util.List<org.apache.accumulo.core.dataImpl.thrift.TRange>>(other.batch.size());
-        for (java.util.Map.Entry<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent, java.util.List<org.apache.accumulo.core.dataImpl.thrift.TRange>> other_element : other.batch.entrySet()) {
-
-          org.apache.accumulo.core.dataImpl.thrift.TKeyExtent other_element_key = other_element.getKey();
-          java.util.List<org.apache.accumulo.core.dataImpl.thrift.TRange> other_element_value = other_element.getValue();
-
-          org.apache.accumulo.core.dataImpl.thrift.TKeyExtent __this__batch_copy_key = new org.apache.accumulo.core.dataImpl.thrift.TKeyExtent(other_element_key);
-
-          java.util.List<org.apache.accumulo.core.dataImpl.thrift.TRange> __this__batch_copy_value = new java.util.ArrayList<org.apache.accumulo.core.dataImpl.thrift.TRange>(other_element_value.size());
-          for (org.apache.accumulo.core.dataImpl.thrift.TRange other_element_value_element : other_element_value) {
-            __this__batch_copy_value.add(new org.apache.accumulo.core.dataImpl.thrift.TRange(other_element_value_element));
-          }
-
-          __this__batch.put(__this__batch_copy_key, __this__batch_copy_value);
-        }
-        this.batch = __this__batch;
-      }
-      if (other.isSetColumns()) {
-        java.util.List<org.apache.accumulo.core.dataImpl.thrift.TColumn> __this__columns = new java.util.ArrayList<org.apache.accumulo.core.dataImpl.thrift.TColumn>(other.columns.size());
-        for (org.apache.accumulo.core.dataImpl.thrift.TColumn other_element : other.columns) {
-          __this__columns.add(new org.apache.accumulo.core.dataImpl.thrift.TColumn(other_element));
-        }
-        this.columns = __this__columns;
-      }
-      if (other.isSetSsiList()) {
-        java.util.List<org.apache.accumulo.core.dataImpl.thrift.IterInfo> __this__ssiList = new java.util.ArrayList<org.apache.accumulo.core.dataImpl.thrift.IterInfo>(other.ssiList.size());
-        for (org.apache.accumulo.core.dataImpl.thrift.IterInfo other_element : other.ssiList) {
-          __this__ssiList.add(new org.apache.accumulo.core.dataImpl.thrift.IterInfo(other_element));
-        }
-        this.ssiList = __this__ssiList;
-      }
-      if (other.isSetSsio()) {
-        java.util.Map<java.lang.String,java.util.Map<java.lang.String,java.lang.String>> __this__ssio = new java.util.HashMap<java.lang.String,java.util.Map<java.lang.String,java.lang.String>>(other.ssio.size());
-        for (java.util.Map.Entry<java.lang.String, java.util.Map<java.lang.String,java.lang.String>> other_element : other.ssio.entrySet()) {
-
-          java.lang.String other_element_key = other_element.getKey();
-          java.util.Map<java.lang.String,java.lang.String> other_element_value = other_element.getValue();
-
-          java.lang.String __this__ssio_copy_key = other_element_key;
-
-          java.util.Map<java.lang.String,java.lang.String> __this__ssio_copy_value = new java.util.HashMap<java.lang.String,java.lang.String>(other_element_value);
-
-          __this__ssio.put(__this__ssio_copy_key, __this__ssio_copy_value);
-        }
-        this.ssio = __this__ssio;
-      }
-      if (other.isSetAuthorizations()) {
-        java.util.List<java.nio.ByteBuffer> __this__authorizations = new java.util.ArrayList<java.nio.ByteBuffer>(other.authorizations);
-        this.authorizations = __this__authorizations;
-      }
-      this.waitForWrites = other.waitForWrites;
-      if (other.isSetSamplerConfig()) {
-        this.samplerConfig = new TSamplerConfiguration(other.samplerConfig);
-      }
-      this.batchTimeOut = other.batchTimeOut;
-      if (other.isSetClassLoaderContext()) {
-        this.classLoaderContext = other.classLoaderContext;
-      }
-      if (other.isSetExecutionHints()) {
-        java.util.Map<java.lang.String,java.lang.String> __this__executionHints = new java.util.HashMap<java.lang.String,java.lang.String>(other.executionHints);
-        this.executionHints = __this__executionHints;
-      }
-    }
-
-    public startMultiScan_args deepCopy() {
-      return new startMultiScan_args(this);
-    }
-
-    @Override
-    public void clear() {
-      this.tinfo = null;
-      this.credentials = null;
-      this.batch = null;
-      this.columns = null;
-      this.ssiList = null;
-      this.ssio = null;
-      this.authorizations = null;
-      setWaitForWritesIsSet(false);
-      this.waitForWrites = false;
-      this.samplerConfig = null;
-      setBatchTimeOutIsSet(false);
-      this.batchTimeOut = 0;
-      this.classLoaderContext = null;
-      this.executionHints = null;
-    }
-
-    @org.apache.thrift.annotation.Nullable
-    public org.apache.accumulo.core.trace.thrift.TInfo getTinfo() {
-      return this.tinfo;
-    }
-
-    public startMultiScan_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo) {
-      this.tinfo = tinfo;
-      return this;
-    }
-
-    public void unsetTinfo() {
-      this.tinfo = null;
-    }
-
-    /** Returns true if field tinfo is set (has been assigned a value) and false otherwise */
-    public boolean isSetTinfo() {
-      return this.tinfo != null;
-    }
-
-    public void setTinfoIsSet(boolean value) {
-      if (!value) {
-        this.tinfo = null;
-      }
-    }
-
-    @org.apache.thrift.annotation.Nullable
-    public org.apache.accumulo.core.securityImpl.thrift.TCredentials getCredentials() {
-      return this.credentials;
-    }
-
-    public startMultiScan_args setCredentials(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) {
-      this.credentials = credentials;
-      return this;
-    }
-
-    public void unsetCredentials() {
-      this.credentials = null;
-    }
-
-    /** Returns true if field credentials is set (has been assigned a value) and false otherwise */
-    public boolean isSetCredentials() {
-      return this.credentials != null;
-    }
-
-    public void setCredentialsIsSet(boolean value) {
-      if (!value) {
-        this.credentials = null;
-      }
-    }
-
-    public int getBatchSize() {
-      return (this.batch == null) ? 0 : this.batch.size();
-    }
-
-    public void putToBatch(org.apache.accumulo.core.dataImpl.thrift.TKeyExtent key, java.util.List<org.apache.accumulo.core.dataImpl.thrift.TRange> val) {
-      if (this.batch == null) {
-        this.batch = new java.util.HashMap<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent,java.util.List<org.apache.accumulo.core.dataImpl.thrift.TRange>>();
-      }
-      this.batch.put(key, val);
-    }
-
-    @org.apache.thrift.annotation.Nullable
-    public java.util.Map<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent,java.util.List<org.apache.accumulo.core.dataImpl.thrift.TRange>> getBatch() {
-      return this.batch;
-    }
-
-    public startMultiScan_args setBatch(@org.apache.thrift.annotation.Nullable java.util.Map<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent,java.util.List<org.apache.accumulo.core.dataImpl.thrift.TRange>> batch) {
-      this.batch = batch;
-      return this;
-    }
-
-    public void unsetBatch() {
-      this.batch = null;
-    }
-
-    /** Returns true if field batch is set (has been assigned a value) and false otherwise */
-    public boolean isSetBatch() {
-      return this.batch != null;
-    }
-
-    public void setBatchIsSet(boolean value) {
-      if (!value) {
-        this.batch = null;
-      }
-    }
-
-    public int getColumnsSize() {
-      return (this.columns == null) ? 0 : this.columns.size();
-    }
-
-    @org.apache.thrift.annotation.Nullable
-    public java.util.Iterator<org.apache.accumulo.core.dataImpl.thrift.TColumn> getColumnsIterator() {
-      return (this.columns == null) ? null : this.columns.iterator();
-    }
-
-    public void addToColumns(org.apache.accumulo.core.dataImpl.thrift.TColumn elem) {
-      if (this.columns == null) {
-        this.columns = new java.util.ArrayList<org.apache.accumulo.core.dataImpl.thrift.TColumn>();
-      }
-      this.columns.add(elem);
-    }
-
-    @org.apache.thrift.annotation.Nullable
-    public java.util.List<org.apache.accumulo.core.dataImpl.thrift.TColumn> getColumns() {
-      return this.columns;
-    }
-
-    public startMultiScan_args setColumns(@org.apache.thrift.annotation.Nullable java.util.List<org.apache.accumulo.core.dataImpl.thrift.TColumn> columns) {
-      this.columns = columns;
-      return this;
-    }
-
-    public void unsetColumns() {
-      this.columns = null;
-    }
-
-    /** Returns true if field columns is set (has been assigned a value) and false otherwise */
-    public boolean isSetColumns() {
-      return this.columns != null;
-    }
-
-    public void setColumnsIsSet(boolean value) {
-      if (!value) {
-        this.columns = null;
-      }
-    }
-
-    public int getSsiListSize() {
-      return (this.ssiList == null) ? 0 : this.ssiList.size();
-    }
-
-    @org.apache.thrift.annotation.Nullable
-    public java.util.Iterator<org.apache.accumulo.core.dataImpl.thrift.IterInfo> getSsiListIterator() {
-      return (this.ssiList == null) ? null : this.ssiList.iterator();
-    }
-
-    public void addToSsiList(org.apache.accumulo.core.dataImpl.thrift.IterInfo elem) {
-      if (this.ssiList == null) {
-        this.ssiList = new java.util.ArrayList<org.apache.accumulo.core.dataImpl.thrift.IterInfo>();
-      }
-      this.ssiList.add(elem);
-    }
-
-    @org.apache.thrift.annotation.Nullable
-    public java.util.List<org.apache.accumulo.core.dataImpl.thrift.IterInfo> getSsiList() {
-      return this.ssiList;
-    }
-
-    public startMultiScan_args setSsiList(@org.apache.thrift.annotation.Nullable java.util.List<org.apache.accumulo.core.dataImpl.thrift.IterInfo> ssiList) {
-      this.ssiList = ssiList;
-      return this;
-    }
-
-    public void unsetSsiList() {
-      this.ssiList = null;
-    }
-
-    /** Returns true if field ssiList is set (has been assigned a value) and false otherwise */
-    public boolean isSetSsiList() {
-      return this.ssiList != null;
-    }
-
-    public void setSsiListIsSet(boolean value) {
-      if (!value) {
-        this.ssiList = null;
-      }
-    }
-
-    public int getSsioSize() {
-      return (this.ssio == null) ? 0 : this.ssio.size();
-    }
-
-    public void putToSsio(java.lang.String key, java.util.Map<java.lang.String,java.lang.String> val) {
-      if (this.ssio == null) {
-        this.ssio = new java.util.HashMap<java.lang.String,java.util.Map<java.lang.String,java.lang.String>>();
-      }
-      this.ssio.put(key, val);
-    }
-
-    @org.apache.thrift.annotation.Nullable
-    public java.util.Map<java.lang.String,java.util.Map<java.lang.String,java.lang.String>> getSsio() {
-      return this.ssio;
-    }
-
-    public startMultiScan_args setSsio(@org.apache.thrift.annotation.Nullable java.util.Map<java.lang.String,java.util.Map<java.lang.String,java.lang.String>> ssio) {
-      this.ssio = ssio;
-      return this;
-    }
-
-    public void unsetSsio() {
-      this.ssio = null;
-    }
-
-    /** Returns true if field ssio is set (has been assigned a value) and false otherwise */
-    public boolean isSetSsio() {
-      return this.ssio != null;
-    }
-
-    public void setSsioIsSet(boolean value) {
-      if (!value) {
-        this.ssio = null;
-      }
-    }
-
-    public int getAuthorizationsSize() {
-      return (this.authorizations == null) ? 0 : this.authorizations.size();
-    }
-
-    @org.apache.thrift.annotation.Nullable
-    public java.util.Iterator<java.nio.ByteBuffer> getAuthorizationsIterator() {
-      return (this.authorizations == null) ? null : this.authorizations.iterator();
-    }
-
-    public void addToAuthorizations(java.nio.ByteBuffer elem) {
-      if (this.authorizations == null) {
-        this.authorizations = new java.util.ArrayList<java.nio.ByteBuffer>();
-      }
-      this.authorizations.add(elem);
-    }
-
-    @org.apache.thrift.annotation.Nullable
-    public java.util.List<java.nio.ByteBuffer> getAuthorizations() {
-      return this.authorizations;
-    }
-
-    public startMultiScan_args setAuthorizations(@org.apache.thrift.annotation.Nullable java.util.List<java.nio.ByteBuffer> authorizations) {
-      this.authorizations = authorizations;
-      return this;
-    }
-
-    public void unsetAuthorizations() {
-      this.authorizations = null;
-    }
-
-    /** Returns true if field authorizations is set (has been assigned a value) and false otherwise */
-    public boolean isSetAuthorizations() {
-      return this.authorizations != null;
-    }
-
-    public void setAuthorizationsIsSet(boolean value) {
-      if (!value) {
-        this.authorizations = null;
-      }
-    }
-
-    public boolean isWaitForWrites() {
-      return this.waitForWrites;
-    }
-
-    public startMultiScan_args setWaitForWrites(boolean waitForWrites) {
-      this.waitForWrites = waitForWrites;
-      setWaitForWritesIsSet(true);
-      return this;
-    }
-
-    public void unsetWaitForWrites() {
-      __isset_bitfield = org.apache.thrift.EncodingUtils.clearBit(__isset_bitfield, __WAITFORWRITES_ISSET_ID);
-    }
-
-    /** Returns true if field waitForWrites is set (has been assigned a value) and false otherwise */
-    public boolean isSetWaitForWrites() {
-      return org.apache.thrift.EncodingUtils.testBit(__isset_bitfield, __WAITFORWRITES_ISSET_ID);
-    }
-
-    public void setWaitForWritesIsSet(boolean value) {
-      __isset_bitfield = org.apache.thrift.EncodingUtils.setBit(__isset_bitfield, __WAITFORWRITES_ISSET_ID, value);
-    }
-
-    @org.apache.thrift.annotation.Nullable
-    public TSamplerConfiguration getSamplerConfig() {
-      return this.samplerConfig;
-    }
-
-    public startMultiScan_args setSamplerConfig(@org.apache.thrift.annotation.Nullable TSamplerConfiguration samplerConfig) {
-      this.samplerConfig = samplerConfig;
-      return this;
-    }
-
-    public void unsetSamplerConfig() {
-      this.samplerConfig = null;
-    }
-
-    /** Returns true if field samplerConfig is set (has been assigned a value) and false otherwise */
-    public boolean isSetSamplerConfig() {
-      return this.samplerConfig != null;
-    }
-
-    public void setSamplerConfigIsSet(boolean value) {
-      if (!value) {
-        this.samplerConfig = null;
-      }
-    }
-
-    public long getBatchTimeOut() {
-      return this.batchTimeOut;
-    }
-
-    public startMultiScan_args setBatchTimeOut(long batchTimeOut) {
-      this.batchTimeOut = batchTimeOut;
-      setBatchTimeOutIsSet(true);
-      return this;
-    }
-
-    public void unsetBatchTimeOut() {
-      __isset_bitfield = org.apache.thrift.EncodingUtils.clearBit(__isset_bitfield, __BATCHTIMEOUT_ISSET_ID);
-    }
-
-    /** Returns true if field batchTimeOut is set (has been assigned a value) and false otherwise */
-    public boolean isSetBatchTimeOut() {
-      return org.apache.thrift.EncodingUtils.testBit(__isset_bitfield, __BATCHTIMEOUT_ISSET_ID);
-    }
-
-    public void setBatchTimeOutIsSet(boolean value) {
-      __isset_bitfield = org.apache.thrift.EncodingUtils.setBit(__isset_bitfield, __BATCHTIMEOUT_ISSET_ID, value);
-    }
-
-    @org.apache.thrift.annotation.Nullable
-    public java.lang.String getClassLoaderContext() {
-      return this.classLoaderContext;
-    }
-
-    public startMultiScan_args setClassLoaderContext(@org.apache.thrift.annotation.Nullable java.lang.String classLoaderContext) {
-      this.classLoaderContext = classLoaderContext;
-      return this;
-    }
-
-    public void unsetClassLoaderContext() {
-      this.classLoaderContext = null;
-    }
-
-    /** Returns true if field classLoaderContext is set (has been assigned a value) and false otherwise */
-    public boolean isSetClassLoaderContext() {
-      return this.classLoaderContext != null;
-    }
-
-    public void setClassLoaderContextIsSet(boolean value) {
-      if (!value) {
-        this.classLoaderContext = null;
-      }
-    }
-
-    public int getExecutionHintsSize() {
-      return (this.executionHints == null) ? 0 : this.executionHints.size();
-    }
-
-    public void putToExecutionHints(java.lang.String key, java.lang.String val) {
-      if (this.executionHints == null) {
-        this.executionHints = new java.util.HashMap<java.lang.String,java.lang.String>();
-      }
-      this.executionHints.put(key, val);
-    }
-
-    @org.apache.thrift.annotation.Nullable
-    public java.util.Map<java.lang.String,java.lang.String> getExecutionHints() {
-      return this.executionHints;
-    }
-
-    public startMultiScan_args setExecutionHints(@org.apache.thrift.annotation.Nullable java.util.Map<java.lang.String,java.lang.String> executionHints) {
-      this.executionHints = executionHints;
-      return this;
-    }
-
-    public void unsetExecutionHints() {
-      this.executionHints = null;
-    }
-
-    /** Returns true if field executionHints is set (has been assigned a value) and false otherwise */
-    public boolean isSetExecutionHints() {
-      return this.executionHints != null;
-    }
-
-    public void setExecutionHintsIsSet(boolean value) {
-      if (!value) {
-        this.executionHints = null;
-      }
-    }
-
-    public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable java.lang.Object value) {
-      switch (field) {
-      case TINFO:
-        if (value == null) {
-          unsetTinfo();
-        } else {
-          setTinfo((org.apache.accumulo.core.trace.thrift.TInfo)value);
-        }
-        break;
-
-      case CREDENTIALS:
-        if (value == null) {
-          unsetCredentials();
-        } else {
-          setCredentials((org.apache.accumulo.core.securityImpl.thrift.TCredentials)value);
-        }
-        break;
-
-      case BATCH:
-        if (value == null) {
-          unsetBatch();
-        } else {
-          setBatch((java.util.Map<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent,java.util.List<org.apache.accumulo.core.dataImpl.thrift.TRange>>)value);
-        }
-        break;
-
-      case COLUMNS:
-        if (value == null) {
-          unsetColumns();
-        } else {
-          setColumns((java.util.List<org.apache.accumulo.core.dataImpl.thrift.TColumn>)value);
-        }
-        break;
-
-      case SSI_LIST:
-        if (value == null) {
-          unsetSsiList();
-        } else {
-          setSsiList((java.util.List<org.apache.accumulo.core.dataImpl.thrift.IterInfo>)value);
-        }
-        break;
-
-      case SSIO:
-        if (value == null) {
-          unsetSsio();
-        } else {
-          setSsio((java.util.Map<java.lang.String,java.util.Map<java.lang.String,java.lang.String>>)value);
-        }
-        break;
-
-      case AUTHORIZATIONS:
-        if (value == null) {
-          unsetAuthorizations();
-        } else {
-          setAuthorizations((java.util.List<java.nio.ByteBuffer>)value);
-        }
-        break;
-
-      case WAIT_FOR_WRITES:
-        if (value == null) {
-          unsetWaitForWrites();
-        } else {
-          setWaitForWrites((java.lang.Boolean)value);
-        }
-        break;
-
-      case SAMPLER_CONFIG:
-        if (value == null) {
-          unsetSamplerConfig();
-        } else {
-          setSamplerConfig((TSamplerConfiguration)value);
-        }
-        break;
-
-      case BATCH_TIME_OUT:
-        if (value == null) {
-          unsetBatchTimeOut();
-        } else {
-          setBatchTimeOut((java.lang.Long)value);
-        }
-        break;
-
-      case CLASS_LOADER_CONTEXT:
-        if (value == null) {
-          unsetClassLoaderContext();
-        } else {
-          setClassLoaderContext((java.lang.String)value);
-        }
-        break;
-
-      case EXECUTION_HINTS:
-        if (value == null) {
-          unsetExecutionHints();
-        } else {
-          setExecutionHints((java.util.Map<java.lang.String,java.lang.String>)value);
-        }
-        break;
-
-      }
-    }
-
-    @org.apache.thrift.annotation.Nullable
-    public java.lang.Object getFieldValue(_Fields field) {
-      switch (field) {
-      case TINFO:
-        return getTinfo();
-
-      case CREDENTIALS:
-        return getCredentials();
-
-      case BATCH:
-        return getBatch();
-
-      case COLUMNS:
-        return getColumns();
-
-      case SSI_LIST:
-        return getSsiList();
-
-      case SSIO:
-        return getSsio();
-
-      case AUTHORIZATIONS:
-        return getAuthorizations();
-
-      case WAIT_FOR_WRITES:
-        return isWaitForWrites();
-
-      case SAMPLER_CONFIG:
-        return getSamplerConfig();
-
-      case BATCH_TIME_OUT:
-        return getBatchTimeOut();
-
-      case CLASS_LOADER_CONTEXT:
-        return getClassLoaderContext();
-
-      case EXECUTION_HINTS:
-        return getExecutionHints();
-
-      }
-      throw new java.lang.IllegalStateException();
-    }
-
-    /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
-    public boolean isSet(_Fields field) {
-      if (field == null) {
-        throw new java.lang.IllegalArgumentException();
-      }
-
-      switch (field) {
-      case TINFO:
-        return isSetTinfo();
-      case CREDENTIALS:
-        return isSetCredentials();
-      case BATCH:
-        return isSetBatch();
-      case COLUMNS:
-        return isSetColumns();
-      case SSI_LIST:
-        return isSetSsiList();
-      case SSIO:
-        return isSetSsio();
-      case AUTHORIZATIONS:
-        return isSetAuthorizations();
-      case WAIT_FOR_WRITES:
-        return isSetWaitForWrites();
-      case SAMPLER_CONFIG:
-        return isSetSamplerConfig();
-      case BATCH_TIME_OUT:
-        return isSetBatchTimeOut();
-      case CLASS_LOADER_CONTEXT:
-        return isSetClassLoaderContext();
-      case EXECUTION_HINTS:
-        return isSetExecutionHints();
-      }
-      throw new java.lang.IllegalStateException();
-    }
-
-    @Override
-    public boolean equals(java.lang.Object that) {
-      if (that instanceof startMultiScan_args)
-        return this.equals((startMultiScan_args)that);
-      return false;
-    }
-
-    public boolean equals(startMultiScan_args that) {
-      if (that == null)
-        return false;
-      if (this == that)
-        return true;
-
-      boolean this_present_tinfo = true && this.isSetTinfo();
-      boolean that_present_tinfo = true && that.isSetTinfo();
-      if (this_present_tinfo || that_present_tinfo) {
-        if (!(this_present_tinfo && that_present_tinfo))
-          return false;
-        if (!this.tinfo.equals(that.tinfo))
-          return false;
-      }
-
-      boolean this_present_credentials = true && this.isSetCredentials();
-      boolean that_present_credentials = true && that.isSetCredentials();
-      if (this_present_credentials || that_present_credentials) {
-        if (!(this_present_credentials && that_present_credentials))
-          return false;
-        if (!this.credentials.equals(that.credentials))
-          return false;
-      }
-
-      boolean this_present_batch = true && this.isSetBatch();
-      boolean that_present_batch = true && that.isSetBatch();
-      if (this_present_batch || that_present_batch) {
-        if (!(this_present_batch && that_present_batch))
-          return false;
-        if (!this.batch.equals(that.batch))
-          return false;
-      }
-
-      boolean this_present_columns = true && this.isSetColumns();
-      boolean that_present_columns = true && that.isSetColumns();
-      if (this_present_columns || that_present_columns) {
-        if (!(this_present_columns && that_present_columns))
-          return false;
-        if (!this.columns.equals(that.columns))
-          return false;
-      }
-
-      boolean this_present_ssiList = true && this.isSetSsiList();
-      boolean that_present_ssiList = true && that.isSetSsiList();
-      if (this_present_ssiList || that_present_ssiList) {
-        if (!(this_present_ssiList && that_present_ssiList))
-          return false;
-        if (!this.ssiList.equals(that.ssiList))
-          return false;
-      }
-
-      boolean this_present_ssio = true && this.isSetSsio();
-      boolean that_present_ssio = true && that.isSetSsio();
-      if (this_present_ssio || that_present_ssio) {
-        if (!(this_present_ssio && that_present_ssio))
-          return false;
-        if (!this.ssio.equals(that.ssio))
-          return false;
-      }
-
-      boolean this_present_authorizations = true && this.isSetAuthorizations();
-      boolean that_present_authorizations = true && that.isSetAuthorizations();
-      if (this_present_authorizations || that_present_authorizations) {
-        if (!(this_present_authorizations && that_present_authorizations))
-          return false;
-        if (!this.authorizations.equals(that.authorizations))
-          return false;
-      }
-
-      boolean this_present_waitForWrites = true;
-      boolean that_present_waitForWrites = true;
-      if (this_present_waitForWrites || that_present_waitForWrites) {
-        if (!(this_present_waitForWrites && that_present_waitForWrites))
-          return false;
-        if (this.waitForWrites != that.waitForWrites)
-          return false;
-      }
-
-      boolean this_present_samplerConfig = true && this.isSetSamplerConfig();
-      boolean that_present_samplerConfig = true && that.isSetSamplerConfig();
-      if (this_present_samplerConfig || that_present_samplerConfig) {
-        if (!(this_present_samplerConfig && that_present_samplerConfig))
-          return false;
-        if (!this.samplerConfig.equals(that.samplerConfig))
-          return false;
-      }
-
-      boolean this_present_batchTimeOut = true;
-      boolean that_present_batchTimeOut = true;
-      if (this_present_batchTimeOut || that_present_batchTimeOut) {
-        if (!(this_present_batchTimeOut && that_present_batchTimeOut))
-          return false;
-        if (this.batchTimeOut != that.batchTimeOut)
-          return false;
-      }
-
-      boolean this_present_classLoaderContext = true && this.isSetClassLoaderContext();
-      boolean that_present_classLoaderContext = true && that.isSetClassLoaderContext();
-      if (this_present_classLoaderContext || that_present_classLoaderContext) {
-        if (!(this_present_classLoaderContext && that_present_classLoaderContext))
-          return false;
-        if (!this.classLoaderContext.equals(that.classLoaderContext))
-          return false;
-      }
-
-      boolean this_present_executionHints = true && this.isSetExecutionHints();
-      boolean that_present_executionHints = true && that.isSetExecutionHints();
-      if (this_present_executionHints || that_present_executionHints) {
-        if (!(this_present_executionHints && that_present_executionHints))
-          return false;
-        if (!this.executionHints.equals(that.executionHints))
-          return false;
-      }
-
-      return true;
-    }
-
-    @Override
-    public int hashCode() {
-      int hashCode = 1;
-
-      hashCode = hashCode * 8191 + ((isSetTinfo()) ? 131071 : 524287);
-      if (isSetTinfo())
-        hashCode = hashCode * 8191 + tinfo.hashCode();
-
-      hashCode = hashCode * 8191 + ((isSetCredentials()) ? 131071 : 524287);
-      if (isSetCredentials())
-        hashCode = hashCode * 8191 + credentials.hashCode();
-
-      hashCode = hashCode * 8191 + ((isSetBatch()) ? 131071 : 524287);
-      if (isSetBatch())
-        hashCode = hashCode * 8191 + batch.hashCode();
-
-      hashCode = hashCode * 8191 + ((isSetColumns()) ? 131071 : 524287);
-      if (isSetColumns())
-        hashCode = hashCode * 8191 + columns.hashCode();
-
-      hashCode = hashCode * 8191 + ((isSetSsiList()) ? 131071 : 524287);
-      if (isSetSsiList())
-        hashCode = hashCode * 8191 + ssiList.hashCode();
-
-      hashCode = hashCode * 8191 + ((isSetSsio()) ? 131071 : 524287);
-      if (isSetSsio())
-        hashCode = hashCode * 8191 + ssio.hashCode();
-
-      hashCode = hashCode * 8191 + ((isSetAuthorizations()) ? 131071 : 524287);
-      if (isSetAuthorizations())
-        hashCode = hashCode * 8191 + authorizations.hashCode();
-
-      hashCode = hashCode * 8191 + ((waitForWrites) ? 131071 : 524287);
-
-      hashCode = hashCode * 8191 + ((isSetSamplerConfig()) ? 131071 : 524287);
-      if (isSetSamplerConfig())
-        hashCode = hashCode * 8191 + samplerConfig.hashCode();
-
-      hashCode = hashCode * 8191 + org.apache.thrift.TBaseHelper.hashCode(batchTimeOut);
-
-      hashCode = hashCode * 8191 + ((isSetClassLoaderContext()) ? 131071 : 524287);
-      if (isSetClassLoaderContext())
-        hashCode = hashCode * 8191 + classLoaderContext.hashCode();
-
-      hashCode = hashCode * 8191 + ((isSetExecutionHints()) ? 131071 : 524287);
-      if (isSetExecutionHints())
-        hashCode = hashCode * 8191 + executionHints.hashCode();
-
-      return hashCode;
-    }
-
-    @Override
-    public int compareTo(startMultiScan_args other) {
-      if (!getClass().equals(other.getClass())) {
-        return getClass().getName().compareTo(other.getClass().getName());
-      }
-
-      int lastComparison = 0;
-
-      lastComparison = java.lang.Boolean.compare(isSetTinfo(), other.isSetTinfo());
-      if (lastComparison != 0) {
-        return lastComparison;
-      }
-      if (isSetTinfo()) {
-        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tinfo, other.tinfo);
-        if (lastComparison != 0) {
-          return lastComparison;
-        }
-      }
-      lastComparison = java.lang.Boolean.compare(isSetCredentials(), 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.compare(isSetBatch(), other.isSetBatch());
-      if (lastComparison != 0) {
-        return lastComparison;
-      }
-      if (isSetBatch()) {
-        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.batch, other.batch);
-        if (lastComparison != 0) {
-          return lastComparison;
-        }
-      }
-      lastComparison = java.lang.Boolean.compare(isSetColumns(), other.isSetColumns());
-      if (lastComparison != 0) {
-        return lastComparison;
-      }
-      if (isSetColumns()) {
-        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.columns, other.columns);
-        if (lastComparison != 0) {
-          return lastComparison;
-        }
-      }
-      lastComparison = java.lang.Boolean.compare(isSetSsiList(), other.isSetSsiList());
-      if (lastComparison != 0) {
-        return lastComparison;
-      }
-      if (isSetSsiList()) {
-        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.ssiList, other.ssiList);
-        if (lastComparison != 0) {
-          return lastComparison;
-        }
-      }
-      lastComparison = java.lang.Boolean.compare(isSetSsio(), other.isSetSsio());
-      if (lastComparison != 0) {
-        return lastComparison;
-      }
-      if (isSetSsio()) {
-        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.ssio, other.ssio);
-        if (lastComparison != 0) {
-          return lastComparison;
-        }
-      }
-      lastComparison = java.lang.Boolean.compare(isSetAuthorizations(), other.isSetAuthorizations());
-      if (lastComparison != 0) {
-        return lastComparison;
-      }
-      if (isSetAuthorizations()) {
-        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.authorizations, other.authorizations);
-        if (lastComparison != 0) {
-          return lastComparison;
-        }
-      }
-      lastComparison = java.lang.Boolean.compare(isSetWaitForWrites(), other.isSetWaitForWrites());
-      if (lastComparison != 0) {
-        return lastComparison;
-      }
-      if (isSetWaitForWrites()) {
-        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.waitForWrites, other.waitForWrites);
-        if (lastComparison != 0) {
-          return lastComparison;
-        }
-      }
-      lastComparison = java.lang.Boolean.compare(isSetSamplerConfig(), other.isSetSamplerConfig());
-      if (lastComparison != 0) {
-        return lastComparison;
-      }
-      if (isSetSamplerConfig()) {
-        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.samplerConfig, other.samplerConfig);
-        if (lastComparison != 0) {
-          return lastComparison;
-        }
-      }
-      lastComparison = java.lang.Boolean.compare(isSetBatchTimeOut(), other.isSetBatchTimeOut());
-      if (lastComparison != 0) {
-        return lastComparison;
-      }
-      if (isSetBatchTimeOut()) {
-        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.batchTimeOut, other.batchTimeOut);
-        if (lastComparison != 0) {
-          return lastComparison;
-        }
-      }
-      lastComparison = java.lang.Boolean.compare(isSetClassLoaderContext(), other.isSetClassLoaderContext());
-      if (lastComparison != 0) {
-        return lastComparison;
-      }
-      if (isSetClassLoaderContext()) {
-        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.classLoaderContext, other.classLoaderContext);
-        if (lastComparison != 0) {
-          return lastComparison;
-        }
-      }
-      lastComparison = java.lang.Boolean.compare(isSetExecutionHints(), other.isSetExecutionHints());
-      if (lastComparison != 0) {
-        return lastComparison;
-      }
-      if (isSetExecutionHints()) {
-        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.executionHints, other.executionHints);
-        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("startMultiScan_args(");
-      boolean first = true;
-
-      sb.append("tinfo:");
-      if (this.tinfo == null) {
-        sb.append("null");
-      } else {
-        sb.append(this.tinfo);
-      }
-      first = false;
-      if (!first) sb.append(", ");
-      sb.append("credentials:");
-      if (this.credentials == null) {
-        sb.append("null");
-      } else {
-        sb.append(this.credentials);
-      }
-      first = false;
-      if (!first) sb.append(", ");
-      sb.append("batch:");
-      if (this.batch == null) {
-        sb.append("null");
-      } else {
-        sb.append(this.batch);
-      }
-      first = false;
-      if (!first) sb.append(", ");
-      sb.append("columns:");
-      if (this.columns == null) {
-        sb.append("null");
-      } else {
-        sb.append(this.columns);
-      }
-      first = false;
-      if (!first) sb.append(", ");
-      sb.append("ssiList:");
-      if (this.ssiList == null) {
-        sb.append("null");
-      } else {
-        sb.append(this.ssiList);
-      }
-      first = false;
-      if (!first) sb.append(", ");
-      sb.append("ssio:");
-      if (this.ssio == null) {
-        sb.append("null");
-      } else {
-        sb.append(this.ssio);
-      }
-      first = false;
-      if (!first) sb.append(", ");
-      sb.append("authorizations:");
-      if (this.authorizations == null) {
-        sb.append("null");
-      } else {
-        org.apache.thrift.TBaseHelper.toString(this.authorizations, sb);
-      }
-      first = false;
-      if (!first) sb.append(", ");
-      sb.append("waitForWrites:");
-      sb.append(this.waitForWrites);
-      first = false;
-      if (!first) sb.append(", ");
-      sb.append("samplerConfig:");
-      if (this.samplerConfig == null) {
-        sb.append("null");
-      } else {
-        sb.append(this.samplerConfig);
-      }
-      first = false;
-      if (!first) sb.append(", ");
-      sb.append("batchTimeOut:");
-      sb.append(this.batchTimeOut);
-      first = false;
-      if (!first) sb.append(", ");
-      sb.append("classLoaderContext:");
-      if (this.classLoaderContext == null) {
-        sb.append("null");
-      } else {
-        sb.append(this.classLoaderContext);
-      }
-      first = false;
-      if (!first) sb.append(", ");
-      sb.append("executionHints:");
-      if (this.executionHints == null) {
-        sb.append("null");
-      } else {
-        sb.append(this.executionHints);
-      }
-      first = false;
-      sb.append(")");
-      return sb.toString();
-    }
-
-    public void validate() throws org.apache.thrift.TException {
-      // check for required fields
-      // check for sub-struct validity
-      if (tinfo != null) {
-        tinfo.validate();
-      }
-      if (credentials != null) {
-        credentials.validate();
-      }
-      if (samplerConfig != null) {
-        samplerConfig.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 startMultiScan_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      public startMultiScan_argsStandardScheme getScheme() {
-        return new startMultiScan_argsStandardScheme();
-      }
-    }
-
-    private static class startMultiScan_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme<startMultiScan_args> {
-
-      public void read(org.apache.thrift.protocol.TProtocol iprot, startMultiScan_args struct) throws org.apache.thrift.TException {
-        org.apache.thrift.protocol.TField schemeField;
-        iprot.readStructBegin();
-        while (true)
-        {
-          schemeField = iprot.readFieldBegin();
-          if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
-            break;
-          }
-          switch (schemeField.id) {
-            case 8: // TINFO
-              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
-                struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
-                struct.tinfo.read(iprot);
-                struct.setTinfoIsSet(true);
-              } else { 
-                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
-              }
-              break;
-            case 1: // 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 2: // BATCH
-              if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
-                {
-                  org.apache.thrift.protocol.TMap _map188 = 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*_map188.size);
-                  @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TKeyExtent _key189;
-                  @org.apache.thrift.annotation.Nullable java.util.List<org.apache.accumulo.core.dataImpl.thrift.TRange> _val190;
-                  for (int _i191 = 0; _i191 < _map188.size; ++_i191)
-                  {
-                    _key189 = new org.apache.accumulo.core.dataImpl.thrift.TKeyExtent();
-                    _key189.read(iprot);
-                    {
-                      org.apache.thrift.protocol.TList _list192 = iprot.readListBegin();
-                      _val190 = new java.util.ArrayList<org.apache.accumulo.core.dataImpl.thrift.TRange>(_list192.size);
-                      @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TRange _elem193;
-                      for (int _i194 = 0; _i194 < _list192.size; ++_i194)
-                      {
-                        _elem193 = new org.apache.accumulo.core.dataImpl.thrift.TRange();
-                        _elem193.read(iprot);
-                        _val190.add(_elem193);
-                      }
-                      iprot.readListEnd();
-                    }
-                    struct.batch.put(_key189, _val190);
-                  }
-                  iprot.readMapEnd();
-                }
-                struct.setBatchIsSet(true);
-              } else { 
-                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
-              }
-              break;
-            case 3: // COLUMNS
-              if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
-                {
-                  org.apache.thrift.protocol.TList _list195 = iprot.readListBegin();
-                  struct.columns = new java.util.ArrayList<org.apache.accumulo.core.dataImpl.thrift.TColumn>(_list195.size);
-                  @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TColumn _elem196;
-                  for (int _i197 = 0; _i197 < _list195.size; ++_i197)
-                  {
-                    _elem196 = new org.apache.accumulo.core.dataImpl.thrift.TColumn();
-                    _elem196.read(iprot);
-                    struct.columns.add(_elem196);
-                  }
-                  iprot.readListEnd();
-                }
-                struct.setColumnsIsSet(true);
-              } else { 
-                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
-              }
-              break;
-            case 4: // SSI_LIST
-              if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
-                {
-                  org.apache.thrift.protocol.TList _list198 = iprot.readListBegin();
-                  struct.ssiList = new java.util.ArrayList<org.apache.accumulo.core.dataImpl.thrift.IterInfo>(_list198.size);
-                  @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.IterInfo _elem199;
-                  for (int _i200 = 0; _i200 < _list198.size; ++_i200)
-                  {
-                    _elem199 = new org.apache.accumulo.core.dataImpl.thrift.IterInfo();
-                    _elem199.read(iprot);
-                    struct.ssiList.add(_elem199);
-                  }
-                  iprot.readListEnd();
-                }
-                struct.setSsiListIsSet(true);
-              } else { 
-                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
-              }
-              break;
-            case 5: // SSIO
-              if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
-                {
-                  org.apache.thrift.protocol.TMap _map201 = iprot.readMapBegin();
-                  struct.ssio = new java.util.HashMap<java.lang.String,java.util.Map<java.lang.String,java.lang.String>>(2*_map201.size);
-                  @org.apache.thrift.annotation.Nullable java.lang.String _key202;
-                  @org.apache.thrift.annotation.Nullable java.util.Map<java.lang.String,java.lang.String> _val203;
-                  for (int _i204 = 0; _i204 < _map201.size; ++_i204)
-                  {
-                    _key202 = iprot.readString();
-                    {
-                      org.apache.thrift.protocol.TMap _map205 = iprot.readMapBegin();
-                      _val203 = new java.util.HashMap<java.lang.String,java.lang.String>(2*_map205.size);
-                      @org.apache.thrift.annotation.Nullable java.lang.String _key206;
-                      @org.apache.thrift.annotation.Nullable java.lang.String _val207;
-                      for (int _i208 = 0; _i208 < _map205.size; ++_i208)
-                      {
-                        _key206 = iprot.readString();
-                        _val207 = iprot.readString();
-                        _val203.put(_key206, _val207);
-                      }
-                      iprot.readMapEnd();
-                    }
-                    struct.ssio.put(_key202, _val203);
-                  }
-                  iprot.readMapEnd();
-                }
-                struct.setSsioIsSet(true);
-              } else { 
-                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
-              }
-              break;
-            case 6: // AUTHORIZATIONS
-              if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
-                {
-                  org.apache.thrift.protocol.TList _list209 = iprot.readListBegin();
-                  struct.authorizations = new java.util.ArrayList<java.nio.ByteBuffer>(_list209.size);
-                  @org.apache.thrift.annotation.Nullable java.nio.ByteBuffer _elem210;
-                  for (int _i211 = 0; _i211 < _list209.size; ++_i211)
-                  {
-                    _elem210 = iprot.readBinary();
-                    struct.authorizations.add(_elem210);
-                  }
-                  iprot.readListEnd();
-                }
-                struct.setAuthorizationsIsSet(true);
-              } else { 
-                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
-              }
-              break;
-            case 7: // WAIT_FOR_WRITES
-              if (schemeField.type == org.apache.thrift.protocol.TType.BOOL) {
-                struct.waitForWrites = iprot.readBool();
-                struct.setWaitForWritesIsSet(true);
-              } else { 
-                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
-              }
-              break;
-            case 9: // SAMPLER_CONFIG
-              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
-                struct.samplerConfig = new TSamplerConfiguration();
-                struct.samplerConfig.read(iprot);
-                struct.setSamplerConfigIsSet(true);
-              } else { 
-                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
-              }
-              break;
-            case 10: // BATCH_TIME_OUT
-              if (schemeField.type == org.apache.thrift.protocol.TType.I64) {
-                struct.batchTimeOut = iprot.readI64();
-                struct.setBatchTimeOutIsSet(true);
-              } else { 
-                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
-              }
-              break;
-            case 11: // CLASS_LOADER_CONTEXT
-              if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
-                struct.classLoaderContext = iprot.readString();
-                struct.setClassLoaderContextIsSet(true);
-              } else { 
-                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
... 38654 lines suppressed ...