You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@accumulo.apache.org by GitBox <gi...@apache.org> on 2018/08/02 17:23:57 UTC

[GitHub] keith-turner closed pull request #571: Add scanner execution hints #555

keith-turner closed pull request #571: Add scanner execution hints #555
URL: https://github.com/apache/accumulo/pull/571
 
 
   

This is a PR merged from a forked repository.
As GitHub hides the original diff on merge, it is displayed below for
the sake of provenance:

As this is a foreign pull request (from a fork), the diff is supplied
below (as it won't show otherwise due to GitHub magic):

diff --git a/client/mapreduce/src/main/java/org/apache/accumulo/core/client/mapred/AbstractInputFormat.java b/client/mapreduce/src/main/java/org/apache/accumulo/core/client/mapred/AbstractInputFormat.java
index 1f849d4272..c4d08bed98 100644
--- a/client/mapreduce/src/main/java/org/apache/accumulo/core/client/mapred/AbstractInputFormat.java
+++ b/client/mapreduce/src/main/java/org/apache/accumulo/core/client/mapred/AbstractInputFormat.java
@@ -587,6 +587,15 @@ public void initialize(InputSplit inSplit, JobConf job) throws IOException {
         scannerBase.setSamplerConfiguration(samplerConfig);
       }
 
+      Map<String,String> executionHints = baseSplit.getExecutionHints();
+      if (executionHints == null || executionHints.size() == 0) {
+        executionHints = tableConfig.getExecutionHints();
+      }
+
+      if (executionHints != null) {
+        scannerBase.setExecutionHints(executionHints);
+      }
+
       scannerIterator = scannerBase.iterator();
       numKeysRead = 0;
     }
diff --git a/client/mapreduce/src/main/java/org/apache/accumulo/core/client/mapred/InputFormatBase.java b/client/mapreduce/src/main/java/org/apache/accumulo/core/client/mapred/InputFormatBase.java
index 078a501e29..8680c1e108 100644
--- a/client/mapreduce/src/main/java/org/apache/accumulo/core/client/mapred/InputFormatBase.java
+++ b/client/mapreduce/src/main/java/org/apache/accumulo/core/client/mapred/InputFormatBase.java
@@ -19,6 +19,7 @@
 import java.io.IOException;
 import java.util.Collection;
 import java.util.List;
+import java.util.Map;
 import java.util.Set;
 
 import org.apache.accumulo.core.client.ClientSideIteratorScanner;
@@ -374,6 +375,16 @@ public static void setSamplerConfiguration(JobConf job, SamplerConfiguration sam
     InputConfigurator.setSamplerConfiguration(CLASS, job, samplerConfig);
   }
 
+  /**
+   * Set these execution hints on scanners created for input splits. See
+   * {@link ScannerBase#setExecutionHints(java.util.Map)}
+   *
+   * @since 2.0.0
+   */
+  public static void setExecutionHints(JobConf job, Map<String,String> hints) {
+    InputConfigurator.setExecutionHints(CLASS, job, hints);
+  }
+
   protected abstract static class RecordReaderBase<K,V> extends AbstractRecordReader<K,V> {
 
     @Override
diff --git a/client/mapreduce/src/main/java/org/apache/accumulo/core/client/mapreduce/AbstractInputFormat.java b/client/mapreduce/src/main/java/org/apache/accumulo/core/client/mapreduce/AbstractInputFormat.java
index 91a6e4995a..7bd4546100 100644
--- a/client/mapreduce/src/main/java/org/apache/accumulo/core/client/mapreduce/AbstractInputFormat.java
+++ b/client/mapreduce/src/main/java/org/apache/accumulo/core/client/mapreduce/AbstractInputFormat.java
@@ -595,6 +595,15 @@ public void initialize(InputSplit inSplit, TaskAttemptContext attempt) throws IO
         scannerBase.setSamplerConfiguration(samplerConfig);
       }
 
+      Map<String,String> executionHints = split.getExecutionHints();
+      if (executionHints == null || executionHints.isEmpty()) {
+        executionHints = tableConfig.getExecutionHints();
+      }
+
+      if (executionHints != null) {
+        scannerBase.setExecutionHints(executionHints);
+      }
+
       scannerIterator = scannerBase.iterator();
       numKeysRead = 0;
     }
diff --git a/client/mapreduce/src/main/java/org/apache/accumulo/core/client/mapreduce/InputFormatBase.java b/client/mapreduce/src/main/java/org/apache/accumulo/core/client/mapreduce/InputFormatBase.java
index 81c15acd3f..bb6ef71e48 100644
--- a/client/mapreduce/src/main/java/org/apache/accumulo/core/client/mapreduce/InputFormatBase.java
+++ b/client/mapreduce/src/main/java/org/apache/accumulo/core/client/mapreduce/InputFormatBase.java
@@ -19,6 +19,7 @@
 import java.io.IOException;
 import java.util.Collection;
 import java.util.List;
+import java.util.Map;
 import java.util.Set;
 
 import org.apache.accumulo.core.client.ClientSideIteratorScanner;
@@ -33,6 +34,7 @@
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.util.Pair;
 import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.mapreduce.InputFormat;
 import org.apache.hadoop.mapreduce.InputSplit;
 import org.apache.hadoop.mapreduce.Job;
@@ -373,6 +375,16 @@ public static void setSamplerConfiguration(Job job, SamplerConfiguration sampler
     InputConfigurator.setSamplerConfiguration(CLASS, job.getConfiguration(), samplerConfig);
   }
 
+  /**
+   * Set these execution hints on scanners created for input splits. See
+   * {@link ScannerBase#setExecutionHints(java.util.Map)}
+   *
+   * @since 2.0.0
+   */
+  public static void setExecutionHints(JobConf job, Map<String,String> hints) {
+    InputConfigurator.setExecutionHints(CLASS, job, hints);
+  }
+
   protected abstract static class RecordReaderBase<K,V> extends AbstractRecordReader<K,V> {
 
     @Override
diff --git a/client/mapreduce/src/main/java/org/apache/accumulo/core/client/mapreduce/InputTableConfig.java b/client/mapreduce/src/main/java/org/apache/accumulo/core/client/mapreduce/InputTableConfig.java
index 40bdac1918..7fd04e23eb 100644
--- a/client/mapreduce/src/main/java/org/apache/accumulo/core/client/mapreduce/InputTableConfig.java
+++ b/client/mapreduce/src/main/java/org/apache/accumulo/core/client/mapreduce/InputTableConfig.java
@@ -21,13 +21,18 @@
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
 import java.util.HashSet;
 import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
 
 import org.apache.accumulo.core.client.IteratorSetting;
 import org.apache.accumulo.core.client.ScannerBase;
 import org.apache.accumulo.core.client.sample.SamplerConfiguration;
 import org.apache.accumulo.core.data.Range;
+import org.apache.accumulo.core.sample.impl.SamplerConfigurationImpl;
 import org.apache.accumulo.core.util.Pair;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.Writable;
@@ -47,6 +52,7 @@
   private boolean useIsolatedScanners = false;
   private boolean offlineScan = false;
   private SamplerConfiguration samplerConfig = null;
+  private Map<String,String> executionHints = Collections.emptyMap();
 
   public InputTableConfig() {}
 
@@ -276,6 +282,22 @@ public SamplerConfiguration getSamplerConfiguration() {
     return samplerConfig;
   }
 
+  /**
+   * The execution hints to set on created scanners. See {@link ScannerBase#setExecutionHints(Map)}
+   *
+   * @since 2.0.0
+   */
+  public void setExecutionHints(Map<String,String> executionHints) {
+    this.executionHints = executionHints;
+  }
+
+  /**
+   * @since 2.0.0
+   */
+  public Map<String,String> getExecutionHints() {
+    return executionHints;
+  }
+
   @Override
   public void write(DataOutput dataOutput) throws IOException {
     if (iterators != null) {
@@ -311,6 +333,22 @@ public void write(DataOutput dataOutput) throws IOException {
     dataOutput.writeBoolean(useLocalIterators);
     dataOutput.writeBoolean(useIsolatedScanners);
     dataOutput.writeBoolean(offlineScan);
+    if (samplerConfig == null) {
+      dataOutput.writeBoolean(false);
+    } else {
+      dataOutput.writeBoolean(true);
+      new SamplerConfigurationImpl(samplerConfig).write(dataOutput);
+    }
+
+    if (executionHints == null || executionHints.size() == 0) {
+      dataOutput.writeInt(0);
+    } else {
+      dataOutput.writeInt(executionHints.size());
+      for (Entry<String,String> entry : executionHints.entrySet()) {
+        dataOutput.writeUTF(entry.getKey());
+        dataOutput.writeUTF(entry.getValue());
+      }
+    }
   }
 
   @Override
@@ -350,6 +388,18 @@ public void readFields(DataInput dataInput) throws IOException {
     useLocalIterators = dataInput.readBoolean();
     useIsolatedScanners = dataInput.readBoolean();
     offlineScan = dataInput.readBoolean();
+
+    if (dataInput.readBoolean()) {
+      samplerConfig = new SamplerConfigurationImpl(dataInput).toSamplerConfiguration();
+    }
+
+    executionHints = new HashMap<>();
+    int numHints = dataInput.readInt();
+    for (int i = 0; i < numHints; i++) {
+      String k = dataInput.readUTF();
+      String v = dataInput.readUTF();
+      executionHints.put(k, v);
+    }
   }
 
   @Override
@@ -375,6 +425,9 @@ public boolean equals(Object o) {
       return false;
     if (ranges != null ? !ranges.equals(that.ranges) : that.ranges != null)
       return false;
+    if (executionHints != null ? !executionHints.equals(that.executionHints)
+        : that.executionHints != null)
+      return false;
     return samplerConfig != null ? samplerConfig.equals(that.samplerConfig)
         : that.samplerConfig == null;
   }
@@ -389,6 +442,7 @@ public int hashCode() {
     result = 31 * result + (useIsolatedScanners ? 1 : 0);
     result = 31 * result + (offlineScan ? 1 : 0);
     result = 31 * result + (samplerConfig == null ? 0 : samplerConfig.hashCode());
+    result = 31 * result + (executionHints == null ? 0 : executionHints.hashCode());
     return result;
   }
 }
diff --git a/client/mapreduce/src/main/java/org/apache/accumulo/core/client/mapreduce/RangeInputSplit.java b/client/mapreduce/src/main/java/org/apache/accumulo/core/client/mapreduce/RangeInputSplit.java
index 95c6f85238..fca133ba45 100644
--- a/client/mapreduce/src/main/java/org/apache/accumulo/core/client/mapreduce/RangeInputSplit.java
+++ b/client/mapreduce/src/main/java/org/apache/accumulo/core/client/mapreduce/RangeInputSplit.java
@@ -22,8 +22,11 @@
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
+import java.util.HashMap;
 import java.util.HashSet;
 import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
 import java.util.Set;
 
 import org.apache.accumulo.core.client.IteratorSetting;
@@ -53,6 +56,7 @@
   private List<IteratorSetting> iterators;
   private SamplerConfiguration samplerConfig;
   private Level level;
+  private Map<String,String> executionHints;
 
   public RangeInputSplit() {
     range = new Range();
@@ -169,6 +173,14 @@ public void readFields(DataInput in) throws IOException {
     if (in.readBoolean()) {
       samplerConfig = new SamplerConfigurationImpl(in).toSamplerConfiguration();
     }
+
+    executionHints = new HashMap<>();
+    int numHints = in.readInt();
+    for (int i = 0; i < numHints; i++) {
+      String k = in.readUTF();
+      String v = in.readUTF();
+      executionHints.put(k, v);
+    }
   }
 
   @Override
@@ -221,6 +233,16 @@ public void write(DataOutput out) throws IOException {
     if (null != samplerConfig) {
       new SamplerConfigurationImpl(samplerConfig).write(out);
     }
+
+    if (executionHints == null || executionHints.size() == 0) {
+      out.writeInt(0);
+    } else {
+      out.writeInt(executionHints.size());
+      for (Entry<String,String> entry : executionHints.entrySet()) {
+        out.writeUTF(entry.getKey());
+        out.writeUTF(entry.getValue());
+      }
+    }
   }
 
   public String getTableName() {
@@ -316,6 +338,7 @@ public String toString() {
     sb.append(" iterators: ").append(iterators);
     sb.append(" logLevel: ").append(level);
     sb.append(" samplerConfig: ").append(samplerConfig);
+    sb.append(" executionHints: ").append(executionHints);
     return sb.toString();
   }
 
@@ -326,4 +349,12 @@ public void setSamplerConfiguration(SamplerConfiguration samplerConfiguration) {
   public SamplerConfiguration getSamplerConfiguration() {
     return samplerConfig;
   }
+
+  public void setExecutionHints(Map<String,String> executionHints) {
+    this.executionHints = executionHints;
+  }
+
+  public Map<String,String> getExecutionHints() {
+    return executionHints;
+  }
 }
diff --git a/client/mapreduce/src/main/java/org/apache/accumulo/core/client/mapreduce/impl/SplitUtils.java b/client/mapreduce/src/main/java/org/apache/accumulo/core/client/mapreduce/impl/SplitUtils.java
index d49e3fef0d..c2357ff1c2 100644
--- a/client/mapreduce/src/main/java/org/apache/accumulo/core/client/mapreduce/impl/SplitUtils.java
+++ b/client/mapreduce/src/main/java/org/apache/accumulo/core/client/mapreduce/impl/SplitUtils.java
@@ -39,6 +39,7 @@ public static void updateSplit(RangeInputSplit split, InputTableConfig tableConf
     split.setIterators(tableConfig.getIterators());
     split.setLogLevel(logLevel);
     split.setSamplerConfiguration(tableConfig.getSamplerConfiguration());
+    split.setExecutionHints(tableConfig.getExecutionHints());
   }
 
   public static float getProgress(ByteSequence start, ByteSequence end, ByteSequence position) {
diff --git a/client/mapreduce/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/impl/InputConfigurator.java b/client/mapreduce/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/impl/InputConfigurator.java
index 0253138b18..a6913ef327 100644
--- a/client/mapreduce/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/impl/InputConfigurator.java
+++ b/client/mapreduce/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/impl/InputConfigurator.java
@@ -28,6 +28,7 @@
 import java.util.ArrayList;
 import java.util.Base64;
 import java.util.Collection;
+import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Iterator;
@@ -94,7 +95,8 @@
     ITERATORS,
     TABLE_CONFIGS,
     SAMPLER_CONFIG,
-    CLASSLOADER_CONTEXT
+    CLASSLOADER_CONTEXT,
+    EXECUTION_HINTS
   }
 
   /**
@@ -827,7 +829,8 @@ public static void validatePermissions(Class<?> implementingClass, Configuration
       queryConfig.setAutoAdjustRanges(getAutoAdjustRanges(implementingClass, conf))
           .setUseIsolatedScanners(isIsolated(implementingClass, conf))
           .setUseLocalIterators(usesLocalIterators(implementingClass, conf))
-          .setOfflineScan(isOfflineScan(implementingClass, conf));
+          .setOfflineScan(isOfflineScan(implementingClass, conf))
+          .setExecutionHints(getExecutionHints(implementingClass, conf));
       return Maps.immutableEntry(tableName, queryConfig);
     }
     return null;
@@ -975,4 +978,31 @@ public static SamplerConfiguration getSamplerConfiguration(Class<?> implementing
 
     return fromBase64(new SamplerConfigurationImpl(), encodedSC).toSamplerConfiguration();
   }
+
+  public static void setExecutionHints(Class<?> implementingClass, Configuration conf,
+      Map<String,String> hints) {
+    MapWritable mapWritable = new MapWritable();
+    hints.forEach((k, v) -> mapWritable.put(new Text(k), new Text(v)));
+
+    String key = enumToConfKey(implementingClass, ScanOpts.EXECUTION_HINTS);
+    String val = toBase64(mapWritable);
+
+    conf.set(key, val);
+  }
+
+  public static Map<String,String> getExecutionHints(Class<?> implementingClass,
+      Configuration conf) {
+    String key = enumToConfKey(implementingClass, ScanOpts.EXECUTION_HINTS);
+    String encodedEH = conf.get(key);
+    if (encodedEH == null) {
+      return Collections.emptyMap();
+    }
+
+    MapWritable mapWritable = new MapWritable();
+    fromBase64(mapWritable, encodedEH);
+
+    HashMap<String,String> hints = new HashMap<>();
+    mapWritable.forEach((k, v) -> hints.put(k.toString(), v.toString()));
+    return hints;
+  }
 }
diff --git a/client/mapreduce/src/test/java/org/apache/accumulo/core/client/mapred/RangeInputSplitTest.java b/client/mapreduce/src/test/java/org/apache/accumulo/core/client/mapred/RangeInputSplitTest.java
index 24eeeaa336..c36511d582 100644
--- a/client/mapreduce/src/test/java/org/apache/accumulo/core/client/mapred/RangeInputSplitTest.java
+++ b/client/mapreduce/src/test/java/org/apache/accumulo/core/client/mapred/RangeInputSplitTest.java
@@ -37,6 +37,8 @@
 import org.junit.Assert;
 import org.junit.Test;
 
+import com.google.common.collect.ImmutableMap;
+
 public class RangeInputSplitTest {
 
   @Test
@@ -84,6 +86,7 @@ public void testAllFieldsWritable() throws IOException {
     split.setFetchedColumns(fetchedColumns);
     split.setIterators(iterators);
     split.setLogLevel(Level.WARN);
+    split.setExecutionHints(ImmutableMap.of("priority", "9"));
 
     ByteArrayOutputStream baos = new ByteArrayOutputStream();
     DataOutputStream dos = new DataOutputStream(baos);
@@ -104,6 +107,7 @@ public void testAllFieldsWritable() throws IOException {
     Assert.assertEquals(split.getFetchedColumns(), newSplit.getFetchedColumns());
     Assert.assertEquals(split.getIterators(), newSplit.getIterators());
     Assert.assertEquals(split.getLogLevel(), newSplit.getLogLevel());
+    Assert.assertEquals(split.getExecutionHints(), newSplit.getExecutionHints());
   }
 
 }
diff --git a/client/mapreduce/src/test/java/org/apache/accumulo/core/client/mapreduce/InputTableConfigTest.java b/client/mapreduce/src/test/java/org/apache/accumulo/core/client/mapreduce/InputTableConfigTest.java
index 372db9d030..65519b9968 100644
--- a/client/mapreduce/src/test/java/org/apache/accumulo/core/client/mapreduce/InputTableConfigTest.java
+++ b/client/mapreduce/src/test/java/org/apache/accumulo/core/client/mapreduce/InputTableConfigTest.java
@@ -29,12 +29,15 @@
 import java.util.Set;
 
 import org.apache.accumulo.core.client.IteratorSetting;
+import org.apache.accumulo.core.client.sample.SamplerConfiguration;
 import org.apache.accumulo.core.data.Range;
 import org.apache.accumulo.core.util.Pair;
 import org.apache.hadoop.io.Text;
 import org.junit.Before;
 import org.junit.Test;
 
+import com.google.common.collect.ImmutableMap;
+
 public class InputTableConfigTest {
 
   private InputTableConfig tableQueryConfig;
@@ -102,6 +105,21 @@ public void testSerialization_iterators() throws IOException {
 
   }
 
+  @Test
+  public void testSamplerConfig() throws IOException {
+    SamplerConfiguration sc = new SamplerConfiguration("com.foo.S1").addOption("k1", "v1");
+    tableQueryConfig.setSamplerConfiguration(sc);
+    InputTableConfig actualConfig = deserialize(serialize(tableQueryConfig));
+    assertEquals(sc, actualConfig.getSamplerConfiguration());
+  }
+
+  @Test
+  public void testExecutionHints() throws IOException {
+    tableQueryConfig.setExecutionHints(ImmutableMap.of("priority", "9"));
+    InputTableConfig actualConfig = deserialize(serialize(tableQueryConfig));
+    assertEquals(ImmutableMap.of("priority", "9"), actualConfig.getExecutionHints());
+  }
+
   private byte[] serialize(InputTableConfig tableQueryConfig) throws IOException {
     ByteArrayOutputStream baos = new ByteArrayOutputStream();
     tableQueryConfig.write(new DataOutputStream(baos));
diff --git a/client/mapreduce/src/test/java/org/apache/accumulo/core/client/mapreduce/RangeInputSplitTest.java b/client/mapreduce/src/test/java/org/apache/accumulo/core/client/mapreduce/RangeInputSplitTest.java
index 12a4b1984d..9d36f9d5e9 100644
--- a/client/mapreduce/src/test/java/org/apache/accumulo/core/client/mapreduce/RangeInputSplitTest.java
+++ b/client/mapreduce/src/test/java/org/apache/accumulo/core/client/mapreduce/RangeInputSplitTest.java
@@ -37,6 +37,8 @@
 import org.junit.Assert;
 import org.junit.Test;
 
+import com.google.common.collect.ImmutableMap;
+
 public class RangeInputSplitTest {
 
   @Test
@@ -87,6 +89,7 @@ public void testAllFieldsWritable() throws IOException {
     split.setFetchedColumns(fetchedColumns);
     split.setIterators(iterators);
     split.setLogLevel(Level.WARN);
+    split.setExecutionHints(ImmutableMap.of("priority", "9"));
 
     ByteArrayOutputStream baos = new ByteArrayOutputStream();
     DataOutputStream dos = new DataOutputStream(baos);
@@ -108,6 +111,7 @@ public void testAllFieldsWritable() throws IOException {
     Assert.assertEquals(split.getFetchedColumns(), newSplit.getFetchedColumns());
     Assert.assertEquals(split.getIterators(), newSplit.getIterators());
     Assert.assertEquals(split.getLogLevel(), newSplit.getLogLevel());
+    Assert.assertEquals(split.getExecutionHints(), newSplit.getExecutionHints());
   }
 
 }
diff --git a/core/src/main/java/org/apache/accumulo/core/client/ScannerBase.java b/core/src/main/java/org/apache/accumulo/core/client/ScannerBase.java
index 6fdc13a56b..207b4affad 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/ScannerBase.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/ScannerBase.java
@@ -17,6 +17,7 @@
 package org.apache.accumulo.core.client;
 
 import java.util.Iterator;
+import java.util.Map;
 import java.util.Map.Entry;
 import java.util.concurrent.TimeUnit;
 
@@ -25,6 +26,11 @@
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.security.Authorizations;
+import org.apache.accumulo.core.spi.scan.HintScanPrioritizer;
+import org.apache.accumulo.core.spi.scan.ScanDispatcher;
+import org.apache.accumulo.core.spi.scan.ScanInfo;
+import org.apache.accumulo.core.spi.scan.ScanPrioritizer;
+import org.apache.accumulo.core.spi.scan.SimpleScanDispatcher;
 import org.apache.hadoop.io.Text;
 
 /**
@@ -286,4 +292,18 @@
    * @since 1.8.0
    */
   String getClassLoaderContext();
+
+  /**
+   * Set hints for the configured {@link ScanPrioritizer} and {@link ScanDispatcher}. These hints
+   * are available on the server side via {@link ScanInfo#getExecutionHints()} Depending on the
+   * configuration, these hints may be ignored. Hints will never impact what data is returned by a
+   * scan, only how quickly it is returned.
+   *
+   * <p>
+   * The default configuration for Accumulo will ignore hints. See {@link HintScanPrioritizer} and
+   * {@link SimpleScanDispatcher} for examples of classes that can react to hints.
+   *
+   * @since 2.0.0
+   */
+  void setExecutionHints(Map<String,String> hints);
 }
diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/ScannerIterator.java b/core/src/main/java/org/apache/accumulo/core/client/impl/ScannerIterator.java
index 48d6ba7ede..bba3c12d61 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/impl/ScannerIterator.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/ScannerIterator.java
@@ -123,7 +123,8 @@ public void run() {
     scanState = new ScanState(context, tableId, authorizations, new Range(range),
         options.fetchedColumns, size, options.serverSideIteratorList,
         options.serverSideIteratorOptions, isolated, readaheadThreshold,
-        options.getSamplerConfiguration(), options.batchTimeOut, options.classLoaderContext);
+        options.getSamplerConfiguration(), options.batchTimeOut, options.classLoaderContext,
+        options.executionHints);
 
     // If we want to start readahead immediately, don't wait for hasNext to be called
     if (0L == readaheadThreshold) {
diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/ScannerOptions.java b/core/src/main/java/org/apache/accumulo/core/client/impl/ScannerOptions.java
index 838d003c9c..7627ecbf75 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/impl/ScannerOptions.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/ScannerOptions.java
@@ -26,6 +26,7 @@
 import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
+import java.util.Objects;
 import java.util.Set;
 import java.util.SortedSet;
 import java.util.TreeSet;
@@ -42,6 +43,8 @@
 import org.apache.accumulo.core.util.TextUtil;
 import org.apache.hadoop.io.Text;
 
+import com.google.common.collect.ImmutableMap;
+
 public class ScannerOptions implements ScannerBase {
 
   protected List<IterInfo> serverSideIteratorList = Collections.emptyList();
@@ -59,6 +62,8 @@
 
   protected String classLoaderContext = null;
 
+  protected Map<String,String> executionHints = Collections.emptyMap();
+
   protected ScannerOptions() {}
 
   public ScannerOptions(ScannerOptions so) {
@@ -179,6 +184,9 @@ protected static void setOptions(ScannerOptions dst, ScannerOptions src) {
 
         dst.samplerConfig = src.samplerConfig;
         dst.batchTimeOut = src.batchTimeOut;
+
+        // its an immutable map, so can avoid copy here
+        dst.executionHints = src.executionHints;
       }
     }
   }
@@ -264,4 +272,9 @@ public String getClassLoaderContext() {
     return this.classLoaderContext;
   }
 
+  @Override
+  public synchronized void setExecutionHints(Map<String,String> hints) {
+    this.executionHints = ImmutableMap.copyOf(Objects.requireNonNull(hints));
+  }
+
 }
diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/TabletServerBatchReaderIterator.java b/core/src/main/java/org/apache/accumulo/core/client/impl/TabletServerBatchReaderIterator.java
index c5c2a74a5c..dd101673bd 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/impl/TabletServerBatchReaderIterator.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/TabletServerBatchReaderIterator.java
@@ -674,12 +674,16 @@ static void doLookup(ClientContext context, String server, Map<KeyExtent,List<Ra
 
         Map<TKeyExtent,List<TRange>> thriftTabletRanges = Translator.translate(requested,
             Translators.KET, new Translator.ListTranslator<>(Translators.RT));
+
+        Map<String,String> execHints = options.executionHints.size() == 0 ? null
+            : options.executionHints;
+
         InitialMultiScan imsr = client.startMultiScan(Tracer.traceInfo(), context.rpcCreds(),
             thriftTabletRanges, Translator.translate(columns, Translators.CT),
             options.serverSideIteratorList, options.serverSideIteratorOptions,
             ByteBufferUtil.toByteBuffers(authorizations.getAuthorizations()), waitForWrites,
             SamplerConfigurationImpl.toThrift(options.getSamplerConfiguration()),
-            options.batchTimeOut, options.classLoaderContext);
+            options.batchTimeOut, options.classLoaderContext, execHints);
         if (waitForWrites)
           ThriftScanner.serversWaitedForWrites.get(ttype).add(server.toString());
 
diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/ThriftScanner.java b/core/src/main/java/org/apache/accumulo/core/client/impl/ThriftScanner.java
index ac2143112d..69a278e82d 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/impl/ThriftScanner.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/ThriftScanner.java
@@ -101,7 +101,8 @@ public static boolean getBatchFromServer(ClientContext context, Range range, Key
         // isolation below
         ScanState scanState = new ScanState(context, extent.getTableId(), authorizations, range,
             fetchedColumns, size, serverSideIteratorList, serverSideIteratorOptions, false,
-            Constants.SCANNER_DEFAULT_READAHEAD_THRESHOLD, null, batchTimeOut, classLoaderContext);
+            Constants.SCANNER_DEFAULT_READAHEAD_THRESHOLD, null, batchTimeOut, classLoaderContext,
+            null);
 
         TabletType ttype = TabletType.type(extent);
         boolean waitForWrites = !serversWaitedForWrites.get(ttype).contains(server);
@@ -109,7 +110,8 @@ public static boolean getBatchFromServer(ClientContext context, Range range, Key
             scanState.range.toThrift(), Translator.translate(scanState.columns, Translators.CT),
             scanState.size, scanState.serverSideIteratorList, scanState.serverSideIteratorOptions,
             scanState.authorizations.getAuthorizationsBB(), waitForWrites, scanState.isolated,
-            scanState.readaheadThreshold, null, scanState.batchTimeOut, classLoaderContext);
+            scanState.readaheadThreshold, null, scanState.batchTimeOut, classLoaderContext,
+            scanState.executionHints);
         if (waitForWrites)
           serversWaitedForWrites.get(ttype).add(server);
 
@@ -167,13 +169,14 @@ public static boolean getBatchFromServer(ClientContext context, Range range, Key
     Map<String,Map<String,String>> serverSideIteratorOptions;
 
     SamplerConfiguration samplerConfig;
+    Map<String,String> executionHints;
 
     public ScanState(ClientContext context, Table.ID tableId, Authorizations authorizations,
         Range range, SortedSet<Column> fetchedColumns, int size,
         List<IterInfo> serverSideIteratorList,
         Map<String,Map<String,String>> serverSideIteratorOptions, boolean isolated,
         long readaheadThreshold, SamplerConfiguration samplerConfig, long batchTimeOut,
-        String classLoaderContext) {
+        String classLoaderContext, Map<String,String> executionHints) {
       this.context = context;
       this.authorizations = authorizations;
       this.classLoaderContext = classLoaderContext;
@@ -205,6 +208,11 @@ public ScanState(ClientContext context, Table.ID tableId, Authorizations authori
       this.samplerConfig = samplerConfig;
 
       this.batchTimeOut = batchTimeOut;
+
+      if (executionHints == null || executionHints.size() == 0)
+        this.executionHints = null; // avoid thrift serlialization for empty map
+      else
+        this.executionHints = executionHints;
     }
   }
 
@@ -455,7 +463,7 @@ else if (log.isTraceEnabled())
             scanState.authorizations.getAuthorizationsBB(), waitForWrites, scanState.isolated,
             scanState.readaheadThreshold,
             SamplerConfigurationImpl.toThrift(scanState.samplerConfig), scanState.batchTimeOut,
-            scanState.classLoaderContext);
+            scanState.classLoaderContext, scanState.executionHints);
         if (waitForWrites)
           serversWaitedForWrites.get(ttype).add(loc.tablet_location);
 
diff --git a/core/src/main/java/org/apache/accumulo/core/spi/scan/HintScanPrioritizer.java b/core/src/main/java/org/apache/accumulo/core/spi/scan/HintScanPrioritizer.java
new file mode 100644
index 0000000000..80c036f7dd
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/spi/scan/HintScanPrioritizer.java
@@ -0,0 +1,105 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.accumulo.core.spi.scan;
+
+import java.util.Comparator;
+import java.util.Map;
+
+import org.apache.accumulo.core.client.ScannerBase;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * When configured for a scan executor, this prioritizer allows scanners to set priorities as
+ * integers.
+ *
+ * <p>
+ * Scanners should put the key/value {@code priority=<integer>} in the map passed to
+ * {@link ScannerBase#setExecutionHints(Map)} to set the priority. Lower integers result in higher
+ * priority.
+ *
+ * <p>
+ * This prioritizer accepts the option {@code default_priority=<integer>} which determines what
+ * priority to use for scans without a hint. If not set, then {@code default_priority} is
+ * {@link Integer#MAX_VALUE}.
+ *
+ * <p>
+ * This prioritizer accepts the option {@code bad_hint_action=fail|log|none}. This option determines
+ * what happens when a priority hint is not an integer. It defaults to {@code log} which logs a
+ * warning. The {@code fail} option throws an exception which may fail the scan. The {@code none}
+ * option silently ignores invalid hints.
+ *
+ * <p>
+ * When two scans have the same priority, the scan is prioritized based on last run time and then
+ * creation time.
+ *
+ * <p>
+ * If a secondary or tertiary priority is needed, this can be done with bit shifting. For example
+ * assume a primary priority of 1 to 3 is desired followed by a secondary priority of 1 to 10 . This
+ * can be encoded as {@code int priority = primary << 4 | secondary}. When the primary bits are
+ * equal the comparison naturally falls back to the secondary bits. The example does not handle the
+ * case where the primary of secondary priorities are outside expected ranges.
+ *
+ * @since 2.0.0
+ */
+public class HintScanPrioritizer implements ScanPrioritizer {
+
+  private static final Logger log = LoggerFactory.getLogger(HintScanPrioritizer.class);
+
+  private enum HintProblemAction {
+    NONE, LOG, FAIL
+  }
+
+  private static int getPriority(ScanInfo si, int defaultPriority, HintProblemAction hpa) {
+    String prio = si.getExecutionHints().get("priority");
+    if (prio != null) {
+      try {
+        return Integer.parseInt(prio);
+      } catch (NumberFormatException nfe) {
+        switch (hpa) {
+          case FAIL:
+            throw nfe;
+          case LOG:
+            log.warn("Unable to parse priority hint {}, falling back to default {}.", prio,
+                defaultPriority);
+            break;
+          case NONE:
+            break;
+          default:
+            throw new IllegalStateException();
+        }
+      }
+    }
+
+    return defaultPriority;
+  }
+
+  @Override
+  public Comparator<ScanInfo> createComparator(Map<String,String> options) {
+    int defaultPriority = Integer
+        .parseInt(options.getOrDefault("default_priority", Integer.MAX_VALUE + ""));
+
+    HintProblemAction hpa = HintProblemAction.valueOf(
+        options.getOrDefault("bad_hint_action", HintProblemAction.LOG.name()).toUpperCase());
+
+    Comparator<ScanInfo> cmp = Comparator.comparingInt(si -> getPriority(si, defaultPriority, hpa));
+
+    return cmp.thenComparingLong(si -> si.getLastRunTime().orElse(0))
+        .thenComparingLong(si -> si.getCreationTime());
+  }
+}
diff --git a/core/src/main/java/org/apache/accumulo/core/spi/scan/ScanInfo.java b/core/src/main/java/org/apache/accumulo/core/spi/scan/ScanInfo.java
index 7961c21b3c..d33b7ee3a2 100644
--- a/core/src/main/java/org/apache/accumulo/core/spi/scan/ScanInfo.java
+++ b/core/src/main/java/org/apache/accumulo/core/spi/scan/ScanInfo.java
@@ -17,11 +17,13 @@
 package org.apache.accumulo.core.spi.scan;
 
 import java.util.Collection;
+import java.util.Map;
 import java.util.OptionalLong;
 import java.util.Set;
 
 import org.apache.accumulo.core.client.BatchScanner;
 import org.apache.accumulo.core.client.Scanner;
+import org.apache.accumulo.core.client.ScannerBase;
 import org.apache.accumulo.core.data.Column;
 import org.apache.accumulo.core.spi.common.IteratorConfiguration;
 import org.apache.accumulo.core.spi.common.Stats;
@@ -113,4 +115,9 @@
    * @return iterators that where configured on the client side scanner
    */
   Collection<IteratorConfiguration> getClientScanIterators();
+
+  /**
+   * @return Hints set by a scanner using {@link ScannerBase#setExecutionHints(Map)}
+   */
+  Map<String,String> getExecutionHints();
 }
diff --git a/core/src/main/java/org/apache/accumulo/core/spi/scan/SimpleScanDispatcher.java b/core/src/main/java/org/apache/accumulo/core/spi/scan/SimpleScanDispatcher.java
index 96e7d2cf17..f7ff73f045 100644
--- a/core/src/main/java/org/apache/accumulo/core/spi/scan/SimpleScanDispatcher.java
+++ b/core/src/main/java/org/apache/accumulo/core/spi/scan/SimpleScanDispatcher.java
@@ -19,6 +19,10 @@
 import java.util.Map;
 import java.util.Set;
 
+import org.apache.accumulo.core.client.ScannerBase;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
 import com.google.common.base.Preconditions;
 import com.google.common.collect.ImmutableSet;
 import com.google.common.collect.Sets;
@@ -34,6 +38,22 @@
  * scans to the named executor.</LI>
  * <LI>{@code table.scan.dispatcher.opts.single_executor=<scan executor name>} : dispatches regular
  * scans to the named executor.</LI>
+ * <LI>{@code table.scan.dispatcher.opts.heed_hints=true|false} : This option defaults to false, so
+ * by default execution hints are ignored. When set to true, the executor can be set on the scanner.
+ * This is done by putting the key/value {@code executor=<scan executor name>} in the map passed to
+ * {@link ScannerBase#setExecutionHints(Map)}
+ * <LI>{@code table.scan.dispatcher.opts.bad_hint_action=none|log|fail} : When
+ * {@code heed_hints=true}, this option determines what to do if the executor in a hint does not
+ * exist. The possible values for this option are {@code none}, {@code log}, or {@code error}.
+ * Setting {@code none} will silently ignore invalid hints. Setting {@code log} will log a warning
+ * for invalid hints. Setting {@code fail} will throw an exception likely causing the scan to fail.
+ * For {@code log} and {@code none}, when there is an invalid hint it will fall back to the table
+ * configuration. The default is {@code log}.
+ * <LI>{@code table.scan.dispatcher.opts.ignored_hint_action=none|log|fail} : When
+ * {@code heed_hints=false}, this option determines what to do if a hint specifies an executor. The
+ * possible values for this option are {@code none}, {@code log}, or {@code fail}. The default is
+ * {@code log}.
+ *
  * </UL>
  *
  * The {@code multi_executor} and {@code single_executor} options override the {@code executor}
@@ -43,12 +63,21 @@
 public class SimpleScanDispatcher implements ScanDispatcher {
 
   private final Set<String> VALID_OPTS = ImmutableSet.of("executor", "multi_executor",
-      "single_executor");
+      "single_executor", "heed_hints", "bad_hint_action", "ignored_hint_action");
   private String multiExecutor;
   private String singleExecutor;
+  private boolean heedHints;
+  private HintProblemAction badHintAction;
+  private HintProblemAction ignoredHintHaction;
 
   public static final String DEFAULT_SCAN_EXECUTOR_NAME = "default";
 
+  private static final Logger log = LoggerFactory.getLogger(SimpleScanDispatcher.class);
+
+  private enum HintProblemAction {
+    NONE, LOG, FAIL
+  }
+
   @Override
   public void init(Map<String,String> options) {
     Set<String> invalidOpts = Sets.difference(options.keySet(), VALID_OPTS);
@@ -57,10 +86,50 @@ public void init(Map<String,String> options) {
     String base = options.getOrDefault("executor", DEFAULT_SCAN_EXECUTOR_NAME);
     multiExecutor = options.getOrDefault("multi_executor", base);
     singleExecutor = options.getOrDefault("single_executor", base);
+    heedHints = Boolean.parseBoolean(options.getOrDefault("heed_hints", "false"));
+    badHintAction = HintProblemAction.valueOf(
+        options.getOrDefault("bad_hint_action", HintProblemAction.LOG.name()).toUpperCase());
+    ignoredHintHaction = HintProblemAction.valueOf(
+        options.getOrDefault("ignored_hint_action", HintProblemAction.LOG.name()).toUpperCase());
   }
 
   @Override
   public String dispatch(ScanInfo scanInfo, Map<String,ScanExecutor> scanExecutors) {
+    if (heedHints) {
+      String executor = scanInfo.getExecutionHints().get("executor");
+      if (executor != null) {
+        if (scanExecutors.containsKey(executor)) {
+          return executor;
+        } else {
+          switch (badHintAction) {
+            case FAIL:
+              throw new IllegalArgumentException(
+                  "Scan execution hint contained unknown executor " + executor);
+            case LOG:
+              log.warn("Scan execution hint contained unknown executor {} ", executor);
+              break;
+            case NONE:
+              break;
+            default:
+              throw new IllegalStateException();
+          }
+        }
+      }
+    } else if (ignoredHintHaction != HintProblemAction.NONE
+        && scanInfo.getExecutionHints().containsKey("executor")) {
+      String executor = scanInfo.getExecutionHints().get("executor");
+      switch (ignoredHintHaction) {
+        case FAIL:
+          throw new IllegalArgumentException(
+              "Scan execution hint contained executor " + executor + " when heed_hints=false");
+        case LOG:
+          log.warn("Scan execution hint contained executor {} when heed_hints=false", executor);
+          break;
+        default:
+          throw new IllegalStateException();
+      }
+    }
+
     switch (scanInfo.getScanType()) {
       case MULTI:
         return multiExecutor;
@@ -68,7 +137,6 @@ public String dispatch(ScanInfo scanInfo, Map<String,ScanExecutor> scanExecutors
         return singleExecutor;
       default:
         throw new IllegalArgumentException("Unexpected scan type " + scanInfo.getScanType());
-
     }
   }
 }
diff --git a/core/src/main/java/org/apache/accumulo/core/tabletserver/thrift/TabletClientService.java b/core/src/main/java/org/apache/accumulo/core/tabletserver/thrift/TabletClientService.java
index 7bb083d2a9..0a2d8f84b6 100644
--- a/core/src/main/java/org/apache/accumulo/core/tabletserver/thrift/TabletClientService.java
+++ b/core/src/main/java/org/apache/accumulo/core/tabletserver/thrift/TabletClientService.java
@@ -28,13 +28,13 @@
 
   public interface Iface extends org.apache.accumulo.core.client.impl.thrift.ClientService.Iface {
 
-    public org.apache.accumulo.core.data.thrift.InitialScan startScan(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, org.apache.accumulo.core.data.thrift.TKeyExtent extent, org.apache.accumulo.core.data.thrift.TRange range, java.util.List<org.apache.accumulo.core.data.thrift.TColumn> columns, int batchSize, java.util.List<org.apache.accumulo.core.data.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) throws org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException, NotServingTabletException, TooManyFilesException, TSampleNotPresentException, org.apache.thrift.TException;
+    public org.apache.accumulo.core.data.thrift.InitialScan startScan(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, org.apache.accumulo.core.data.thrift.TKeyExtent extent, org.apache.accumulo.core.data.thrift.TRange range, java.util.List<org.apache.accumulo.core.data.thrift.TColumn> columns, int batchSize, java.util.List<org.apache.accumulo.core.data.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) throws org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException, NotServingTabletException, TooManyFilesException, TSampleNotPresentException, org.apache.thrift.TException;
 
     public org.apache.accumulo.core.data.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.data.thrift.InitialMultiScan startMultiScan(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, java.util.Map<org.apache.accumulo.core.data.thrift.TKeyExtent,java.util.List<org.apache.accumulo.core.data.thrift.TRange>> batch, java.util.List<org.apache.accumulo.core.data.thrift.TColumn> columns, java.util.List<org.apache.accumulo.core.data.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) throws org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException, TSampleNotPresentException, org.apache.thrift.TException;
+    public org.apache.accumulo.core.data.thrift.InitialMultiScan startMultiScan(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, java.util.Map<org.apache.accumulo.core.data.thrift.TKeyExtent,java.util.List<org.apache.accumulo.core.data.thrift.TRange>> batch, java.util.List<org.apache.accumulo.core.data.thrift.TColumn> columns, java.util.List<org.apache.accumulo.core.data.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) throws org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException, TSampleNotPresentException, org.apache.thrift.TException;
 
     public org.apache.accumulo.core.data.thrift.MultiScanResult continueMultiScan(org.apache.accumulo.core.trace.thrift.TInfo tinfo, long scanID) throws NoSuchScanIDException, TSampleNotPresentException, org.apache.thrift.TException;
 
@@ -104,13 +104,13 @@
 
   public interface AsyncIface extends org.apache.accumulo.core.client.impl.thrift.ClientService .AsyncIface {
 
-    public void startScan(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, org.apache.accumulo.core.data.thrift.TKeyExtent extent, org.apache.accumulo.core.data.thrift.TRange range, java.util.List<org.apache.accumulo.core.data.thrift.TColumn> columns, int batchSize, java.util.List<org.apache.accumulo.core.data.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, org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.data.thrift.InitialScan> resultHandler) throws org.apache.thrift.TException;
+    public void startScan(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, org.apache.accumulo.core.data.thrift.TKeyExtent extent, org.apache.accumulo.core.data.thrift.TRange range, java.util.List<org.apache.accumulo.core.data.thrift.TColumn> columns, int batchSize, java.util.List<org.apache.accumulo.core.data.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, org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.data.thrift.InitialScan> resultHandler) throws org.apache.thrift.TException;
 
     public void continueScan(org.apache.accumulo.core.trace.thrift.TInfo tinfo, long scanID, org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.data.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.security.thrift.TCredentials credentials, java.util.Map<org.apache.accumulo.core.data.thrift.TKeyExtent,java.util.List<org.apache.accumulo.core.data.thrift.TRange>> batch, java.util.List<org.apache.accumulo.core.data.thrift.TColumn> columns, java.util.List<org.apache.accumulo.core.data.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, org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.data.thrift.InitialMultiScan> resultHandler) throws org.apache.thrift.TException;
+    public void startMultiScan(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, java.util.Map<org.apache.accumulo.core.data.thrift.TKeyExtent,java.util.List<org.apache.accumulo.core.data.thrift.TRange>> batch, java.util.List<org.apache.accumulo.core.data.thrift.TColumn> columns, java.util.List<org.apache.accumulo.core.data.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, org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.data.thrift.InitialMultiScan> resultHandler) throws org.apache.thrift.TException;
 
     public void continueMultiScan(org.apache.accumulo.core.trace.thrift.TInfo tinfo, long scanID, org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.data.thrift.MultiScanResult> resultHandler) throws org.apache.thrift.TException;
 
@@ -198,13 +198,13 @@ public Client(org.apache.thrift.protocol.TProtocol iprot, org.apache.thrift.prot
       super(iprot, oprot);
     }
 
-    public org.apache.accumulo.core.data.thrift.InitialScan startScan(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, org.apache.accumulo.core.data.thrift.TKeyExtent extent, org.apache.accumulo.core.data.thrift.TRange range, java.util.List<org.apache.accumulo.core.data.thrift.TColumn> columns, int batchSize, java.util.List<org.apache.accumulo.core.data.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) throws org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException, NotServingTabletException, TooManyFilesException, TSampleNotPresentException, org.apache.thrift.TException
+    public org.apache.accumulo.core.data.thrift.InitialScan startScan(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, org.apache.accumulo.core.data.thrift.TKeyExtent extent, org.apache.accumulo.core.data.thrift.TRange range, java.util.List<org.apache.accumulo.core.data.thrift.TColumn> columns, int batchSize, java.util.List<org.apache.accumulo.core.data.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) throws org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException, NotServingTabletException, TooManyFilesException, TSampleNotPresentException, org.apache.thrift.TException
     {
-      send_startScan(tinfo, credentials, extent, range, columns, batchSize, ssiList, ssio, authorizations, waitForWrites, isolated, readaheadThreshold, samplerConfig, batchTimeOut, classLoaderContext);
+      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.security.thrift.TCredentials credentials, org.apache.accumulo.core.data.thrift.TKeyExtent extent, org.apache.accumulo.core.data.thrift.TRange range, java.util.List<org.apache.accumulo.core.data.thrift.TColumn> columns, int batchSize, java.util.List<org.apache.accumulo.core.data.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) throws org.apache.thrift.TException
+    public void send_startScan(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, org.apache.accumulo.core.data.thrift.TKeyExtent extent, org.apache.accumulo.core.data.thrift.TRange range, java.util.List<org.apache.accumulo.core.data.thrift.TColumn> columns, int batchSize, java.util.List<org.apache.accumulo.core.data.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) throws org.apache.thrift.TException
     {
       startScan_args args = new startScan_args();
       args.setTinfo(tinfo);
@@ -222,6 +222,7 @@ public void send_startScan(org.apache.accumulo.core.trace.thrift.TInfo tinfo, or
       args.setSamplerConfig(samplerConfig);
       args.setBatchTimeOut(batchTimeOut);
       args.setClassLoaderContext(classLoaderContext);
+      args.setExecutionHints(executionHints);
       sendBase("startScan", args);
     }
 
@@ -296,13 +297,13 @@ public void send_closeScan(org.apache.accumulo.core.trace.thrift.TInfo tinfo, lo
       sendBaseOneway("closeScan", args);
     }
 
-    public org.apache.accumulo.core.data.thrift.InitialMultiScan startMultiScan(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, java.util.Map<org.apache.accumulo.core.data.thrift.TKeyExtent,java.util.List<org.apache.accumulo.core.data.thrift.TRange>> batch, java.util.List<org.apache.accumulo.core.data.thrift.TColumn> columns, java.util.List<org.apache.accumulo.core.data.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) throws org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException, TSampleNotPresentException, org.apache.thrift.TException
+    public org.apache.accumulo.core.data.thrift.InitialMultiScan startMultiScan(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, java.util.Map<org.apache.accumulo.core.data.thrift.TKeyExtent,java.util.List<org.apache.accumulo.core.data.thrift.TRange>> batch, java.util.List<org.apache.accumulo.core.data.thrift.TColumn> columns, java.util.List<org.apache.accumulo.core.data.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) throws org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException, TSampleNotPresentException, org.apache.thrift.TException
     {
-      send_startMultiScan(tinfo, credentials, batch, columns, ssiList, ssio, authorizations, waitForWrites, samplerConfig, batchTimeOut, classLoaderContext);
+      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.security.thrift.TCredentials credentials, java.util.Map<org.apache.accumulo.core.data.thrift.TKeyExtent,java.util.List<org.apache.accumulo.core.data.thrift.TRange>> batch, java.util.List<org.apache.accumulo.core.data.thrift.TColumn> columns, java.util.List<org.apache.accumulo.core.data.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) throws org.apache.thrift.TException
+    public void send_startMultiScan(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, java.util.Map<org.apache.accumulo.core.data.thrift.TKeyExtent,java.util.List<org.apache.accumulo.core.data.thrift.TRange>> batch, java.util.List<org.apache.accumulo.core.data.thrift.TColumn> columns, java.util.List<org.apache.accumulo.core.data.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) throws org.apache.thrift.TException
     {
       startMultiScan_args args = new startMultiScan_args();
       args.setTinfo(tinfo);
@@ -316,6 +317,7 @@ public void send_startMultiScan(org.apache.accumulo.core.trace.thrift.TInfo tinf
       args.setSamplerConfig(samplerConfig);
       args.setBatchTimeOut(batchTimeOut);
       args.setClassLoaderContext(classLoaderContext);
+      args.setExecutionHints(executionHints);
       sendBase("startMultiScan", args);
     }
 
@@ -1107,9 +1109,9 @@ public AsyncClient(org.apache.thrift.protocol.TProtocolFactory protocolFactory,
       super(protocolFactory, clientManager, transport);
     }
 
-    public void startScan(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, org.apache.accumulo.core.data.thrift.TKeyExtent extent, org.apache.accumulo.core.data.thrift.TRange range, java.util.List<org.apache.accumulo.core.data.thrift.TColumn> columns, int batchSize, java.util.List<org.apache.accumulo.core.data.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, org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.data.thrift.InitialScan> resultHandler) throws org.apache.thrift.TException {
+    public void startScan(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, org.apache.accumulo.core.data.thrift.TKeyExtent extent, org.apache.accumulo.core.data.thrift.TRange range, java.util.List<org.apache.accumulo.core.data.thrift.TColumn> columns, int batchSize, java.util.List<org.apache.accumulo.core.data.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, org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.data.thrift.InitialScan> 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, resultHandler, this, ___protocolFactory, ___transport);
+      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);
       this.___currentMethod = method_call;
       ___manager.call(method_call);
     }
@@ -1130,7 +1132,8 @@ public void startScan(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apa
       private TSamplerConfiguration samplerConfig;
       private long batchTimeOut;
       private java.lang.String classLoaderContext;
-      public startScan_call(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, org.apache.accumulo.core.data.thrift.TKeyExtent extent, org.apache.accumulo.core.data.thrift.TRange range, java.util.List<org.apache.accumulo.core.data.thrift.TColumn> columns, int batchSize, java.util.List<org.apache.accumulo.core.data.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, org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.data.thrift.InitialScan> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+      private java.util.Map<java.lang.String,java.lang.String> executionHints;
+      public startScan_call(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, org.apache.accumulo.core.data.thrift.TKeyExtent extent, org.apache.accumulo.core.data.thrift.TRange range, java.util.List<org.apache.accumulo.core.data.thrift.TColumn> columns, int batchSize, java.util.List<org.apache.accumulo.core.data.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, org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.data.thrift.InitialScan> 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;
@@ -1147,6 +1150,7 @@ public startScan_call(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apa
         this.samplerConfig = samplerConfig;
         this.batchTimeOut = batchTimeOut;
         this.classLoaderContext = classLoaderContext;
+        this.executionHints = executionHints;
       }
 
       public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
@@ -1167,6 +1171,7 @@ public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apa
         args.setSamplerConfig(samplerConfig);
         args.setBatchTimeOut(batchTimeOut);
         args.setClassLoaderContext(classLoaderContext);
+        args.setExecutionHints(executionHints);
         args.write(prot);
         prot.writeMessageEnd();
       }
@@ -1251,9 +1256,9 @@ public Void getResult() throws org.apache.thrift.TException {
       }
     }
 
-    public void startMultiScan(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, java.util.Map<org.apache.accumulo.core.data.thrift.TKeyExtent,java.util.List<org.apache.accumulo.core.data.thrift.TRange>> batch, java.util.List<org.apache.accumulo.core.data.thrift.TColumn> columns, java.util.List<org.apache.accumulo.core.data.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, org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.data.thrift.InitialMultiScan> resultHandler) throws org.apache.thrift.TException {
+    public void startMultiScan(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, java.util.Map<org.apache.accumulo.core.data.thrift.TKeyExtent,java.util.List<org.apache.accumulo.core.data.thrift.TRange>> batch, java.util.List<org.apache.accumulo.core.data.thrift.TColumn> columns, java.util.List<org.apache.accumulo.core.data.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, org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.data.thrift.InitialMultiScan> 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, resultHandler, this, ___protocolFactory, ___transport);
+      startMultiScan_call method_call = new startMultiScan_call(tinfo, credentials, batch, columns, ssiList, ssio, authorizations, waitForWrites, samplerConfig, batchTimeOut, classLoaderContext, executionHints, resultHandler, this, ___protocolFactory, ___transport);
       this.___currentMethod = method_call;
       ___manager.call(method_call);
     }
@@ -1270,7 +1275,8 @@ public void startMultiScan(org.apache.accumulo.core.trace.thrift.TInfo tinfo, or
       private TSamplerConfiguration samplerConfig;
       private long batchTimeOut;
       private java.lang.String classLoaderContext;
-      public startMultiScan_call(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, java.util.Map<org.apache.accumulo.core.data.thrift.TKeyExtent,java.util.List<org.apache.accumulo.core.data.thrift.TRange>> batch, java.util.List<org.apache.accumulo.core.data.thrift.TColumn> columns, java.util.List<org.apache.accumulo.core.data.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, org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.data.thrift.InitialMultiScan> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+      private java.util.Map<java.lang.String,java.lang.String> executionHints;
+      public startMultiScan_call(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, java.util.Map<org.apache.accumulo.core.data.thrift.TKeyExtent,java.util.List<org.apache.accumulo.core.data.thrift.TRange>> batch, java.util.List<org.apache.accumulo.core.data.thrift.TColumn> columns, java.util.List<org.apache.accumulo.core.data.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, org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.data.thrift.InitialMultiScan> 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;
@@ -1283,6 +1289,7 @@ public startMultiScan_call(org.apache.accumulo.core.trace.thrift.TInfo tinfo, or
         this.samplerConfig = samplerConfig;
         this.batchTimeOut = batchTimeOut;
         this.classLoaderContext = classLoaderContext;
+        this.executionHints = executionHints;
       }
 
       public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
@@ -1299,6 +1306,7 @@ public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apa
         args.setSamplerConfig(samplerConfig);
         args.setBatchTimeOut(batchTimeOut);
         args.setClassLoaderContext(classLoaderContext);
+        args.setExecutionHints(executionHints);
         args.write(prot);
         prot.writeMessageEnd();
       }
@@ -2656,7 +2664,7 @@ protected boolean handleRuntimeExceptions() {
       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);
+          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.client.impl.thrift.ThriftSecurityException sec) {
           result.sec = sec;
         } catch (NotServingTabletException nste) {
@@ -2750,7 +2758,7 @@ protected boolean handleRuntimeExceptions() {
       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);
+          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.client.impl.thrift.ThriftSecurityException sec) {
           result.sec = sec;
         } catch (TSampleNotPresentException tsnpe) {
@@ -3761,7 +3769,7 @@ protected boolean isOneway() {
       }
 
       public void start(I iface, startScan_args args, org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.data.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,resultHandler);
+        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);
       }
     }
 
@@ -3941,7 +3949,7 @@ protected boolean isOneway() {
       }
 
       public void start(I iface, startMultiScan_args args, org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.data.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,resultHandler);
+        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);
       }
     }
 
@@ -5712,6 +5720,7 @@ public void start(I iface, contiuneGetSummaries_args args, org.apache.thrift.asy
     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();
@@ -5731,6 +5740,7 @@ public void start(I iface, contiuneGetSummaries_args args, org.apache.thrift.asy
     public TSamplerConfiguration samplerConfig; // required
     public long batchTimeOut; // required
     public java.lang.String classLoaderContext; // required
+    public 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 {
@@ -5748,7 +5758,8 @@ public void start(I iface, contiuneGetSummaries_args args, org.apache.thrift.asy
       READAHEAD_THRESHOLD((short)12, "readaheadThreshold"),
       SAMPLER_CONFIG((short)13, "samplerConfig"),
       BATCH_TIME_OUT((short)14, "batchTimeOut"),
-      CLASS_LOADER_CONTEXT((short)15, "classLoaderContext");
+      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>();
 
@@ -5793,6 +5804,8 @@ public static _Fields findByThriftId(int fieldId) {
             return BATCH_TIME_OUT;
           case 15: // CLASS_LOADER_CONTEXT
             return CLASS_LOADER_CONTEXT;
+          case 16: // EXECUTION_HINTS
+            return EXECUTION_HINTS;
           default:
             return null;
         }
@@ -5879,6 +5892,10 @@ public short getThriftFieldId() {
           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);
     }
@@ -5901,7 +5918,8 @@ public startScan_args(
       long readaheadThreshold,
       TSamplerConfiguration samplerConfig,
       long batchTimeOut,
-      java.lang.String classLoaderContext)
+      java.lang.String classLoaderContext,
+      java.util.Map<java.lang.String,java.lang.String> executionHints)
     {
       this();
       this.tinfo = tinfo;
@@ -5924,6 +5942,7 @@ public startScan_args(
       this.batchTimeOut = batchTimeOut;
       setBatchTimeOutIsSet(true);
       this.classLoaderContext = classLoaderContext;
+      this.executionHints = executionHints;
     }
 
     /**
@@ -5987,6 +6006,10 @@ public startScan_args(startScan_args other) {
       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() {
@@ -6015,6 +6038,7 @@ public void clear() {
       setBatchTimeOutIsSet(false);
       this.batchTimeOut = 0;
       this.classLoaderContext = null;
+      this.executionHints = null;
     }
 
     public org.apache.accumulo.core.trace.thrift.TInfo getTinfo() {
@@ -6428,6 +6452,41 @@ public void setClassLoaderContextIsSet(boolean value) {
       }
     }
 
+    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);
+    }
+
+    public java.util.Map<java.lang.String,java.lang.String> getExecutionHints() {
+      return this.executionHints;
+    }
+
+    public startScan_args setExecutionHints(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, java.lang.Object value) {
       switch (field) {
       case TINFO:
@@ -6550,6 +6609,14 @@ public void setFieldValue(_Fields field, java.lang.Object value) {
         }
         break;
 
+      case EXECUTION_HINTS:
+        if (value == null) {
+          unsetExecutionHints();
+        } else {
+          setExecutionHints((java.util.Map<java.lang.String,java.lang.String>)value);
+        }
+        break;
+
       }
     }
 
@@ -6600,6 +6667,9 @@ public void setFieldValue(_Fields field, java.lang.Object value) {
       case CLASS_LOADER_CONTEXT:
         return getClassLoaderContext();
 
+      case EXECUTION_HINTS:
+        return getExecutionHints();
+
       }
       throw new java.lang.IllegalStateException();
     }
@@ -6641,6 +6711,8 @@ public boolean isSet(_Fields field) {
         return isSetBatchTimeOut();
       case CLASS_LOADER_CONTEXT:
         return isSetClassLoaderContext();
+      case EXECUTION_HINTS:
+        return isSetExecutionHints();
       }
       throw new java.lang.IllegalStateException();
     }
@@ -6795,6 +6867,15 @@ public boolean equals(startScan_args that) {
           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;
     }
 
@@ -6852,6 +6933,10 @@ public int hashCode() {
       if (isSetClassLoaderContext())
         hashCode = hashCode * 8191 + classLoaderContext.hashCode();
 
+      hashCode = hashCode * 8191 + ((isSetExecutionHints()) ? 131071 : 524287);
+      if (isSetExecutionHints())
+        hashCode = hashCode * 8191 + executionHints.hashCode();
+
       return hashCode;
     }
 
@@ -7013,6 +7098,16 @@ public int compareTo(startScan_args other) {
           return lastComparison;
         }
       }
+      lastComparison = java.lang.Boolean.valueOf(isSetExecutionHints()).compareTo(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;
     }
 
@@ -7132,6 +7227,14 @@ public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.
         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();
     }
@@ -7373,6 +7476,26 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, startScan_args stru
                 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 _map133 = iprot.readMapBegin();
+                  struct.executionHints = new java.util.HashMap<java.lang.String,java.lang.String>(2*_map133.size);
+                  java.lang.String _key134;
+                  java.lang.String _val135;
+                  for (int _i136 = 0; _i136 < _map133.size; ++_i136)
+                  {
+                    _key134 = iprot.readString();
+                    _val135 = iprot.readString();
+                    struct.executionHints.put(_key134, _val135);
+                  }
+                  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);
           }
@@ -7407,9 +7530,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, startScan_args str
           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.data.thrift.TColumn _iter133 : struct.columns)
+            for (org.apache.accumulo.core.data.thrift.TColumn _iter137 : struct.columns)
             {
-              _iter133.write(oprot);
+              _iter137.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -7422,9 +7545,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, startScan_args str
           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.data.thrift.IterInfo _iter134 : struct.ssiList)
+            for (org.apache.accumulo.core.data.thrift.IterInfo _iter138 : struct.ssiList)
             {
-              _iter134.write(oprot);
+              _iter138.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -7434,15 +7557,15 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, startScan_args str
           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>> _iter135 : struct.ssio.entrySet())
+            for (java.util.Map.Entry<java.lang.String, java.util.Map<java.lang.String,java.lang.String>> _iter139 : struct.ssio.entrySet())
             {
-              oprot.writeString(_iter135.getKey());
+              oprot.writeString(_iter139.getKey());
               {
-                oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, _iter135.getValue().size()));
-                for (java.util.Map.Entry<java.lang.String, java.lang.String> _iter136 : _iter135.getValue().entrySet())
+                oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, _iter139.getValue().size()));
+                for (java.util.Map.Entry<java.lang.String, java.lang.String> _iter140 : _iter139.getValue().entrySet())
                 {
-                  oprot.writeString(_iter136.getKey());
-                  oprot.writeString(_iter136.getValue());
+                  oprot.writeString(_iter140.getKey());
+                  oprot.writeString(_iter140.getValue());
                 }
                 oprot.writeMapEnd();
               }
@@ -7455,9 +7578,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, startScan_args str
           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 _iter137 : struct.authorizations)
+            for (java.nio.ByteBuffer _iter141 : struct.authorizations)
             {
-              oprot.writeBinary(_iter137);
+              oprot.writeBinary(_iter141);
             }
             oprot.writeListEnd();
           }
@@ -7490,6 +7613,19 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, startScan_args str
           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> _iter142 : struct.executionHints.entrySet())
+            {
+              oprot.writeString(_iter142.getKey());
+              oprot.writeString(_iter142.getValue());
+            }
+            oprot.writeMapEnd();
+          }
+          oprot.writeFieldEnd();
+        }
         oprot.writeFieldStop();
         oprot.writeStructEnd();
       }
@@ -7553,7 +7689,10 @@ public void write(org.apache.thrift.protocol.TProtocol prot, startScan_args stru
         if (struct.isSetClassLoaderContext()) {
           optionals.set(14);
         }
-        oprot.writeBitSet(optionals, 15);
+        if (struct.isSetExecutionHints()) {
+          optionals.set(15);
+        }
+        oprot.writeBitSet(optionals, 16);
         if (struct.isSetTinfo()) {
           struct.tinfo.write(oprot);
         }
@@ -7569,9 +7708,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, startScan_args stru
         if (struct.isSetColumns()) {
           {
             oprot.writeI32(struct.columns.size());
-            for (org.apache.accumulo.core.data.thrift.TColumn _iter138 : struct.columns)
+            for (org.apache.accumulo.core.data.thrift.TColumn _iter143 : struct.columns)
             {
-              _iter138.write(oprot);
+              _iter143.write(oprot);
             }
           }
         }
@@ -7581,24 +7720,24 @@ public void write(org.apache.thrift.protocol.TProtocol prot, startScan_args stru
         if (struct.isSetSsiList()) {
           {
             oprot.writeI32(struct.ssiList.size());
-            for (org.apache.accumulo.core.data.thrift.IterInfo _iter139 : struct.ssiList)
+            for (org.apache.accumulo.core.data.thrift.IterInfo _iter144 : struct.ssiList)
             {
-              _iter139.write(oprot);
+              _iter144.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>> _iter140 : struct.ssio.entrySet())
+            for (java.util.Map.Entry<java.lang.String, java.util.Map<java.lang.String,java.lang.String>> _iter145 : struct.ssio.entrySet())
             {
-              oprot.writeString(_iter140.getKey());
+              oprot.writeString(_iter145.getKey());
               {
-                oprot.writeI32(_iter140.getValue().size());
-                for (java.util.Map.Entry<java.lang.String, java.lang.String> _iter141 : _iter140.getValue().entrySet())
+                oprot.writeI32(_iter145.getValue().size());
+                for (java.util.Map.Entry<java.lang.String, java.lang.String> _iter146 : _iter145.getValue().entrySet())
                 {
-                  oprot.writeString(_iter141.getKey());
-                  oprot.writeString(_iter141.getValue());
+                  oprot.writeString(_iter146.getKey());
+                  oprot.writeString(_iter146.getValue());
                 }
               }
             }
@@ -7607,9 +7746,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, startScan_args stru
         if (struct.isSetAuthorizations()) {
           {
             oprot.writeI32(struct.authorizations.size());
-            for (java.nio.ByteBuffer _iter142 : struct.authorizations)
+            for (java.nio.ByteBuffer _iter147 : struct.authorizations)
             {
-              oprot.writeBinary(_iter142);
+              oprot.writeBinary(_iter147);
             }
           }
         }
@@ -7631,12 +7770,22 @@ public void write(org.apache.thrift.protocol.TProtocol prot, startScan_args stru
         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> _iter148 : struct.executionHints.entrySet())
+            {
+              oprot.writeString(_iter148.getKey());
+              oprot.writeString(_iter148.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(15);
+        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);
@@ -7659,14 +7808,14 @@ public void read(org.apache.thrift.protocol.TProtocol prot, startScan_args struc
         }
         if (incoming.get(4)) {
           {
-            org.apache.thrift.protocol.TList _list143 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.columns = new java.util.ArrayList<org.apache.accumulo.core.data.thrift.TColumn>(_list143.size);
-            org.apache.accumulo.core.data.thrift.TColumn _elem144;
-            for (int _i145 = 0; _i145 < _list143.size; ++_i145)
+            org.apache.thrift.protocol.TList _list149 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.columns = new java.util.ArrayList<org.apache.accumulo.core.data.thrift.TColumn>(_list149.size);
+            org.apache.accumulo.core.data.thrift.TColumn _elem150;
+            for (int _i151 = 0; _i151 < _list149.size; ++_i151)
             {
-              _elem144 = new org.apache.accumulo.core.data.thrift.TColumn();
-              _elem144.read(iprot);
-              struct.columns.add(_elem144);
+              _elem150 = new org.apache.accumulo.core.data.thrift.TColumn();
+              _elem150.read(iprot);
+              struct.columns.add(_elem150);
             }
           }
           struct.setColumnsIsSet(true);
@@ -7677,53 +7826,53 @@ public void read(org.apache.thrift.protocol.TProtocol prot, startScan_args struc
         }
         if (incoming.get(6)) {
           {
-            org.apache.thrift.protocol.TList _list146 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.ssiList = new java.util.ArrayList<org.apache.accumulo.core.data.thrift.IterInfo>(_list146.size);
-            org.apache.accumulo.core.data.thrift.IterInfo _elem147;
-            for (int _i148 = 0; _i148 < _list146.size; ++_i148)
+            org.apache.thrift.protocol.TList _list152 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.ssiList = new java.util.ArrayList<org.apache.accumulo.core.data.thrift.IterInfo>(_list152.size);
+            org.apache.accumulo.core.data.thrift.IterInfo _elem153;
+            for (int _i154 = 0; _i154 < _list152.size; ++_i154)
             {
-              _elem147 = new org.apache.accumulo.core.data.thrift.IterInfo();
-              _elem147.read(iprot);
-              struct.ssiList.add(_elem147);
+              _elem153 = new org.apache.accumulo.core.data.thrift.IterInfo();
+              _elem153.read(iprot);
+              struct.ssiList.add(_elem153);
             }
           }
           struct.setSsiListIsSet(true);
         }
         if (incoming.get(7)) {
           {
-            org.apache.thrift.protocol.TMap _map149 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.MAP, iprot.readI32());
-            struct.ssio = new java.util.HashMap<java.lang.String,java.util.Map<java.lang.String,java.lang.String>>(2*_map149.size);
-            java.lang.String _key150;
-            java.util.Map<java.lang.String,java.lang.String> _val151;
-            for (int _i152 = 0; _i152 < _map149.size; ++_i152)
+            org.apache.thrift.protocol.TMap _map155 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.MAP, iprot.readI32());
+            struct.ssio = new java.util.HashMap<java.lang.String,java.util.Map<java.lang.String,java.lang.String>>(2*_map155.size);
+            java.lang.String _key156;
+            java.util.Map<java.lang.String,java.lang.String> _val157;
+            for (int _i158 = 0; _i158 < _map155.size; ++_i158)
             {
-              _key150 = iprot.readString();
+              _key156 = iprot.readString();
               {
-                org.apache.thrift.protocol.TMap _map153 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-                _val151 = new java.util.HashMap<java.lang.String,java.lang.String>(2*_map153.size);
-                java.lang.String _key154;
-                java.lang.String _val155;
-                for (int _i156 = 0; _i156 < _map153.size; ++_i156)
+                org.apache.thrift.protocol.TMap _map159 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+                _val157 = new java.util.HashMap<java.lang.String,java.lang.String>(2*_map159.size);
+                java.lang.String _key160;
+                java.lang.String _val161;
+                for (int _i162 = 0; _i162 < _map159.size; ++_i162)
                 {
-                  _key154 = iprot.readString();
-                  _val155 = iprot.readString();
-                  _val151.put(_key154, _val155);
+                  _key160 = iprot.readString();
+                  _val161 = iprot.readString();
+                  _val157.put(_key160, _val161);
                 }
               }
-              struct.ssio.put(_key150, _val151);
+              struct.ssio.put(_key156, _val157);
             }
           }
           struct.setSsioIsSet(true);
         }
         if (incoming.get(8)) {
           {
-            org.apache.thrift.protocol.TList _list157 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.authorizations = new java.util.ArrayList<java.nio.ByteBuffer>(_list157.size);
-            java.nio.ByteBuffer _elem158;
-            for (int _i159 = 0; _i159 < _list157.size; ++_i159)
+            org.apache.thrift.protocol.TList _list163 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.authorizations = new java.util.ArrayList<java.nio.ByteBuffer>(_list163.size);
+            java.nio.ByteBuffer _elem164;
+            for (int _i165 = 0; _i165 < _list163.size; ++_i165)
             {
-              _elem158 = iprot.readBinary();
-              struct.authorizations.add(_elem158);
+              _elem164 = iprot.readBinary();
+              struct.authorizations.add(_elem164);
             }
           }
           struct.setAuthorizationsIsSet(true);
@@ -7753,6 +7902,21 @@ public void read(org.apache.thrift.protocol.TProtocol prot, startScan_args struc
           struct.classLoaderContext = iprot.readString();
           struct.setClassLoaderContextIsSet(true);
         }
+        if (incoming.get(15)) {
+          {
+            org.apache.thrift.protocol.TMap _map166 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.executionHints = new java.util.HashMap<java.lang.String,java.lang.String>(2*_map166.size);
+            java.lang.String _key167;
+            java.lang.String _val168;
+            for (int _i169 = 0; _i169 < _map166.size; ++_i169)
+            {
+              _key167 = iprot.readString();
+              _val168 = iprot.readString();
+              struct.executionHints.put(_key167, _val168);
+            }
+          }
+          struct.setExecutionHintsIsSet(true);
+        }
       }
     }
 
@@ -10291,6 +10455,7 @@ public void read(org.apache.thrift.protocol.TProtocol prot, closeScan_args struc
     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();
@@ -10306,6 +10471,7 @@ public void read(org.apache.thrift.protocol.TProtocol prot, closeScan_args struc
     public TSamplerConfiguration samplerConfig; // required
     public long batchTimeOut; // required
     public java.lang.String classLoaderContext; // required
+    public 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 {
@@ -10319,7 +10485,8 @@ public void read(org.apache.thrift.protocol.TProtocol prot, closeScan_args struc
       WAIT_FOR_WRITES((short)7, "waitForWrites"),
       SAMPLER_CONFIG((short)9, "samplerConfig"),
       BATCH_TIME_OUT((short)10, "batchTimeOut"),
-      CLASS_LOADER_CONTEXT((short)11, "classLoaderContext");
+      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>();
 
@@ -10356,6 +10523,8 @@ public static _Fields findByThriftId(int fieldId) {
             return BATCH_TIME_OUT;
           case 11: // CLASS_LOADER_CONTEXT
             return CLASS_LOADER_CONTEXT;
+          case 12: // EXECUTION_HINTS
+            return EXECUTION_HINTS;
           default:
             return null;
         }
@@ -10431,6 +10600,10 @@ public short getThriftFieldId() {
           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);
     }
@@ -10449,7 +10622,8 @@ public startMultiScan_args(
       boolean waitForWrites,
       TSamplerConfiguration samplerConfig,
       long batchTimeOut,
-      java.lang.String classLoaderContext)
+      java.lang.String classLoaderContext,
+      java.util.Map<java.lang.String,java.lang.String> executionHints)
     {
       this();
       this.tinfo = tinfo;
@@ -10465,6 +10639,7 @@ public startMultiScan_args(
       this.batchTimeOut = batchTimeOut;
       setBatchTimeOutIsSet(true);
       this.classLoaderContext = classLoaderContext;
+      this.executionHints = executionHints;
     }
 
     /**
@@ -10537,6 +10712,10 @@ public startMultiScan_args(startMultiScan_args other) {
       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() {
@@ -10558,6 +10737,7 @@ public void clear() {
       setBatchTimeOutIsSet(false);
       this.batchTimeOut = 0;
       this.classLoaderContext = null;
+      this.executionHints = null;
     }
 
     public org.apache.accumulo.core.trace.thrift.TInfo getTinfo() {
@@ -10889,6 +11069,41 @@ public void setClassLoaderContextIsSet(boolean value) {
       }
     }
 
+    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);
+    }
+
+    public java.util.Map<java.lang.String,java.lang.String> getExecutionHints() {
+      return this.executionHints;
+    }
+
+    public startMultiScan_args setExecutionHints(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, java.lang.Object value) {
       switch (field) {
       case TINFO:
@@ -10979,6 +11194,14 @@ public void setFieldValue(_Fields field, java.lang.Object value) {
         }
         break;
 
+      case EXECUTION_HINTS:
+        if (value == null) {
+          unsetExecutionHints();
+        } else {
+          setExecutionHints((java.util.Map<java.lang.String,java.lang.String>)value);
+        }
+        break;
+
       }
     }
 
@@ -11017,6 +11240,9 @@ public void setFieldValue(_Fields field, java.lang.Object value) {
       case CLASS_LOADER_CONTEXT:
         return getClassLoaderContext();
 
+      case EXECUTION_HINTS:
+        return getExecutionHints();
+
       }
       throw new java.lang.IllegalStateException();
     }
@@ -11050,6 +11276,8 @@ public boolean isSet(_Fields field) {
         return isSetBatchTimeOut();
       case CLASS_LOADER_CONTEXT:
         return isSetClassLoaderContext();
+      case EXECUTION_HINTS:
+        return isSetExecutionHints();
       }
       throw new java.lang.IllegalStateException();
     }
@@ -11168,6 +11396,15 @@ public boolean equals(startMultiScan_args that) {
           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;
     }
 
@@ -11215,6 +11452,10 @@ public int hashCode() {
       if (isSetClassLoaderContext())
         hashCode = hashCode * 8191 + classLoaderContext.hashCode();
 
+      hashCode = hashCode * 8191 + ((isSetExecutionHints()) ? 131071 : 524287);
+      if (isSetExecutionHints())
+        hashCode = hashCode * 8191 + executionHints.hashCode();
+
       return hashCode;
     }
 
@@ -11336,6 +11577,16 @@ public int compareTo(startMultiScan_args other) {
           return lastComparison;
         }
       }
+      lastComparison = java.lang.Boolean.valueOf(isSetExecutionHints()).compareTo(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;
     }
 
@@ -11435,6 +11686,14 @@ public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.
         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();
     }
@@ -11510,27 +11769,27 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, startMultiScan_args
             case 2: // BATCH
               if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
                 {
-                  org.apache.thrift.protocol.TMap _map160 = iprot.readMapBegin();
-                  struct.batch = new java.util.HashMap<org.apache.accumulo.core.data.thrift.TKeyExtent,java.util.List<org.apache.accumulo.core.data.thrift.TRange>>(2*_map160.size);
-                  org.apache.accumulo.core.data.thrift.TKeyExtent _key161;
-                  java.util.List<org.apache.accumulo.core.data.thrift.TRange> _val162;
-                  for (int _i163 = 0; _i163 < _map160.size; ++_i163)
+                  org.apache.thrift.protocol.TMap _map170 = iprot.readMapBegin();
+                  struct.batch = new java.util.HashMap<org.apache.accumulo.core.data.thrift.TKeyExtent,java.util.List<org.apache.accumulo.core.data.thrift.TRange>>(2*_map170.size);
+                  org.apache.accumulo.core.data.thrift.TKeyExtent _key171;
+                  java.util.List<org.apache.accumulo.core.data.thrift.TRange> _val172;
+                  for (int _i173 = 0; _i173 < _map170.size; ++_i173)
                   {
-                    _key161 = new org.apache.accumulo.core.data.thrift.TKeyExtent();
-                    _key161.read(iprot);
+                    _key171 = new org.apache.accumulo.core.data.thrift.TKeyExtent();
+                    _key171.read(iprot);
                     {
-                      org.apache.thrift.protocol.TList _list164 = iprot.readListBegin();
-                      _val162 = new java.util.ArrayList<org.apache.accumulo.core.data.thrift.TRange>(_list164.size);
-                      org.apache.accumulo.core.data.thrift.TRange _elem165;
-                      for (int _i166 = 0; _i166 < _list164.size; ++_i166)
+                      org.apache.thrift.protocol.TList _list174 = iprot.readListBegin();
+                      _val172 = new java.util.ArrayList<org.apache.accumulo.core.data.thrift.TRange>(_list174.size);
+                      org.apache.accumulo.core.data.thrift.TRange _elem175;
+                      for (int _i176 = 0; _i176 < _list174.size; ++_i176)
                       {
-                        _elem165 = new org.apache.accumulo.core.data.thrift.TRange();
-                        _elem165.read(iprot);
-                        _val162.add(_elem165);
+                        _elem175 = new org.apache.accumulo.core.data.thrift.TRange();
+                        _elem175.read(iprot);
+                        _val172.add(_elem175);
                       }
                       iprot.readListEnd();
                     }
-                    struct.batch.put(_key161, _val162);
+                    struct.batch.put(_key171, _val172);
                   }
                   iprot.readMapEnd();
                 }
@@ -11542,14 +11801,14 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, startMultiScan_args
             case 3: // COLUMNS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list167 = iprot.readListBegin();
-                  struct.columns = new java.util.ArrayList<org.apache.accumulo.core.data.thrift.TColumn>(_list167.size);
-                  org.apache.accumulo.core.data.thrift.TColumn _elem168;
-                  for (int _i169 = 0; _i169 < _list167.size; ++_i169)
+                  org.apache.thrift.protocol.TList _list177 = iprot.readListBegin();
+                  struct.columns = new java.util.ArrayList<org.apache.accumulo.core.data.thrift.TColumn>(_list177.size);
+                  org.apache.accumulo.core.data.thrift.TColumn _elem178;
+                  for (int _i179 = 0; _i179 < _list177.size; ++_i179)
                   {
-                    _elem168 = new org.apache.accumulo.core.data.thrift.TColumn();
-                    _elem168.read(iprot);
-                    struct.columns.add(_elem168);
+                    _elem178 = new org.apache.accumulo.core.data.thrift.TColumn();
+                    _elem178.read(iprot);
+                    struct.columns.add(_elem178);
                   }
                   iprot.readListEnd();
                 }
@@ -11561,14 +11820,14 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, startMultiScan_args
             case 4: // SSI_LIST
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list170 = iprot.readListBegin();
-                  struct.ssiList = new java.util.ArrayList<org.apache.accumulo.core.data.thrift.IterInfo>(_list170.size);
-                  org.apache.accumulo.core.data.thrift.IterInfo _elem171;
-                  for (int _i172 = 0; _i172 < _list170.size; ++_i172)
+                  org.apache.thrift.protocol.TList _list180 = iprot.readListBegin();
+                  struct.ssiList = new java.util.ArrayList<org.apache.accumulo.core.data.thrift.IterInfo>(_list180.size);
+                  org.apache.accumulo.core.data.thrift.IterInfo _elem181;
+                  for (int _i182 = 0; _i182 < _list180.size; ++_i182)
                   {
-                    _elem171 = new org.apache.accumulo.core.data.thrift.IterInfo();
-                    _elem171.read(iprot);
-                    struct.ssiList.add(_elem171);
+                    _elem181 = new org.apache.accumulo.core.data.thrift.IterInfo();
+                    _elem181.read(iprot);
+                    struct.ssiList.add(_elem181);
                   }
                   iprot.readListEnd();
                 }
@@ -11580,27 +11839,27 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, startMultiScan_args
             case 5: // SSIO
               if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
                 {
-                  org.apache.thrift.protocol.TMap _map173 = iprot.readMapBegin();
-                  struct.ssio = new java.util.HashMap<java.lang.String,java.util.Map<java.lang.String,java.lang.String>>(2*_map173.size);
-                  java.lang.String _key174;
-                  java.util.Map<java.lang.String,java.lang.String> _val175;
-                  for (int _i176 = 0; _i176 < _map173.size; ++_i176)
+                  org.apache.thrift.protocol.TMap _map183 = iprot.readMapBegin();
+                  struct.ssio = new java.util.HashMap<java.lang.String,java.util.Map<java.lang.String,java.lang.String>>(2*_map183.size);
+                  java.lang.String _key184;
+                  java.util.Map<java.lang.String,java.lang.String> _val185;
+                  for (int _i186 = 0; _i186 < _map183.size; ++_i186)
                   {
-                    _key174 = iprot.readString();
+                    _key184 = iprot.readString();
                     {
-                      org.apache.thrift.protocol.TMap _map177 = iprot.readMapBegin();
-                      _val175 = new java.util.HashMap<java.lang.String,java.lang.String>(2*_map177.size);
-                      java.lang.String _key178;
-                      java.lang.String _val179;
-                      for (int _i180 = 0; _i180 < _map177.size; ++_i180)
+                      org.apache.thrift.protocol.TMap _map187 = iprot.readMapBegin();
+                      _val185 = new java.util.HashMap<java.lang.String,java.lang.String>(2*_map187.size);
+                      java.lang.String _key188;
+                      java.lang.String _val189;
+                      for (int _i190 = 0; _i190 < _map187.size; ++_i190)
                       {
-                        _key178 = iprot.readString();
-                        _val179 = iprot.readString();
-                        _val175.put(_key178, _val179);
+                        _key188 = iprot.readString();
+                        _val189 = iprot.readString();
+                        _val185.put(_key188, _val189);
                       }
                       iprot.readMapEnd();
                     }
-                    struct.ssio.put(_key174, _val175);
+                    struct.ssio.put(_key184, _val185);
                   }
                   iprot.readMapEnd();
                 }
@@ -11612,13 +11871,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, startMultiScan_args
             case 6: // AUTHORIZATIONS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list181 = iprot.readListBegin();
-                  struct.authorizations = new java.util.ArrayList<java.nio.ByteBuffer>(_list181.size);
-                  java.nio.ByteBuffer _elem182;
-                  for (int _i183 = 0; _i183 < _list181.size; ++_i183)
+                  org.apache.thrift.protocol.TList _list191 = iprot.readListBegin();
+                  struct.authorizations = new java.util.ArrayList<java.nio.ByteBuffer>(_list191.size);
+                  java.nio.ByteBuffer _elem192;
+                  for (int _i193 = 0; _i193 < _list191.size; ++_i193)
                   {
-                    _elem182 = iprot.readBinary();
-                    struct.authorizations.add(_elem182);
+                    _elem192 = iprot.readBinary();
+                    struct.authorizations.add(_elem192);
                   }
                   iprot.readListEnd();
                 }
@@ -11660,6 +11919,26 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, startMultiScan_args
                 org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
               }
               break;
+            case 12: // EXECUTION_HINTS
+              if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
+                {
+                  org.apache.thrift.protocol.TMap _map194 = iprot.readMapBegin();
+                  struct.executionHints = new java.util.HashMap<java.lang.String,java.lang.String>(2*_map194.size);
+                  java.lang.String _key195;
+                  java.lang.String _val196;
+                  for (int _i197 = 0; _i197 < _map194.size; ++_i197)
+                  {
+                    _key195 = iprot.readString();
+                    _val196 = iprot.readString();
+                    struct.executionHints.put(_key195, _val196);
+                  }
+                  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);
           }
@@ -11684,14 +11963,14 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, startMultiScan_arg
           oprot.writeFieldBegin(BATCH_FIELD_DESC);
           {
             oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRUCT, org.apache.thrift.protocol.TType.LIST, struct.batch.size()));
-            for (java.util.Map.Entry<org.apache.accumulo.core.data.thrift.TKeyExtent, java.util.List<org.apache.accumulo.core.data.thrift.TRange>> _iter184 : struct.batch.entrySet())
+            for (java.util.Map.Entry<org.apache.accumulo.core.data.thrift.TKeyExtent, java.util.List<org.apache.accumulo.core.data.thrift.TRange>> _iter198 : struct.batch.entrySet())
             {
-              _iter184.getKey().write(oprot);
+              _iter198.getKey().write(oprot);
               {
-                oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, _iter184.getValue().size()));
-                for (org.apache.accumulo.core.data.thrift.TRange _iter185 : _iter184.getValue())
+                oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, _iter198.getValue().size()));
+                for (org.apache.accumulo.core.data.thrift.TRange _iter199 : _iter198.getValue())
                 {
-                  _iter185.write(oprot);
+                  _iter199.write(oprot);
                 }
                 oprot.writeListEnd();
               }
@@ -11704,9 +11983,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, startMultiScan_arg
           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.data.thrift.TColumn _iter186 : struct.columns)
+            for (org.apache.accumulo.core.data.thrift.TColumn _iter200 : struct.columns)
             {
-              _iter186.write(oprot);
+              _iter200.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -11716,9 +11995,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, startMultiScan_arg
           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.data.thrift.IterInfo _iter187 : struct.ssiList)
+            for (org.apache.accumulo.core.data.thrift.IterInfo _iter201 : struct.ssiList)
             {
-              _iter187.write(oprot);
+              _iter201.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -11728,15 +12007,15 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, startMultiScan_arg
           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>> _iter188 : struct.ssio.entrySet())
+            for (java.util.Map.Entry<java.lang.String, java.util.Map<java.lang.String,java.lang.String>> _iter202 : struct.ssio.entrySet())
             {
-              oprot.writeString(_iter188.getKey());
+              oprot.writeString(_iter202.getKey());
               {
-                oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, _iter188.getValue().size()));
-                for (java.util.Map.Entry<java.lang.String, java.lang.String> _iter189 : _iter188.getValue().entrySet())
+                oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, _iter202.getValue().size()));
+                for (java.util.Map.Entry<java.lang.String, java.lang.String> _iter203 : _iter202.getValue().entrySet())
                 {
-                  oprot.writeString(_iter189.getKey());
-                  oprot.writeString(_iter189.getValue());
+                  oprot.writeString(_iter203.getKey());
+                  oprot.writeString(_iter203.getValue());
                 }
                 oprot.writeMapEnd();
               }
@@ -11749,9 +12028,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, startMultiScan_arg
           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 _iter190 : struct.authorizations)
+            for (java.nio.ByteBuffer _iter204 : struct.authorizations)
             {
-              oprot.writeBinary(_iter190);
+              oprot.writeBinary(_iter204);
             }
             oprot.writeListEnd();
           }
@@ -11778,6 +12057,19 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, startMultiScan_arg
           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> _iter205 : struct.executionHints.entrySet())
+            {
+              oprot.writeString(_iter205.getKey());
+              oprot.writeString(_iter205.getValue());
+            }
+            oprot.writeMapEnd();
+          }
+          oprot.writeFieldEnd();
+        }
         oprot.writeFieldStop();
         oprot.writeStructEnd();
       }
@@ -11829,7 +12121,10 @@ public void write(org.apache.thrift.protocol.TProtocol prot, startMultiScan_args
         if (struct.isSetClassLoaderContext()) {
           optionals.set(10);
         }
-        oprot.writeBitSet(optionals, 11);
+        if (struct.isSetExecutionHints()) {
+          optionals.set(11);
+        }
+        oprot.writeBitSet(optionals, 12);
         if (struct.isSetTinfo()) {
           struct.tinfo.write(oprot);
         }
@@ -11839,14 +12134,14 @@ public void write(org.apache.thrift.protocol.TProtocol prot, startMultiScan_args
         if (struct.isSetBatch()) {
           {
             oprot.writeI32(struct.batch.size());
-            for (java.util.Map.Entry<org.apache.accumulo.core.data.thrift.TKeyExtent, java.util.List<org.apache.accumulo.core.data.thrift.TRange>> _iter191 : struct.batch.entrySet())
+            for (java.util.Map.Entry<org.apache.accumulo.core.data.thrift.TKeyExtent, java.util.List<org.apache.accumulo.core.data.thrift.TRange>> _iter206 : struct.batch.entrySet())
             {
-              _iter191.getKey().write(oprot);
+              _iter206.getKey().write(oprot);
               {
-                oprot.writeI32(_iter191.getValue().size());
-                for (org.apache.accumulo.core.data.thrift.TRange _iter192 : _iter191.getValue())
+                oprot.writeI32(_iter206.getValue().size());
+                for (org.apache.accumulo.core.data.thrift.TRange _iter207 : _iter206.getValue())
                 {
-                  _iter192.write(oprot);
+                  _iter207.write(oprot);
                 }
               }
             }
@@ -11855,33 +12150,33 @@ public void write(org.apache.thrift.protocol.TProtocol prot, startMultiScan_args
         if (struct.isSetColumns()) {
           {
             oprot.writeI32(struct.columns.size());
-            for (org.apache.accumulo.core.data.thrift.TColumn _iter193 : struct.columns)
+            for (org.apache.accumulo.core.data.thrift.TColumn _iter208 : struct.columns)
             {
-              _iter193.write(oprot);
+              _iter208.write(oprot);
             }
           }
         }
         if (struct.isSetSsiList()) {
           {
             oprot.writeI32(struct.ssiList.size());
-            for (org.apache.accumulo.core.data.thrift.IterInfo _iter194 : struct.ssiList)
+            for (org.apache.accumulo.core.data.thrift.IterInfo _iter209 : struct.ssiList)
             {
-              _iter194.write(oprot);
+              _iter209.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>> _iter195 : struct.ssio.entrySet())
+            for (java.util.Map.Entry<java.lang.String, java.util.Map<java.lang.String,java.lang.String>> _iter210 : struct.ssio.entrySet())
             {
-              oprot.writeString(_iter195.getKey());
+              oprot.writeString(_iter210.getKey());
               {
-                oprot.writeI32(_iter195.getValue().size());
-                for (java.util.Map.Entry<java.lang.String, java.lang.String> _iter196 : _iter195.getValue().entrySet())
+                oprot.writeI32(_iter210.getValue().size());
+                for (java.util.Map.Entry<java.lang.String, java.lang.String> _iter211 : _iter210.getValue().entrySet())
                 {
-                  oprot.writeString(_iter196.getKey());
-                  oprot.writeString(_iter196.getValue());
+                  oprot.writeString(_iter211.getKey());
+                  oprot.writeString(_iter211.getValue());
                 }
               }
             }
@@ -11890,9 +12185,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, startMultiScan_args
         if (struct.isSetAuthorizations()) {
           {
             oprot.writeI32(struct.authorizations.size());
-            for (java.nio.ByteBuffer _iter197 : struct.authorizations)
+            for (java.nio.ByteBuffer _iter212 : struct.authorizations)
             {
-              oprot.writeBinary(_iter197);
+              oprot.writeBinary(_iter212);
             }
           }
         }
@@ -11908,12 +12203,22 @@ public void write(org.apache.thrift.protocol.TProtocol prot, startMultiScan_args
         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> _iter213 : struct.executionHints.entrySet())
+            {
+              oprot.writeString(_iter213.getKey());
+              oprot.writeString(_iter213.getValue());
+            }
+          }
+        }
       }
 
       @Override
       public void read(org.apache.thrift.protocol.TProtocol prot, startMultiScan_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(11);
+        java.util.BitSet incoming = iprot.readBitSet(12);
         if (incoming.get(0)) {
           struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
           struct.tinfo.read(iprot);
@@ -11926,93 +12231,93 @@ public void read(org.apache.thrift.protocol.TProtocol prot, startMultiScan_args
         }
         if (incoming.get(2)) {
           {
-            org.apache.thrift.protocol.TMap _map198 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRUCT, org.apache.thrift.protocol.TType.LIST, iprot.readI32());
-            struct.batch = new java.util.HashMap<org.apache.accumulo.core.data.thrift.TKeyExtent,java.util.List<org.apache.accumulo.core.data.thrift.TRange>>(2*_map198.size);
-            org.apache.accumulo.core.data.thrift.TKeyExtent _key199;
-            java.util.List<org.apache.accumulo.core.data.thrift.TRange> _val200;
-            for (int _i201 = 0; _i201 < _map198.size; ++_i201)
+            org.apache.thrift.protocol.TMap _map214 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRUCT, org.apache.thrift.protocol.TType.LIST, iprot.readI32());
+            struct.batch = new java.util.HashMap<org.apache.accumulo.core.data.thrift.TKeyExtent,java.util.List<org.apache.accumulo.core.data.thrift.TRange>>(2*_map214.size);
+            org.apache.accumulo.core.data.thrift.TKeyExtent _key215;
+            java.util.List<org.apache.accumulo.core.data.thrift.TRange> _val216;
+            for (int _i217 = 0; _i217 < _map214.size; ++_i217)
             {
-              _key199 = new org.apache.accumulo.core.data.thrift.TKeyExtent();
-              _key199.read(iprot);
+              _key215 = new org.apache.accumulo.core.data.thrift.TKeyExtent();
+              _key215.read(iprot);
               {
-                org.apache.thrift.protocol.TList _list202 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-                _val200 = new java.util.ArrayList<org.apache.accumulo.core.data.thrift.TRange>(_list202.size);
-                org.apache.accumulo.core.data.thrift.TRange _elem203;
-                for (int _i204 = 0; _i204 < _list202.size; ++_i204)
+                org.apache.thrift.protocol.TList _list218 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+                _val216 = new java.util.ArrayList<org.apache.accumulo.core.data.thrift.TRange>(_list218.size);
+                org.apache.accumulo.core.data.thrift.TRange _elem219;
+                for (int _i220 = 0; _i220 < _list218.size; ++_i220)
                 {
-                  _elem203 = new org.apache.accumulo.core.data.thrift.TRange();
-                  _elem203.read(iprot);
-                  _val200.add(_elem203);
+                  _elem219 = new org.apache.accumulo.core.data.thrift.TRange();
+                  _elem219.read(iprot);
+                  _val216.add(_elem219);
                 }
               }
-              struct.batch.put(_key199, _val200);
+              struct.batch.put(_key215, _val216);
             }
           }
           struct.setBatchIsSet(true);
         }
         if (incoming.get(3)) {
           {
-            org.apache.thrift.protocol.TList _list205 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.columns = new java.util.ArrayList<org.apache.accumulo.core.data.thrift.TColumn>(_list205.size);
-            org.apache.accumulo.core.data.thrift.TColumn _elem206;
-            for (int _i207 = 0; _i207 < _list205.size; ++_i207)
+            org.apache.thrift.protocol.TList _list221 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.columns = new java.util.ArrayList<org.apache.accumulo.core.data.thrift.TColumn>(_list221.size);
+            org.apache.accumulo.core.data.thrift.TColumn _elem222;
+            for (int _i223 = 0; _i223 < _list221.size; ++_i223)
             {
-              _elem206 = new org.apache.accumulo.core.data.thrift.TColumn();
-              _elem206.read(iprot);
-              struct.columns.add(_elem206);
+              _elem222 = new org.apache.accumulo.core.data.thrift.TColumn();
+              _elem222.read(iprot);
+              struct.columns.add(_elem222);
             }
           }
           struct.setColumnsIsSet(true);
         }
         if (incoming.get(4)) {
           {
-            org.apache.thrift.protocol.TList _list208 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.ssiList = new java.util.ArrayList<org.apache.accumulo.core.data.thrift.IterInfo>(_list208.size);
-            org.apache.accumulo.core.data.thrift.IterInfo _elem209;
-            for (int _i210 = 0; _i210 < _list208.size; ++_i210)
+            org.apache.thrift.protocol.TList _list224 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.ssiList = new java.util.ArrayList<org.apache.accumulo.core.data.thrift.IterInfo>(_list224.size);
+            org.apache.accumulo.core.data.thrift.IterInfo _elem225;
+            for (int _i226 = 0; _i226 < _list224.size; ++_i226)
             {
-              _elem209 = new org.apache.accumulo.core.data.thrift.IterInfo();
-              _elem209.read(iprot);
-              struct.ssiList.add(_elem209);
+              _elem225 = new org.apache.accumulo.core.data.thrift.IterInfo();
+              _elem225.read(iprot);
+              struct.ssiList.add(_elem225);
             }
           }
           struct.setSsiListIsSet(true);
         }
         if (incoming.get(5)) {
           {
-            org.apache.thrift.protocol.TMap _map211 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.MAP, iprot.readI32());
-            struct.ssio = new java.util.HashMap<java.lang.String,java.util.Map<java.lang.String,java.lang.String>>(2*_map211.size);
-            java.lang.String _key212;
-            java.util.Map<java.lang.String,java.lang.String> _val213;
-            for (int _i214 = 0; _i214 < _map211.size; ++_i214)
+            org.apache.thrift.protocol.TMap _map227 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.MAP, iprot.readI32());
+            struct.ssio = new java.util.HashMap<java.lang.String,java.util.Map<java.lang.String,java.lang.String>>(2*_map227.size);
+            java.lang.String _key228;
+            java.util.Map<java.lang.String,java.lang.String> _val229;
+            for (int _i230 = 0; _i230 < _map227.size; ++_i230)
             {
-              _key212 = iprot.readString();
+              _key228 = iprot.readString();
               {
-                org.apache.thrift.protocol.TMap _map215 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-                _val213 = new java.util.HashMap<java.lang.String,java.lang.String>(2*_map215.size);
-                java.lang.String _key216;
-                java.lang.String _val217;
-                for (int _i218 = 0; _i218 < _map215.size; ++_i218)
+                org.apache.thrift.protocol.TMap _map231 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+                _val229 = new java.util.HashMap<java.lang.String,java.lang.String>(2*_map231.size);
+                java.lang.String _key232;
+                java.lang.String _val233;
+                for (int _i234 = 0; _i234 < _map231.size; ++_i234)
                 {
-                  _key216 = iprot.readString();
-                  _val217 = iprot.readString();
-                  _val213.put(_key216, _val217);
+                  _key232 = iprot.readString();
+                  _val233 = iprot.readString();
+                  _val229.put(_key232, _val233);
                 }
               }
-              struct.ssio.put(_key212, _val213);
+              struct.ssio.put(_key228, _val229);
             }
           }
           struct.setSsioIsSet(true);
         }
         if (incoming.get(6)) {
           {
-            org.apache.thrift.protocol.TList _list219 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.authorizations = new java.util.ArrayList<java.nio.ByteBuffer>(_list219.size);
-            java.nio.ByteBuffer _elem220;
-            for (int _i221 = 0; _i221 < _list219.size; ++_i221)
+            org.apache.thrift.protocol.TList _list235 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.authorizations = new java.util.ArrayList<java.nio.ByteBuffer>(_list235.size);
+            java.nio.ByteBuffer _elem236;
+            for (int _i237 = 0; _i237 < _list235.size; ++_i237)
             {
-              _elem220 = iprot.readBinary();
-              struct.authorizations.add(_elem220);
+              _elem236 = iprot.readBinary();
+              struct.authorizations.add(_elem236);
             }
           }
           struct.setAuthorizationsIsSet(true);
@@ -12034,6 +12339,21 @@ public void read(org.apache.thrift.protocol.TProtocol prot, startMultiScan_args
           struct.classLoaderContext = iprot.readString();
           struct.setClassLoaderContextIsSet(true);
         }
+        if (incoming.get(11)) {
+          {
+            org.apache.thrift.protocol.TMap _map238 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.executionHints = new java.util.HashMap<java.lang.String,java.lang.String>(2*_map238.size);
+            java.lang.String _key239;
+            java.lang.String _val240;
+            for (int _i241 = 0; _i241 < _map238.size; ++_i241)
+            {
+              _key239 = iprot.readString();
+              _val240 = iprot.readString();
+              struct.executionHints.put(_key239, _val240);
+            }
+          }
+          struct.setExecutionHintsIsSet(true);
+        }
       }
     }
 
@@ -16147,14 +16467,14 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, applyUpdates_args s
             case 4: // MUTATIONS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list222 = iprot.readListBegin();
-                  struct.mutations = new java.util.ArrayList<org.apache.accumulo.core.data.thrift.TMutation>(_list222.size);
-                  org.apache.accumulo.core.data.thrift.TMutation _elem223;
-                  for (int _i224 = 0; _i224 < _list222.size; ++_i224)
+                  org.apache.thrift.protocol.TList _list242 = iprot.readListBegin();
+                  struct.mutations = new java.util.ArrayList<org.apache.accumulo.core.data.thrift.TMutation>(_list242.size);
+                  org.apache.accumulo.core.data.thrift.TMutation _elem243;
+                  for (int _i244 = 0; _i244 < _list242.size; ++_i244)
                   {
-                    _elem223 = new org.apache.accumulo.core.data.thrift.TMutation();
-                    _elem223.read(iprot);
-                    struct.mutations.add(_elem223);
+                    _elem243 = new org.apache.accumulo.core.data.thrift.TMutation();
+                    _elem243.read(iprot);
+                    struct.mutations.add(_elem243);
                   }
                   iprot.readListEnd();
                 }
@@ -16195,9 +16515,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, applyUpdates_args
           oprot.writeFieldBegin(MUTATIONS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.mutations.size()));
-            for (org.apache.accumulo.core.data.thrift.TMutation _iter225 : struct.mutations)
+            for (org.apache.accumulo.core.data.thrift.TMutation _iter245 : struct.mutations)
             {
-              _iter225.write(oprot);
+              _iter245.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -16246,9 +16566,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, applyUpdates_args s
         if (struct.isSetMutations()) {
           {
             oprot.writeI32(struct.mutations.size());
-            for (org.apache.accumulo.core.data.thrift.TMutation _iter226 : struct.mutations)
+            for (org.apache.accumulo.core.data.thrift.TMutation _iter246 : struct.mutations)
             {
-              _iter226.write(oprot);
+              _iter246.write(oprot);
             }
           }
         }
@@ -16274,14 +16594,14 @@ public void read(org.apache.thrift.protocol.TProtocol prot, applyUpdates_args st
         }
         if (incoming.get(3)) {
           {
-            org.apache.thrift.protocol.TList _list227 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.mutations = new java.util.ArrayList<org.apache.accumulo.core.data.thrift.TMutation>(_list227.size);
-            org.apache.accumulo.core.data.thrift.TMutation _elem228;
-            for (int _i229 = 0; _i229 < _list227.size; ++_i229)
+            org.apache.thrift.protocol.TList _list247 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.mutations = new java.util.ArrayList<org.apache.accumulo.core.data.thrift.TMutation>(_list247.size);
+            org.apache.accumulo.core.data.thrift.TMutation _elem248;
+            for (int _i249 = 0; _i249 < _list247.size; ++_i249)
             {
-              _elem228 = new org.apache.accumulo.core.data.thrift.TMutation();
-              _elem228.read(iprot);
-              struct.mutations.add(_elem228);
+              _elem248 = new org.apache.accumulo.core.data.thrift.TMutation();
+              _elem248.read(iprot);
+              struct.mutations.add(_elem248);
             }
           }
           struct.setMutationsIsSet(true);
@@ -19380,13 +19700,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, startConditionalUpd
             case 3: // AUTHORIZATIONS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list230 = iprot.readListBegin();
-                  struct.authorizations = new java.util.ArrayList<java.nio.ByteBuffer>(_list230.size);
-                  java.nio.ByteBuffer _elem231;
-                  for (int _i232 = 0; _i232 < _list230.size; ++_i232)
+                  org.apache.thrift.protocol.TList _list250 = iprot.readListBegin();
+                  struct.authorizations = new java.util.ArrayList<java.nio.ByteBuffer>(_list250.size);
+                  java.nio.ByteBuffer _elem251;
+                  for (int _i252 = 0; _i252 < _list250.size; ++_i252)
                   {
-                    _elem231 = iprot.readBinary();
-                    struct.authorizations.add(_elem231);
+                    _elem251 = iprot.readBinary();
+                    struct.authorizations.add(_elem251);
                   }
                   iprot.readListEnd();
                 }
@@ -19448,9 +19768,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, startConditionalUp
           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 _iter233 : struct.authorizations)
+            for (java.nio.ByteBuffer _iter253 : struct.authorizations)
             {
-              oprot.writeBinary(_iter233);
+              oprot.writeBinary(_iter253);
             }
             oprot.writeListEnd();
           }
@@ -19517,9 +19837,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, startConditionalUpd
         if (struct.isSetAuthorizations()) {
           {
             oprot.writeI32(struct.authorizations.size());
-            for (java.nio.ByteBuffer _iter234 : struct.authorizations)
+            for (java.nio.ByteBuffer _iter254 : struct.authorizations)
             {
-              oprot.writeBinary(_iter234);
+              oprot.writeBinary(_iter254);
             }
           }
         }
@@ -19550,13 +19870,13 @@ public void read(org.apache.thrift.protocol.TProtocol prot, startConditionalUpda
         }
         if (incoming.get(2)) {
           {
-            org.apache.thrift.protocol.TList _list235 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.authorizations = new java.util.ArrayList<java.nio.ByteBuffer>(_list235.size);
-            java.nio.ByteBuffer _elem236;
-            for (int _i237 = 0; _i237 < _list235.size; ++_i237)
+            org.apache.thrift.protocol.TList _list255 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.authorizations = new java.util.ArrayList<java.nio.ByteBuffer>(_list255.size);
+            java.nio.ByteBuffer _elem256;
+            for (int _i257 = 0; _i257 < _list255.size; ++_i257)
             {
-              _elem236 = iprot.readBinary();
-              struct.authorizations.add(_elem236);
+              _elem256 = iprot.readBinary();
+              struct.authorizations.add(_elem256);
             }
           }
           struct.setAuthorizationsIsSet(true);
@@ -20652,27 +20972,27 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, conditionalUpdate_a
             case 3: // MUTATIONS
               if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
                 {
-                  org.apache.thrift.protocol.TMap _map238 = iprot.readMapBegin();
-                  struct.mutations = new java.util.HashMap<org.apache.accumulo.core.data.thrift.TKeyExtent,java.util.List<org.apache.accumulo.core.data.thrift.TConditionalMutation>>(2*_map238.size);
-                  org.apache.accumulo.core.data.thrift.TKeyExtent _key239;
-                  java.util.List<org.apache.accumulo.core.data.thrift.TConditionalMutation> _val240;
-                  for (int _i241 = 0; _i241 < _map238.size; ++_i241)
+                  org.apache.thrift.protocol.TMap _map258 = iprot.readMapBegin();
+                  struct.mutations = new java.util.HashMap<org.apache.accumulo.core.data.thrift.TKeyExtent,java.util.List<org.apache.accumulo.core.data.thrift.TConditionalMutation>>(2*_map258.size);
+                  org.apache.accumulo.core.data.thrift.TKeyExtent _key259;
+                  java.util.List<org.apache.accumulo.core.data.thrift.TConditionalMutation> _val260;
+                  for (int _i261 = 0; _i261 < _map258.size; ++_i261)
                   {
-                    _key239 = new org.apache.accumulo.core.data.thrift.TKeyExtent();
-                    _key239.read(iprot);
+                    _key259 = new org.apache.accumulo.core.data.thrift.TKeyExtent();
+                    _key259.read(iprot);
                     {
-                      org.apache.thrift.protocol.TList _list242 = iprot.readListBegin();
-                      _val240 = new java.util.ArrayList<org.apache.accumulo.core.data.thrift.TConditionalMutation>(_list242.size);
-                      org.apache.accumulo.core.data.thrift.TConditionalMutation _elem243;
-                      for (int _i244 = 0; _i244 < _list242.size; ++_i244)
+                      org.apache.thrift.protocol.TList _list262 = iprot.readListBegin();
+                      _val260 = new java.util.ArrayList<org.apache.accumulo.core.data.thrift.TConditionalMutation>(_list262.size);
+                      org.apache.accumulo.core.data.thrift.TConditionalMutation _elem263;
+                      for (int _i264 = 0; _i264 < _list262.size; ++_i264)
                       {
-                        _elem243 = new org.apache.accumulo.core.data.thrift.TConditionalMutation();
-                        _elem243.read(iprot);
-                        _val240.add(_elem243);
+                        _elem263 = new org.apache.accumulo.core.data.thrift.TConditionalMutation();
+                        _elem263.read(iprot);
+                        _val260.add(_elem263);
                       }
                       iprot.readListEnd();
                     }
-                    struct.mutations.put(_key239, _val240);
+                    struct.mutations.put(_key259, _val260);
                   }
                   iprot.readMapEnd();
                 }
@@ -20684,13 +21004,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, conditionalUpdate_a
             case 4: // SYMBOLS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list245 = iprot.readListBegin();
-                  struct.symbols = new java.util.ArrayList<java.lang.String>(_list245.size);
-                  java.lang.String _elem246;
-                  for (int _i247 = 0; _i247 < _list245.size; ++_i247)
+                  org.apache.thrift.protocol.TList _list265 = iprot.readListBegin();
+                  struct.symbols = new java.util.ArrayList<java.lang.String>(_list265.size);
+                  java.lang.String _elem266;
+                  for (int _i267 = 0; _i267 < _list265.size; ++_i267)
                   {
-                    _elem246 = iprot.readString();
-                    struct.symbols.add(_elem246);
+                    _elem266 = iprot.readString();
+                    struct.symbols.add(_elem266);
                   }
                   iprot.readListEnd();
                 }
@@ -20726,14 +21046,14 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, conditionalUpdate_
           oprot.writeFieldBegin(MUTATIONS_FIELD_DESC);
           {
             oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRUCT, org.apache.thrift.protocol.TType.LIST, struct.mutations.size()));
-            for (java.util.Map.Entry<org.apache.accumulo.core.data.thrift.TKeyExtent, java.util.List<org.apache.accumulo.core.data.thrift.TConditionalMutation>> _iter248 : struct.mutations.entrySet())
+            for (java.util.Map.Entry<org.apache.accumulo.core.data.thrift.TKeyExtent, java.util.List<org.apache.accumulo.core.data.thrift.TConditionalMutation>> _iter268 : struct.mutations.entrySet())
             {
-              _iter248.getKey().write(oprot);
+              _iter268.getKey().write(oprot);
               {
-                oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, _iter248.getValue().size()));
-                for (org.apache.accumulo.core.data.thrift.TConditionalMutation _iter249 : _iter248.getValue())
+                oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, _iter268.getValue().size()));
+                for (org.apache.accumulo.core.data.thrift.TConditionalMutation _iter269 : _iter268.getValue())
                 {
-                  _iter249.write(oprot);
+                  _iter269.write(oprot);
                 }
                 oprot.writeListEnd();
               }
@@ -20746,9 +21066,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, conditionalUpdate_
           oprot.writeFieldBegin(SYMBOLS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.symbols.size()));
-            for (java.lang.String _iter250 : struct.symbols)
+            for (java.lang.String _iter270 : struct.symbols)
             {
-              oprot.writeString(_iter250);
+              oprot.writeString(_iter270);
             }
             oprot.writeListEnd();
           }
@@ -20794,14 +21114,14 @@ public void write(org.apache.thrift.protocol.TProtocol prot, conditionalUpdate_a
         if (struct.isSetMutations()) {
           {
             oprot.writeI32(struct.mutations.size());
-            for (java.util.Map.Entry<org.apache.accumulo.core.data.thrift.TKeyExtent, java.util.List<org.apache.accumulo.core.data.thrift.TConditionalMutation>> _iter251 : struct.mutations.entrySet())
+            for (java.util.Map.Entry<org.apache.accumulo.core.data.thrift.TKeyExtent, java.util.List<org.apache.accumulo.core.data.thrift.TConditionalMutation>> _iter271 : struct.mutations.entrySet())
             {
-              _iter251.getKey().write(oprot);
+              _iter271.getKey().write(oprot);
               {
-                oprot.writeI32(_iter251.getValue().size());
-                for (org.apache.accumulo.core.data.thrift.TConditionalMutation _iter252 : _iter251.getValue())
+                oprot.writeI32(_iter271.getValue().size());
+                for (org.apache.accumulo.core.data.thrift.TConditionalMutation _iter272 : _iter271.getValue())
                 {
-                  _iter252.write(oprot);
+                  _iter272.write(oprot);
                 }
               }
             }
@@ -20810,9 +21130,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, conditionalUpdate_a
         if (struct.isSetSymbols()) {
           {
             oprot.writeI32(struct.symbols.size());
-            for (java.lang.String _iter253 : struct.symbols)
+            for (java.lang.String _iter273 : struct.symbols)
             {
-              oprot.writeString(_iter253);
+              oprot.writeString(_iter273);
             }
           }
         }
@@ -20833,39 +21153,39 @@ public void read(org.apache.thrift.protocol.TProtocol prot, conditionalUpdate_ar
         }
         if (incoming.get(2)) {
           {
-            org.apache.thrift.protocol.TMap _map254 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRUCT, org.apache.thrift.protocol.TType.LIST, iprot.readI32());
-            struct.mutations = new java.util.HashMap<org.apache.accumulo.core.data.thrift.TKeyExtent,java.util.List<org.apache.accumulo.core.data.thrift.TConditionalMutation>>(2*_map254.size);
-            org.apache.accumulo.core.data.thrift.TKeyExtent _key255;
-            java.util.List<org.apache.accumulo.core.data.thrift.TConditionalMutation> _val256;
-            for (int _i257 = 0; _i257 < _map254.size; ++_i257)
+            org.apache.thrift.protocol.TMap _map274 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRUCT, org.apache.thrift.protocol.TType.LIST, iprot.readI32());
+            struct.mutations = new java.util.HashMap<org.apache.accumulo.core.data.thrift.TKeyExtent,java.util.List<org.apache.accumulo.core.data.thrift.TConditionalMutation>>(2*_map274.size);
+            org.apache.accumulo.core.data.thrift.TKeyExtent _key275;
+            java.util.List<org.apache.accumulo.core.data.thrift.TConditionalMutation> _val276;
+            for (int _i277 = 0; _i277 < _map274.size; ++_i277)
             {
-              _key255 = new org.apache.accumulo.core.data.thrift.TKeyExtent();
-              _key255.read(iprot);
+              _key275 = new org.apache.accumulo.core.data.thrift.TKeyExtent();
+              _key275.read(iprot);
               {
-                org.apache.thrift.protocol.TList _list258 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-                _val256 = new java.util.ArrayList<org.apache.accumulo.core.data.thrift.TConditionalMutation>(_list258.size);
-                org.apache.accumulo.core.data.thrift.TConditionalMutation _elem259;
-                for (int _i260 = 0; _i260 < _list258.size; ++_i260)
+                org.apache.thrift.protocol.TList _list278 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+                _val276 = new java.util.ArrayList<org.apache.accumulo.core.data.thrift.TConditionalMutation>(_list278.size);
+                org.apache.accumulo.core.data.thrift.TConditionalMutation _elem279;
+                for (int _i280 = 0; _i280 < _list278.size; ++_i280)
                 {
-                  _elem259 = new org.apache.accumulo.core.data.thrift.TConditionalMutation();
-                  _elem259.read(iprot);
-                  _val256.add(_elem259);
+                  _elem279 = new org.apache.accumulo.core.data.thrift.TConditionalMutation();
+                  _elem279.read(iprot);
+                  _val276.add(_elem279);
                 }
               }
-              struct.mutations.put(_key255, _val256);
+              struct.mutations.put(_key275, _val276);
             }
           }
           struct.setMutationsIsSet(true);
         }
         if (incoming.get(3)) {
           {
-            org.apache.thrift.protocol.TList _list261 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.symbols = new java.util.ArrayList<java.lang.String>(_list261.size);
-            java.lang.String _elem262;
-            for (int _i263 = 0; _i263 < _list261.size; ++_i263)
+            org.apache.thrift.protocol.TList _list281 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.symbols = new java.util.ArrayList<java.lang.String>(_list281.size);
+            java.lang.String _elem282;
+            for (int _i283 = 0; _i283 < _list281.size; ++_i283)
             {
-              _elem262 = iprot.readString();
-              struct.symbols.add(_elem262);
+              _elem282 = iprot.readString();
+              struct.symbols.add(_elem282);
             }
           }
           struct.setSymbolsIsSet(true);
@@ -21273,14 +21593,14 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, conditionalUpdate_r
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list264 = iprot.readListBegin();
-                  struct.success = new java.util.ArrayList<org.apache.accumulo.core.data.thrift.TCMResult>(_list264.size);
-                  org.apache.accumulo.core.data.thrift.TCMResult _elem265;
-                  for (int _i266 = 0; _i266 < _list264.size; ++_i266)
+                  org.apache.thrift.protocol.TList _list284 = iprot.readListBegin();
+                  struct.success = new java.util.ArrayList<org.apache.accumulo.core.data.thrift.TCMResult>(_list284.size);
+                  org.apache.accumulo.core.data.thrift.TCMResult _elem285;
+                  for (int _i286 = 0; _i286 < _list284.size; ++_i286)
                   {
-                    _elem265 = new org.apache.accumulo.core.data.thrift.TCMResult();
-                    _elem265.read(iprot);
-                    struct.success.add(_elem265);
+                    _elem285 = new org.apache.accumulo.core.data.thrift.TCMResult();
+                    _elem285.read(iprot);
+                    struct.success.add(_elem285);
                   }
                   iprot.readListEnd();
                 }
@@ -21317,9 +21637,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, conditionalUpdate_
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size()));
-            for (org.apache.accumulo.core.data.thrift.TCMResult _iter267 : struct.success)
+            for (org.apache.accumulo.core.data.thrift.TCMResult _iter287 : struct.success)
             {
-              _iter267.write(oprot);
+              _iter287.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -21358,9 +21678,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, conditionalUpdate_r
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (org.apache.accumulo.core.data.thrift.TCMResult _iter268 : struct.success)
+            for (org.apache.accumulo.core.data.thrift.TCMResult _iter288 : struct.success)
             {
-              _iter268.write(oprot);
+              _iter288.write(oprot);
             }
           }
         }
@@ -21375,14 +21695,14 @@ public void read(org.apache.thrift.protocol.TProtocol prot, conditionalUpdate_re
         java.util.BitSet incoming = iprot.readBitSet(2);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list269 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.success = new java.util.ArrayList<org.apache.accumulo.core.data.thrift.TCMResult>(_list269.size);
-            org.apache.accumulo.core.data.thrift.TCMResult _elem270;
-            for (int _i271 = 0; _i271 < _list269.size; ++_i271)
+            org.apache.thrift.protocol.TList _list289 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.success = new java.util.ArrayList<org.apache.accumulo.core.data.thrift.TCMResult>(_list289.size);
+            org.apache.accumulo.core.data.thrift.TCMResult _elem290;
+            for (int _i291 = 0; _i291 < _list289.size; ++_i291)
             {
-              _elem270 = new org.apache.accumulo.core.data.thrift.TCMResult();
-              _elem270.read(iprot);
-              struct.success.add(_elem270);
+              _elem290 = new org.apache.accumulo.core.data.thrift.TCMResult();
+              _elem290.read(iprot);
+              struct.success.add(_elem290);
             }
           }
           struct.setSuccessIsSet(true);
@@ -23260,29 +23580,29 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, bulkImport_args str
             case 2: // FILES
               if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
                 {
-                  org.apache.thrift.protocol.TMap _map272 = iprot.readMapBegin();
-                  struct.files = new java.util.HashMap<org.apache.accumulo.core.data.thrift.TKeyExtent,java.util.Map<java.lang.String,org.apache.accumulo.core.data.thrift.MapFileInfo>>(2*_map272.size);
-                  org.apache.accumulo.core.data.thrift.TKeyExtent _key273;
-                  java.util.Map<java.lang.String,org.apache.accumulo.core.data.thrift.MapFileInfo> _val274;
-                  for (int _i275 = 0; _i275 < _map272.size; ++_i275)
+                  org.apache.thrift.protocol.TMap _map292 = iprot.readMapBegin();
+                  struct.files = new java.util.HashMap<org.apache.accumulo.core.data.thrift.TKeyExtent,java.util.Map<java.lang.String,org.apache.accumulo.core.data.thrift.MapFileInfo>>(2*_map292.size);
+                  org.apache.accumulo.core.data.thrift.TKeyExtent _key293;
+                  java.util.Map<java.lang.String,org.apache.accumulo.core.data.thrift.MapFileInfo> _val294;
+                  for (int _i295 = 0; _i295 < _map292.size; ++_i295)
                   {
-                    _key273 = new org.apache.accumulo.core.data.thrift.TKeyExtent();
-                    _key273.read(iprot);
+                    _key293 = new org.apache.accumulo.core.data.thrift.TKeyExtent();
+                    _key293.read(iprot);
                     {
-                      org.apache.thrift.protocol.TMap _map276 = iprot.readMapBegin();
-                      _val274 = new java.util.HashMap<java.lang.String,org.apache.accumulo.core.data.thrift.MapFileInfo>(2*_map276.size);
-                      java.lang.String _key277;
-                      org.apache.accumulo.core.data.thrift.MapFileInfo _val278;
-                      for (int _i279 = 0; _i279 < _map276.size; ++_i279)
+                      org.apache.thrift.protocol.TMap _map296 = iprot.readMapBegin();
+                      _val294 = new java.util.HashMap<java.lang.String,org.apache.accumulo.core.data.thrift.MapFileInfo>(2*_map296.size);
+                      java.lang.String _key297;
+                      org.apache.accumulo.core.data.thrift.MapFileInfo _val298;
+                      for (int _i299 = 0; _i299 < _map296.size; ++_i299)
                       {
-                        _key277 = iprot.readString();
-                        _val278 = new org.apache.accumulo.core.data.thrift.MapFileInfo();
-                        _val278.read(iprot);
-                        _val274.put(_key277, _val278);
+                        _key297 = iprot.readString();
+                        _val298 = new org.apache.accumulo.core.data.thrift.MapFileInfo();
+                        _val298.read(iprot);
+                        _val294.put(_key297, _val298);
                       }
                       iprot.readMapEnd();
                     }
-                    struct.files.put(_key273, _val274);
+                    struct.files.put(_key293, _val294);
                   }
                   iprot.readMapEnd();
                 }
@@ -23323,15 +23643,15 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, bulkImport_args st
           oprot.writeFieldBegin(FILES_FIELD_DESC);
           {
             oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRUCT, org.apache.thrift.protocol.TType.MAP, struct.files.size()));
-            for (java.util.Map.Entry<org.apache.accumulo.core.data.thrift.TKeyExtent, java.util.Map<java.lang.String,org.apache.accumulo.core.data.thrift.MapFileInfo>> _iter280 : struct.files.entrySet())
+            for (java.util.Map.Entry<org.apache.accumulo.core.data.thrift.TKeyExtent, java.util.Map<java.lang.String,org.apache.accumulo.core.data.thrift.MapFileInfo>> _iter300 : struct.files.entrySet())
             {
-              _iter280.getKey().write(oprot);
+              _iter300.getKey().write(oprot);
               {
-                oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT, _iter280.getValue().size()));
-                for (java.util.Map.Entry<java.lang.String, org.apache.accumulo.core.data.thrift.MapFileInfo> _iter281 : _iter280.getValue().entrySet())
+                oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT, _iter300.getValue().size()));
+                for (java.util.Map.Entry<java.lang.String, org.apache.accumulo.core.data.thrift.MapFileInfo> _iter301 : _iter300.getValue().entrySet())
                 {
-                  oprot.writeString(_iter281.getKey());
-                  _iter281.getValue().write(oprot);
+                  oprot.writeString(_iter301.getKey());
+                  _iter301.getValue().write(oprot);
                 }
                 oprot.writeMapEnd();
               }
@@ -23397,15 +23717,15 @@ public void write(org.apache.thrift.protocol.TProtocol prot, bulkImport_args str
         if (struct.isSetFiles()) {
           {
             oprot.writeI32(struct.files.size());
-            for (java.util.Map.Entry<org.apache.accumulo.core.data.thrift.TKeyExtent, java.util.Map<java.lang.String,org.apache.accumulo.core.data.thrift.MapFileInfo>> _iter282 : struct.files.entrySet())
+            for (java.util.Map.Entry<org.apache.accumulo.core.data.thrift.TKeyExtent, java.util.Map<java.lang.String,org.apache.accumulo.core.data.thrift.MapFileInfo>> _iter302 : struct.files.entrySet())
             {
-              _iter282.getKey().write(oprot);
+              _iter302.getKey().write(oprot);
               {
-                oprot.writeI32(_iter282.getValue().size());
-                for (java.util.Map.Entry<java.lang.String, org.apache.accumulo.core.data.thrift.MapFileInfo> _iter283 : _iter282.getValue().entrySet())
+                oprot.writeI32(_iter302.getValue().size());
+                for (java.util.Map.Entry<java.lang.String, org.apache.accumulo.core.data.thrift.MapFileInfo> _iter303 : _iter302.getValue().entrySet())
                 {
-                  oprot.writeString(_iter283.getKey());
-                  _iter283.getValue().write(oprot);
+                  oprot.writeString(_iter303.getKey());
+                  _iter303.getValue().write(oprot);
                 }
               }
             }
@@ -23436,28 +23756,28 @@ public void read(org.apache.thrift.protocol.TProtocol prot, bulkImport_args stru
         }
         if (incoming.get(3)) {
           {
-            org.apache.thrift.protocol.TMap _map284 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRUCT, org.apache.thrift.protocol.TType.MAP, iprot.readI32());
-            struct.files = new java.util.HashMap<org.apache.accumulo.core.data.thrift.TKeyExtent,java.util.Map<java.lang.String,org.apache.accumulo.core.data.thrift.MapFileInfo>>(2*_map284.size);
-            org.apache.accumulo.core.data.thrift.TKeyExtent _key285;
-            java.util.Map<java.lang.String,org.apache.accumulo.core.data.thrift.MapFileInfo> _val286;
-            for (int _i287 = 0; _i287 < _map284.size; ++_i287)
+            org.apache.thrift.protocol.TMap _map304 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRUCT, org.apache.thrift.protocol.TType.MAP, iprot.readI32());
+            struct.files = new java.util.HashMap<org.apache.accumulo.core.data.thrift.TKeyExtent,java.util.Map<java.lang.String,org.apache.accumulo.core.data.thrift.MapFileInfo>>(2*_map304.size);
+            org.apache.accumulo.core.data.thrift.TKeyExtent _key305;
+            java.util.Map<java.lang.String,org.apache.accumulo.core.data.thrift.MapFileInfo> _val306;
+            for (int _i307 = 0; _i307 < _map304.size; ++_i307)
             {
-              _key285 = new org.apache.accumulo.core.data.thrift.TKeyExtent();
-              _key285.read(iprot);
+              _key305 = new org.apache.accumulo.core.data.thrift.TKeyExtent();
+              _key305.read(iprot);
               {
-                org.apache.thrift.protocol.TMap _map288 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-                _val286 = new java.util.HashMap<java.lang.String,org.apache.accumulo.core.data.thrift.MapFileInfo>(2*_map288.size);
-                java.lang.String _key289;
-                org.apache.accumulo.core.data.thrift.MapFileInfo _val290;
-                for (int _i291 = 0; _i291 < _map288.size; ++_i291)
+                org.apache.thrift.protocol.TMap _map308 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+                _val306 = new java.util.HashMap<java.lang.String,org.apache.accumulo.core.data.thrift.MapFileInfo>(2*_map308.size);
+                java.lang.String _key309;
+                org.apache.accumulo.core.data.thrift.MapFileInfo _val310;
+                for (int _i311 = 0; _i311 < _map308.size; ++_i311)
                 {
-                  _key289 = iprot.readString();
-                  _val290 = new org.apache.accumulo.core.data.thrift.MapFileInfo();
-                  _val290.read(iprot);
-                  _val286.put(_key289, _val290);
+                  _key309 = iprot.readString();
+                  _val310 = new org.apache.accumulo.core.data.thrift.MapFileInfo();
+                  _val310.read(iprot);
+                  _val306.put(_key309, _val310);
                 }
               }
-              struct.files.put(_key285, _val286);
+              struct.files.put(_key305, _val306);
             }
           }
           struct.setFilesIsSet(true);
@@ -23869,14 +24189,14 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, bulkImport_result s
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list292 = iprot.readListBegin();
-                  struct.success = new java.util.ArrayList<org.apache.accumulo.core.data.thrift.TKeyExtent>(_list292.size);
-                  org.apache.accumulo.core.data.thrift.TKeyExtent _elem293;
-                  for (int _i294 = 0; _i294 < _list292.size; ++_i294)
+                  org.apache.thrift.protocol.TList _list312 = iprot.readListBegin();
+                  struct.success = new java.util.ArrayList<org.apache.accumulo.core.data.thrift.TKeyExtent>(_list312.size);
+                  org.apache.accumulo.core.data.thrift.TKeyExtent _elem313;
+                  for (int _i314 = 0; _i314 < _list312.size; ++_i314)
                   {
-                    _elem293 = new org.apache.accumulo.core.data.thrift.TKeyExtent();
-                    _elem293.read(iprot);
-                    struct.success.add(_elem293);
+                    _elem313 = new org.apache.accumulo.core.data.thrift.TKeyExtent();
+                    _elem313.read(iprot);
+                    struct.success.add(_elem313);
                   }
                   iprot.readListEnd();
                 }
@@ -23913,9 +24233,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, bulkImport_result
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size()));
-            for (org.apache.accumulo.core.data.thrift.TKeyExtent _iter295 : struct.success)
+            for (org.apache.accumulo.core.data.thrift.TKeyExtent _iter315 : struct.success)
             {
-              _iter295.write(oprot);
+              _iter315.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -23954,9 +24274,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, bulkImport_result s
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (org.apache.accumulo.core.data.thrift.TKeyExtent _iter296 : struct.success)
+            for (org.apache.accumulo.core.data.thrift.TKeyExtent _iter316 : struct.success)
             {
-              _iter296.write(oprot);
+              _iter316.write(oprot);
             }
           }
         }
@@ -23971,14 +24291,14 @@ public void read(org.apache.thrift.protocol.TProtocol prot, bulkImport_result st
         java.util.BitSet incoming = iprot.readBitSet(2);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list297 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.success = new java.util.ArrayList<org.apache.accumulo.core.data.thrift.TKeyExtent>(_list297.size);
-            org.apache.accumulo.core.data.thrift.TKeyExtent _elem298;
-            for (int _i299 = 0; _i299 < _list297.size; ++_i299)
+            org.apache.thrift.protocol.TList _list317 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.success = new java.util.ArrayList<org.apache.accumulo.core.data.thrift.TKeyExtent>(_list317.size);
+            org.apache.accumulo.core.data.thrift.TKeyExtent _elem318;
+            for (int _i319 = 0; _i319 < _list317.size; ++_i319)
             {
-              _elem298 = new org.apache.accumulo.core.data.thrift.TKeyExtent();
-              _elem298.read(iprot);
-              struct.success.add(_elem298);
+              _elem318 = new org.apache.accumulo.core.data.thrift.TKeyExtent();
+              _elem318.read(iprot);
+              struct.success.add(_elem318);
             }
           }
           struct.setSuccessIsSet(true);
@@ -24845,16 +25165,16 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, loadFiles_args stru
             case 6: // FILES
               if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
                 {
-                  org.apache.thrift.protocol.TMap _map300 = iprot.readMapBegin();
-                  struct.files = new java.util.HashMap<java.lang.String,org.apache.accumulo.core.data.thrift.MapFileInfo>(2*_map300.size);
-                  java.lang.String _key301;
-                  org.apache.accumulo.core.data.thrift.MapFileInfo _val302;
-                  for (int _i303 = 0; _i303 < _map300.size; ++_i303)
+                  org.apache.thrift.protocol.TMap _map320 = iprot.readMapBegin();
+                  struct.files = new java.util.HashMap<java.lang.String,org.apache.accumulo.core.data.thrift.MapFileInfo>(2*_map320.size);
+                  java.lang.String _key321;
+                  org.apache.accumulo.core.data.thrift.MapFileInfo _val322;
+                  for (int _i323 = 0; _i323 < _map320.size; ++_i323)
                   {
-                    _key301 = iprot.readString();
-                    _val302 = new org.apache.accumulo.core.data.thrift.MapFileInfo();
-                    _val302.read(iprot);
-                    struct.files.put(_key301, _val302);
+                    _key321 = iprot.readString();
+                    _val322 = new org.apache.accumulo.core.data.thrift.MapFileInfo();
+                    _val322.read(iprot);
+                    struct.files.put(_key321, _val322);
                   }
                   iprot.readMapEnd();
                 }
@@ -24913,10 +25233,10 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, loadFiles_args str
           oprot.writeFieldBegin(FILES_FIELD_DESC);
           {
             oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT, struct.files.size()));
-            for (java.util.Map.Entry<java.lang.String, org.apache.accumulo.core.data.thrift.MapFileInfo> _iter304 : struct.files.entrySet())
+            for (java.util.Map.Entry<java.lang.String, org.apache.accumulo.core.data.thrift.MapFileInfo> _iter324 : struct.files.entrySet())
             {
-              oprot.writeString(_iter304.getKey());
-              _iter304.getValue().write(oprot);
+              oprot.writeString(_iter324.getKey());
+              _iter324.getValue().write(oprot);
             }
             oprot.writeMapEnd();
           }
@@ -24983,10 +25303,10 @@ public void write(org.apache.thrift.protocol.TProtocol prot, loadFiles_args stru
         if (struct.isSetFiles()) {
           {
             oprot.writeI32(struct.files.size());
-            for (java.util.Map.Entry<java.lang.String, org.apache.accumulo.core.data.thrift.MapFileInfo> _iter305 : struct.files.entrySet())
+            for (java.util.Map.Entry<java.lang.String, org.apache.accumulo.core.data.thrift.MapFileInfo> _iter325 : struct.files.entrySet())
             {
-              oprot.writeString(_iter305.getKey());
-              _iter305.getValue().write(oprot);
+              oprot.writeString(_iter325.getKey());
+              _iter325.getValue().write(oprot);
             }
           }
         }
@@ -25024,16 +25344,16 @@ public void read(org.apache.thrift.protocol.TProtocol prot, loadFiles_args struc
         }
         if (incoming.get(5)) {
           {
-            org.apache.thrift.protocol.TMap _map306 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.files = new java.util.HashMap<java.lang.String,org.apache.accumulo.core.data.thrift.MapFileInfo>(2*_map306.size);
-            java.lang.String _key307;
-            org.apache.accumulo.core.data.thrift.MapFileInfo _val308;
-            for (int _i309 = 0; _i309 < _map306.size; ++_i309)
+            org.apache.thrift.protocol.TMap _map326 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.files = new java.util.HashMap<java.lang.String,org.apache.accumulo.core.data.thrift.MapFileInfo>(2*_map326.size);
+            java.lang.String _key327;
+            org.apache.accumulo.core.data.thrift.MapFileInfo _val328;
+            for (int _i329 = 0; _i329 < _map326.size; ++_i329)
             {
-              _key307 = iprot.readString();
-              _val308 = new org.apache.accumulo.core.data.thrift.MapFileInfo();
-              _val308.read(iprot);
-              struct.files.put(_key307, _val308);
+              _key327 = iprot.readString();
+              _val328 = new org.apache.accumulo.core.data.thrift.MapFileInfo();
+              _val328.read(iprot);
+              struct.files.put(_key327, _val328);
             }
           }
           struct.setFilesIsSet(true);
@@ -32963,14 +33283,14 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, getTabletStats_resu
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list310 = iprot.readListBegin();
-                  struct.success = new java.util.ArrayList<TabletStats>(_list310.size);
-                  TabletStats _elem311;
-                  for (int _i312 = 0; _i312 < _list310.size; ++_i312)
+                  org.apache.thrift.protocol.TList _list330 = iprot.readListBegin();
+                  struct.success = new java.util.ArrayList<TabletStats>(_list330.size);
+                  TabletStats _elem331;
+                  for (int _i332 = 0; _i332 < _list330.size; ++_i332)
                   {
-                    _elem311 = new TabletStats();
-                    _elem311.read(iprot);
-                    struct.success.add(_elem311);
+                    _elem331 = new TabletStats();
+                    _elem331.read(iprot);
+                    struct.success.add(_elem331);
                   }
                   iprot.readListEnd();
                 }
@@ -33007,9 +33327,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, getTabletStats_res
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size()));
-            for (TabletStats _iter313 : struct.success)
+            for (TabletStats _iter333 : struct.success)
             {
-              _iter313.write(oprot);
+              _iter333.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -33048,9 +33368,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, getTabletStats_resu
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (TabletStats _iter314 : struct.success)
+            for (TabletStats _iter334 : struct.success)
             {
-              _iter314.write(oprot);
+              _iter334.write(oprot);
             }
           }
         }
@@ -33065,14 +33385,14 @@ public void read(org.apache.thrift.protocol.TProtocol prot, getTabletStats_resul
         java.util.BitSet incoming = iprot.readBitSet(2);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list315 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.success = new java.util.ArrayList<TabletStats>(_list315.size);
-            TabletStats _elem316;
-            for (int _i317 = 0; _i317 < _list315.size; ++_i317)
+            org.apache.thrift.protocol.TList _list335 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.success = new java.util.ArrayList<TabletStats>(_list335.size);
+            TabletStats _elem336;
+            for (int _i337 = 0; _i337 < _list335.size; ++_i337)
             {
-              _elem316 = new TabletStats();
-              _elem316.read(iprot);
-              struct.success.add(_elem316);
+              _elem336 = new TabletStats();
+              _elem336.read(iprot);
+              struct.success.add(_elem336);
             }
           }
           struct.setSuccessIsSet(true);
@@ -36434,14 +36754,14 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, getActiveScans_resu
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list318 = iprot.readListBegin();
-                  struct.success = new java.util.ArrayList<ActiveScan>(_list318.size);
-                  ActiveScan _elem319;
-                  for (int _i320 = 0; _i320 < _list318.size; ++_i320)
+                  org.apache.thrift.protocol.TList _list338 = iprot.readListBegin();
+                  struct.success = new java.util.ArrayList<ActiveScan>(_list338.size);
+                  ActiveScan _elem339;
+                  for (int _i340 = 0; _i340 < _list338.size; ++_i340)
                   {
-                    _elem319 = new ActiveScan();
-                    _elem319.read(iprot);
-                    struct.success.add(_elem319);
+                    _elem339 = new ActiveScan();
+                    _elem339.read(iprot);
+                    struct.success.add(_elem339);
                   }
                   iprot.readListEnd();
                 }
@@ -36478,9 +36798,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, getActiveScans_res
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size()));
-            for (ActiveScan _iter321 : struct.success)
+            for (ActiveScan _iter341 : struct.success)
             {
-              _iter321.write(oprot);
+              _iter341.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -36519,9 +36839,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, getActiveScans_resu
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (ActiveScan _iter322 : struct.success)
+            for (ActiveScan _iter342 : struct.success)
             {
-              _iter322.write(oprot);
+              _iter342.write(oprot);
             }
           }
         }
@@ -36536,14 +36856,14 @@ public void read(org.apache.thrift.protocol.TProtocol prot, getActiveScans_resul
         java.util.BitSet incoming = iprot.readBitSet(2);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list323 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.success = new java.util.ArrayList<ActiveScan>(_list323.size);
-            ActiveScan _elem324;
-            for (int _i325 = 0; _i325 < _list323.size; ++_i325)
+            org.apache.thrift.protocol.TList _list343 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.success = new java.util.ArrayList<ActiveScan>(_list343.size);
+            ActiveScan _elem344;
+            for (int _i345 = 0; _i345 < _list343.size; ++_i345)
             {
-              _elem324 = new ActiveScan();
-              _elem324.read(iprot);
-              struct.success.add(_elem324);
+              _elem344 = new ActiveScan();
+              _elem344.read(iprot);
+              struct.success.add(_elem344);
             }
           }
           struct.setSuccessIsSet(true);
@@ -37432,14 +37752,14 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, getActiveCompaction
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list326 = iprot.readListBegin();
-                  struct.success = new java.util.ArrayList<ActiveCompaction>(_list326.size);
-                  ActiveCompaction _elem327;
-                  for (int _i328 = 0; _i328 < _list326.size; ++_i328)
+                  org.apache.thrift.protocol.TList _list346 = iprot.readListBegin();
+                  struct.success = new java.util.ArrayList<ActiveCompaction>(_list346.size);
+                  ActiveCompaction _elem347;
+                  for (int _i348 = 0; _i348 < _list346.size; ++_i348)
                   {
-                    _elem327 = new ActiveCompaction();
-                    _elem327.read(iprot);
-                    struct.success.add(_elem327);
+                    _elem347 = new ActiveCompaction();
+                    _elem347.read(iprot);
+                    struct.success.add(_elem347);
                   }
                   iprot.readListEnd();
                 }
@@ -37476,9 +37796,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, getActiveCompactio
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size()));
-            for (ActiveCompaction _iter329 : struct.success)
+            for (ActiveCompaction _iter349 : struct.success)
             {
-              _iter329.write(oprot);
+              _iter349.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -37517,9 +37837,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, getActiveCompaction
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (ActiveCompaction _iter330 : struct.success)
+            for (ActiveCompaction _iter350 : struct.success)
             {
-              _iter330.write(oprot);
+              _iter350.write(oprot);
             }
           }
         }
@@ -37534,14 +37854,14 @@ public void read(org.apache.thrift.protocol.TProtocol prot, getActiveCompactions
         java.util.BitSet incoming = iprot.readBitSet(2);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list331 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.success = new java.util.ArrayList<ActiveCompaction>(_list331.size);
-            ActiveCompaction _elem332;
-            for (int _i333 = 0; _i333 < _list331.size; ++_i333)
+            org.apache.thrift.protocol.TList _list351 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.success = new java.util.ArrayList<ActiveCompaction>(_list351.size);
+            ActiveCompaction _elem352;
+            for (int _i353 = 0; _i353 < _list351.size; ++_i353)
             {
-              _elem332 = new ActiveCompaction();
-              _elem332.read(iprot);
-              struct.success.add(_elem332);
+              _elem352 = new ActiveCompaction();
+              _elem352.read(iprot);
+              struct.success.add(_elem352);
             }
           }
           struct.setSuccessIsSet(true);
@@ -38056,13 +38376,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, removeLogs_args str
             case 3: // FILENAMES
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list334 = iprot.readListBegin();
-                  struct.filenames = new java.util.ArrayList<java.lang.String>(_list334.size);
-                  java.lang.String _elem335;
-                  for (int _i336 = 0; _i336 < _list334.size; ++_i336)
+                  org.apache.thrift.protocol.TList _list354 = iprot.readListBegin();
+                  struct.filenames = new java.util.ArrayList<java.lang.String>(_list354.size);
+                  java.lang.String _elem355;
+                  for (int _i356 = 0; _i356 < _list354.size; ++_i356)
                   {
-                    _elem335 = iprot.readString();
-                    struct.filenames.add(_elem335);
+                    _elem355 = iprot.readString();
+                    struct.filenames.add(_elem355);
                   }
                   iprot.readListEnd();
                 }
@@ -38100,9 +38420,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, removeLogs_args st
           oprot.writeFieldBegin(FILENAMES_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.filenames.size()));
-            for (java.lang.String _iter337 : struct.filenames)
+            for (java.lang.String _iter357 : struct.filenames)
             {
-              oprot.writeString(_iter337);
+              oprot.writeString(_iter357);
             }
             oprot.writeListEnd();
           }
@@ -38145,9 +38465,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, removeLogs_args str
         if (struct.isSetFilenames()) {
           {
             oprot.writeI32(struct.filenames.size());
-            for (java.lang.String _iter338 : struct.filenames)
+            for (java.lang.String _iter358 : struct.filenames)
             {
-              oprot.writeString(_iter338);
+              oprot.writeString(_iter358);
             }
           }
         }
@@ -38169,13 +38489,13 @@ public void read(org.apache.thrift.protocol.TProtocol prot, removeLogs_args stru
         }
         if (incoming.get(2)) {
           {
-            org.apache.thrift.protocol.TList _list339 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.filenames = new java.util.ArrayList<java.lang.String>(_list339.size);
-            java.lang.String _elem340;
-            for (int _i341 = 0; _i341 < _list339.size; ++_i341)
+            org.apache.thrift.protocol.TList _list359 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.filenames = new java.util.ArrayList<java.lang.String>(_list359.size);
+            java.lang.String _elem360;
+            for (int _i361 = 0; _i361 < _list359.size; ++_i361)
             {
-              _elem340 = iprot.readString();
-              struct.filenames.add(_elem340);
+              _elem360 = iprot.readString();
+              struct.filenames.add(_elem360);
             }
           }
           struct.setFilenamesIsSet(true);
@@ -38975,13 +39295,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, getActiveLogs_resul
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list342 = iprot.readListBegin();
-                  struct.success = new java.util.ArrayList<java.lang.String>(_list342.size);
-                  java.lang.String _elem343;
-                  for (int _i344 = 0; _i344 < _list342.size; ++_i344)
+                  org.apache.thrift.protocol.TList _list362 = iprot.readListBegin();
+                  struct.success = new java.util.ArrayList<java.lang.String>(_list362.size);
+                  java.lang.String _elem363;
+                  for (int _i364 = 0; _i364 < _list362.size; ++_i364)
                   {
-                    _elem343 = iprot.readString();
-                    struct.success.add(_elem343);
+                    _elem363 = iprot.readString();
+                    struct.success.add(_elem363);
                   }
                   iprot.readListEnd();
                 }
@@ -39009,9 +39329,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, getActiveLogs_resu
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.success.size()));
-            for (java.lang.String _iter345 : struct.success)
+            for (java.lang.String _iter365 : struct.success)
             {
-              oprot.writeString(_iter345);
+              oprot.writeString(_iter365);
             }
             oprot.writeListEnd();
           }
@@ -39042,9 +39362,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, getActiveLogs_resul
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (java.lang.String _iter346 : struct.success)
+            for (java.lang.String _iter366 : struct.success)
             {
-              oprot.writeString(_iter346);
+              oprot.writeString(_iter366);
             }
           }
         }
@@ -39056,13 +39376,13 @@ public void read(org.apache.thrift.protocol.TProtocol prot, getActiveLogs_result
         java.util.BitSet incoming = iprot.readBitSet(1);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list347 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.success = new java.util.ArrayList<java.lang.String>(_list347.size);
-            java.lang.String _elem348;
-            for (int _i349 = 0; _i349 < _list347.size; ++_i349)
+            org.apache.thrift.protocol.TList _list367 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.success = new java.util.ArrayList<java.lang.String>(_list367.size);
+            java.lang.String _elem368;
+            for (int _i369 = 0; _i369 < _list367.size; ++_i369)
             {
-              _elem348 = iprot.readString();
-              struct.success.add(_elem348);
+              _elem368 = iprot.readString();
+              struct.success.add(_elem368);
             }
           }
           struct.setSuccessIsSet(true);
@@ -42095,26 +42415,26 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, startGetSummariesFr
             case 4: // FILES
               if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
                 {
-                  org.apache.thrift.protocol.TMap _map350 = iprot.readMapBegin();
-                  struct.files = new java.util.HashMap<java.lang.String,java.util.List<org.apache.accumulo.core.data.thrift.TRowRange>>(2*_map350.size);
-                  java.lang.String _key351;
-                  java.util.List<org.apache.accumulo.core.data.thrift.TRowRange> _val352;
-                  for (int _i353 = 0; _i353 < _map350.size; ++_i353)
+                  org.apache.thrift.protocol.TMap _map370 = iprot.readMapBegin();
+                  struct.files = new java.util.HashMap<java.lang.String,java.util.List<org.apache.accumulo.core.data.thrift.TRowRange>>(2*_map370.size);
+                  java.lang.String _key371;
+                  java.util.List<org.apache.accumulo.core.data.thrift.TRowRange> _val372;
+                  for (int _i373 = 0; _i373 < _map370.size; ++_i373)
                   {
-                    _key351 = iprot.readString();
+                    _key371 = iprot.readString();
                     {
-                      org.apache.thrift.protocol.TList _list354 = iprot.readListBegin();
-                      _val352 = new java.util.ArrayList<org.apache.accumulo.core.data.thrift.TRowRange>(_list354.size);
-                      org.apache.accumulo.core.data.thrift.TRowRange _elem355;
-                      for (int _i356 = 0; _i356 < _list354.size; ++_i356)
+                      org.apache.thrift.protocol.TList _list374 = iprot.readListBegin();
+                      _val372 = new java.util.ArrayList<org.apache.accumulo.core.data.thrift.TRowRange>(_list374.size);
+                      org.apache.accumulo.core.data.thrift.TRowRange _elem375;
+                      for (int _i376 = 0; _i376 < _list374.size; ++_i376)
                       {
-                        _elem355 = new org.apache.accumulo.core.data.thrift.TRowRange();
-                        _elem355.read(iprot);
-                        _val352.add(_elem355);
+                        _elem375 = new org.apache.accumulo.core.data.thrift.TRowRange();
+                        _elem375.read(iprot);
+                        _val372.add(_elem375);
                       }
                       iprot.readListEnd();
                     }
-                    struct.files.put(_key351, _val352);
+                    struct.files.put(_key371, _val372);
                   }
                   iprot.readMapEnd();
                 }
@@ -42157,14 +42477,14 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, startGetSummariesF
           oprot.writeFieldBegin(FILES_FIELD_DESC);
           {
             oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.LIST, struct.files.size()));
-            for (java.util.Map.Entry<java.lang.String, java.util.List<org.apache.accumulo.core.data.thrift.TRowRange>> _iter357 : struct.files.entrySet())
+            for (java.util.Map.Entry<java.lang.String, java.util.List<org.apache.accumulo.core.data.thrift.TRowRange>> _iter377 : struct.files.entrySet())
             {
-              oprot.writeString(_iter357.getKey());
+              oprot.writeString(_iter377.getKey());
               {
-                oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, _iter357.getValue().size()));
-                for (org.apache.accumulo.core.data.thrift.TRowRange _iter358 : _iter357.getValue())
+                oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, _iter377.getValue().size()));
+                for (org.apache.accumulo.core.data.thrift.TRowRange _iter378 : _iter377.getValue())
                 {
-                  _iter358.write(oprot);
+                  _iter378.write(oprot);
                 }
                 oprot.writeListEnd();
               }
@@ -42216,14 +42536,14 @@ public void write(org.apache.thrift.protocol.TProtocol prot, startGetSummariesFr
         if (struct.isSetFiles()) {
           {
             oprot.writeI32(struct.files.size());
-            for (java.util.Map.Entry<java.lang.String, java.util.List<org.apache.accumulo.core.data.thrift.TRowRange>> _iter359 : struct.files.entrySet())
+            for (java.util.Map.Entry<java.lang.String, java.util.List<org.apache.accumulo.core.data.thrift.TRowRange>> _iter379 : struct.files.entrySet())
             {
-              oprot.writeString(_iter359.getKey());
+              oprot.writeString(_iter379.getKey());
               {
-                oprot.writeI32(_iter359.getValue().size());
-                for (org.apache.accumulo.core.data.thrift.TRowRange _iter360 : _iter359.getValue())
+                oprot.writeI32(_iter379.getValue().size());
+                for (org.apache.accumulo.core.data.thrift.TRowRange _iter380 : _iter379.getValue())
                 {
-                  _iter360.write(oprot);
+                  _iter380.write(oprot);
                 }
               }
             }
@@ -42252,25 +42572,25 @@ public void read(org.apache.thrift.protocol.TProtocol prot, startGetSummariesFro
         }
         if (incoming.get(3)) {
           {
-            org.apache.thrift.protocol.TMap _map361 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.LIST, iprot.readI32());
-            struct.files = new java.util.HashMap<java.lang.String,java.util.List<org.apache.accumulo.core.data.thrift.TRowRange>>(2*_map361.size);
-            java.lang.String _key362;
-            java.util.List<org.apache.accumulo.core.data.thrift.TRowRange> _val363;
-            for (int _i364 = 0; _i364 < _map361.size; ++_i364)
+            org.apache.thrift.protocol.TMap _map381 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.LIST, iprot.readI32());
+            struct.files = new java.util.HashMap<java.lang.String,java.util.List<org.apache.accumulo.core.data.thrift.TRowRange>>(2*_map381.size);
+            java.lang.String _key382;
+            java.util.List<org.apache.accumulo.core.data.thrift.TRowRange> _val383;
+            for (int _i384 = 0; _i384 < _map381.size; ++_i384)
             {
-              _key362 = iprot.readString();
+              _key382 = iprot.readString();
               {
-                org.apache.thrift.protocol.TList _list365 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-                _val363 = new java.util.ArrayList<org.apache.accumulo.core.data.thrift.TRowRange>(_list365.size);
-                org.apache.accumulo.core.data.thrift.TRowRange _elem366;
-                for (int _i367 = 0; _i367 < _list365.size; ++_i367)
+                org.apache.thrift.protocol.TList _list385 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+                _val383 = new java.util.ArrayList<org.apache.accumulo.core.data.thrift.TRowRange>(_list385.size);
+                org.apache.accumulo.core.data.thrift.TRowRange _elem386;
+                for (int _i387 = 0; _i387 < _list385.size; ++_i387)
                 {
-                  _elem366 = new org.apache.accumulo.core.data.thrift.TRowRange();
-                  _elem366.read(iprot);
-                  _val363.add(_elem366);
+                  _elem386 = new org.apache.accumulo.core.data.thrift.TRowRange();
+                  _elem386.read(iprot);
+                  _val383.add(_elem386);
                 }
               }
-              struct.files.put(_key362, _val363);
+              struct.files.put(_key382, _val383);
             }
           }
           struct.setFilesIsSet(true);
diff --git a/core/src/main/thrift/tabletserver.thrift b/core/src/main/thrift/tabletserver.thrift
index 2b7372f01b..44e69ba0a5 100644
--- a/core/src/main/thrift/tabletserver.thrift
+++ b/core/src/main/thrift/tabletserver.thrift
@@ -169,7 +169,8 @@ service TabletClientService extends client.ClientService {
                              12:i64 readaheadThreshold,
                              13:TSamplerConfiguration samplerConfig,
                              14:i64 batchTimeOut,
-                             15:string classLoaderContext /* name of the classloader context */)  throws (1:client.ThriftSecurityException sec, 2:NotServingTabletException nste, 3:TooManyFilesException tmfe, 4:TSampleNotPresentException tsnpe),
+                             15:string classLoaderContext, /* name of the classloader context */
+                             16:map<string,string> executionHints)  throws (1:client.ThriftSecurityException sec, 2:NotServingTabletException nste, 3:TooManyFilesException tmfe, 4:TSampleNotPresentException tsnpe),
                              
   data.ScanResult continueScan(2:trace.TInfo tinfo, 1:data.ScanID scanID)  throws (1:NoSuchScanIDException nssi, 2:NotServingTabletException nste, 3:TooManyFilesException tmfe, 4:TSampleNotPresentException tsnpe),
   oneway void closeScan(2:trace.TInfo tinfo, 1:data.ScanID scanID),
@@ -185,7 +186,8 @@ service TabletClientService extends client.ClientService {
                                   7:bool waitForWrites,
                                   9:TSamplerConfiguration samplerConfig,
                                   10:i64 batchTimeOut,
-                                  11:string classLoaderContext /* name of the classloader context */)  throws (1:client.ThriftSecurityException sec, 2:TSampleNotPresentException tsnpe),
+                                  11:string classLoaderContext, /* name of the classloader context */
+                                  12:map<string, string> executionHints)  throws (1:client.ThriftSecurityException sec, 2:TSampleNotPresentException tsnpe),
   data.MultiScanResult continueMultiScan(2:trace.TInfo tinfo, 1:data.ScanID scanID) throws (1:NoSuchScanIDException nssi, 2:TSampleNotPresentException tsnpe),
   void closeMultiScan(2:trace.TInfo tinfo, 1:data.ScanID scanID) throws (1:NoSuchScanIDException nssi),
   
diff --git a/core/src/test/java/org/apache/accumulo/core/spi/scan/SimpleScanDispatcherTest.java b/core/src/test/java/org/apache/accumulo/core/spi/scan/SimpleScanDispatcherTest.java
index b59858a9c0..63f21ac58a 100644
--- a/core/src/test/java/org/apache/accumulo/core/spi/scan/SimpleScanDispatcherTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/spi/scan/SimpleScanDispatcherTest.java
@@ -17,6 +17,7 @@
 package org.apache.accumulo.core.spi.scan;
 
 import java.util.Collections;
+import java.util.HashMap;
 import java.util.Map;
 
 import org.apache.accumulo.core.conf.Property;
@@ -35,14 +36,27 @@ public void testProps() {
         .endsWith(SimpleScanDispatcher.DEFAULT_SCAN_EXECUTOR_NAME + ".prioritizer"));
   }
 
-  private void runTest(Map<String,String> opts, String expectedSingle, String expectedMulti) {
-    ScanInfo msi = new TestScanInfo("a", Type.MULTI, 4);
-    ScanInfo ssi = new TestScanInfo("a", Type.SINGLE, 4);
+  private void runTest(Map<String,String> opts, Map<String,String> hints, String expectedSingle,
+      String expectedMulti) {
+    TestScanInfo msi = new TestScanInfo("a", Type.MULTI, 4);
+    msi.executionHints = hints;
+    TestScanInfo ssi = new TestScanInfo("a", Type.SINGLE, 4);
+    ssi.executionHints = hints;
 
     SimpleScanDispatcher ssd1 = new SimpleScanDispatcher();
     ssd1.init(opts);
-    Assert.assertEquals(expectedMulti, ssd1.dispatch(msi, null));
-    Assert.assertEquals(expectedSingle, ssd1.dispatch(ssi, null));
+
+    Map<String,ScanExecutor> executors = new HashMap<>();
+    executors.put("E1", null);
+    executors.put("E2", null);
+    executors.put("E3", null);
+
+    Assert.assertEquals(expectedMulti, ssd1.dispatch(msi, executors));
+    Assert.assertEquals(expectedSingle, ssd1.dispatch(ssi, executors));
+  }
+
+  private void runTest(Map<String,String> opts, String expectedSingle, String expectedMulti) {
+    runTest(opts, Collections.emptyMap(), expectedSingle, expectedMulti);
   }
 
   @Test
@@ -59,4 +73,32 @@ public void testBasic() {
     runTest(ImmutableMap.of("executor", "E1", "single_executor", "E2", "multi_executor", "E3"),
         "E2", "E3");
   }
+
+  @Test
+  public void testHints() {
+    runTest(ImmutableMap.of("executor", "E1", "heed_hints", "true"), ImmutableMap.of(), "E1", "E1");
+    runTest(ImmutableMap.of("executor", "E1", "heed_hints", "true"),
+        ImmutableMap.of("executor", "E2"), "E2", "E2");
+    runTest(ImmutableMap.of("executor", "E1", "heed_hints", "true"),
+        ImmutableMap.of("executor", "E5"), "E1", "E1");
+    runTest(ImmutableMap.of("executor", "E1", "heed_hints", "true", "ignored_hint_action", "fail"),
+        ImmutableMap.of("executor", "E5"), "E1", "E1");
+    runTest(ImmutableMap.of("executor", "E1", "heed_hints", "true", "bad_hint_action", "fail",
+        "ignored_hint_action", "fail"), ImmutableMap.of("executor", "E2"), "E2", "E2");
+    runTest(ImmutableMap.of("executor", "E1", "heed_hints", "false"),
+        ImmutableMap.of("executor", "E2"), "E1", "E1");
+    runTest(ImmutableMap.of("executor", "E1"), ImmutableMap.of("executor", "E2"), "E1", "E1");
+  }
+
+  @Test(expected = IllegalArgumentException.class)
+  public void testBadHint() {
+    runTest(ImmutableMap.of("executor", "E1", "heed_hints", "true", "bad_hint_action", "fail"),
+        ImmutableMap.of("executor", "E5"), "E2", "E2");
+  }
+
+  @Test(expected = IllegalArgumentException.class)
+  public void testIgnoredHint() {
+    runTest(ImmutableMap.of("executor", "E1", "heed_hints", "false", "ignored_hint_action", "fail"),
+        ImmutableMap.of("executor", "E2"), "E1", "E1");
+  }
 }
diff --git a/core/src/test/java/org/apache/accumulo/core/spi/scan/TestScanInfo.java b/core/src/test/java/org/apache/accumulo/core/spi/scan/TestScanInfo.java
index 68a96502f8..ee3db08bb4 100644
--- a/core/src/test/java/org/apache/accumulo/core/spi/scan/TestScanInfo.java
+++ b/core/src/test/java/org/apache/accumulo/core/spi/scan/TestScanInfo.java
@@ -18,6 +18,8 @@
 package org.apache.accumulo.core.spi.scan;
 
 import java.util.Collection;
+import java.util.Collections;
+import java.util.Map;
 import java.util.OptionalLong;
 import java.util.Set;
 
@@ -34,6 +36,7 @@
   OptionalLong lastRunTime = OptionalLong.empty();
   Stat runTimeStats = new Stat();
   Stat idleTimeStats = new Stat();
+  Map<String,String> executionHints = Collections.emptyMap();
 
   TestScanInfo(String testId, Type scanType, long creationTime, int... times) {
     this.testId = testId;
@@ -98,4 +101,9 @@ public Stats getIdleTimeStats(long currentTime) {
   public Collection<IteratorConfiguration> getClientScanIterators() {
     throw new UnsupportedOperationException();
   }
+
+  @Override
+  public Map<String,String> getExecutionHints() {
+    return executionHints;
+  }
 }
diff --git a/server/base/src/main/java/org/apache/accumulo/server/util/VerifyTabletAssignments.java b/server/base/src/main/java/org/apache/accumulo/server/util/VerifyTabletAssignments.java
index a133a1ed70..5e5b41ab2a 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/util/VerifyTabletAssignments.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/util/VerifyTabletAssignments.java
@@ -194,7 +194,7 @@ private static void checkTabletServer(ClientContext context,
     List<TColumn> emptyListColumn = Collections.emptyList();
     InitialMultiScan is = client.startMultiScan(tinfo, context.rpcCreds(), batch, emptyListColumn,
         emptyListIterInfo, emptyMapSMapSS, Authorizations.EMPTY.getAuthorizationsBB(), false, null,
-        0L, null);
+        0L, null, null);
     if (is.result.more) {
       MultiScanResult result = client.continueMultiScan(tinfo, is.scanID);
       checkFailures(entry.getKey(), failures, result);
diff --git a/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/tservers/TabletServerResource.java b/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/tservers/TabletServerResource.java
index da4fcc62d9..b2584ec4b1 100644
--- a/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/tservers/TabletServerResource.java
+++ b/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/tservers/TabletServerResource.java
@@ -41,7 +41,6 @@
 import org.apache.accumulo.core.client.impl.ClientContext;
 import org.apache.accumulo.core.client.impl.Table;
 import org.apache.accumulo.core.client.impl.Tables;
-import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.data.impl.KeyExtent;
 import org.apache.accumulo.core.master.thrift.MasterMonitorInfo;
 import org.apache.accumulo.core.master.thrift.RecoveryStatus;
@@ -233,7 +232,7 @@ public TabletServerSummary getTserverDetails(
   }
 
   private static final int concurrentScans = Monitor.getContext().getConfiguration()
-      .getCount(Property.TSERV_READ_AHEAD_MAXCONCURRENT);
+      .getScanExecutors().stream().mapToInt(sec -> sec.maxThreads).sum();
 
   /**
    * Generates the server stats
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java
index 86f51660ac..25f39322b9 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java
@@ -559,8 +559,9 @@ public InitialScan startScan(TInfo tinfo, TCredentials credentials, TKeyExtent t
         TRange range, List<TColumn> columns, int batchSize, List<IterInfo> ssiList,
         Map<String,Map<String,String>> ssio, List<ByteBuffer> authorizations, boolean waitForWrites,
         boolean isolated, long readaheadThreshold, TSamplerConfiguration tSamplerConfig,
-        long batchTimeOut, String context) throws NotServingTabletException,
-        ThriftSecurityException, org.apache.accumulo.core.tabletserver.thrift.TooManyFilesException,
+        long batchTimeOut, String context, Map<String,String> executionHints)
+        throws NotServingTabletException, ThriftSecurityException,
+        org.apache.accumulo.core.tabletserver.thrift.TooManyFilesException,
         TSampleNotPresentException {
 
       Table.ID tableId = Table.ID.of(new String(textent.getTable(), UTF_8));
@@ -605,7 +606,7 @@ public InitialScan startScan(TInfo tinfo, TCredentials credentials, TKeyExtent t
 
       final SingleScanSession scanSession = new SingleScanSession(credentials, extent, columnSet,
           ssiList, ssio, new Authorizations(authorizations), readaheadThreshold, batchTimeOut,
-          context);
+          context, executionHints);
       scanSession.scanner = tablet.createScanner(new Range(range), batchSize, scanSession.columnSet,
           scanSession.auths, ssiList, ssio, isolated, scanSession.interruptFlag,
           SamplerConfigurationImpl.fromThrift(tSamplerConfig), scanSession.batchTimeOut,
@@ -740,7 +741,8 @@ public void closeScan(TInfo tinfo, long scanID) {
     public InitialMultiScan startMultiScan(TInfo tinfo, TCredentials credentials,
         Map<TKeyExtent,List<TRange>> tbatch, List<TColumn> tcolumns, List<IterInfo> ssiList,
         Map<String,Map<String,String>> ssio, List<ByteBuffer> authorizations, boolean waitForWrites,
-        TSamplerConfiguration tSamplerConfig, long batchTimeOut, String context)
+        TSamplerConfiguration tSamplerConfig, long batchTimeOut, String context,
+        Map<String,String> executionHints)
         throws ThriftSecurityException, TSampleNotPresentException {
       // find all of the tables that need to be scanned
       final HashSet<Table.ID> tables = new HashSet<>();
@@ -785,7 +787,8 @@ public InitialMultiScan startMultiScan(TInfo tinfo, TCredentials credentials,
 
       final MultiScanSession mss = new MultiScanSession(credentials, threadPoolExtent, batch,
           ssiList, ssio, new Authorizations(authorizations),
-          SamplerConfigurationImpl.fromThrift(tSamplerConfig), batchTimeOut, context);
+          SamplerConfigurationImpl.fromThrift(tSamplerConfig), batchTimeOut, context,
+          executionHints);
 
       mss.numTablets = batch.size();
       for (List<Range> ranges : batch.values()) {
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/session/MultiScanSession.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/session/MultiScanSession.java
index f8db1f42c5..27ec7f3b7b 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/session/MultiScanSession.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/session/MultiScanSession.java
@@ -47,8 +47,9 @@
   public MultiScanSession(TCredentials credentials, KeyExtent threadPoolExtent,
       Map<KeyExtent,List<Range>> queries, List<IterInfo> ssiList,
       Map<String,Map<String,String>> ssio, Authorizations authorizations,
-      SamplerConfiguration samplerConfig, long batchTimeOut, String context) {
-    super(credentials, new HashSet<>(), ssiList, ssio, authorizations);
+      SamplerConfiguration samplerConfig, long batchTimeOut, String context,
+      Map<String,String> executionHints) {
+    super(credentials, new HashSet<>(), ssiList, ssio, authorizations, executionHints);
     this.queries = queries;
     this.threadPoolExtent = threadPoolExtent;
     this.samplerConfig = samplerConfig;
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/session/ScanSession.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/session/ScanSession.java
index 5cde06c657..b062e34de6 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/session/ScanSession.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/session/ScanSession.java
@@ -73,14 +73,21 @@ public static ScanMeasurer wrap(ScanSession scanInfo, Runnable r) {
   public final List<IterInfo> ssiList;
   public final Map<String,Map<String,String>> ssio;
   public final Authorizations auths;
+  private Map<String,String> executionHints;
 
   ScanSession(TCredentials credentials, HashSet<Column> cols, List<IterInfo> ssiList,
-      Map<String,Map<String,String>> ssio, Authorizations auths) {
+      Map<String,Map<String,String>> ssio, Authorizations auths,
+      Map<String,String> executionHints) {
     super(credentials);
     this.columnSet = cols;
     this.ssiList = ssiList;
     this.ssio = ssio;
     this.auths = auths;
+    if (executionHints == null) {
+      this.executionHints = Collections.emptyMap();
+    } else {
+      this.executionHints = Collections.unmodifiableMap(executionHints);
+    }
   }
 
   @Override
@@ -153,6 +160,11 @@ public int getPriority() {
     return Lists.transform(ssiList, IterConfImpl::new);
   }
 
+  @Override
+  public Map<String,String> getExecutionHints() {
+    return executionHints;
+  }
+
   public void finishedRun(long start, long finish) {
     long idleTime = start - getLastRunTime().orElse(getCreationTime());
     long runTime = finish - start;
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/session/SingleScanSession.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/session/SingleScanSession.java
index fb3e29fe56..323d28323d 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/session/SingleScanSession.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/session/SingleScanSession.java
@@ -43,8 +43,9 @@
 
   public SingleScanSession(TCredentials credentials, KeyExtent extent, HashSet<Column> columnSet,
       List<IterInfo> ssiList, Map<String,Map<String,String>> ssio, Authorizations authorizations,
-      long readaheadThreshold, long batchTimeOut, String context) {
-    super(credentials, columnSet, ssiList, ssio, authorizations);
+      long readaheadThreshold, long batchTimeOut, String context,
+      Map<String,String> executionHints) {
+    super(credentials, columnSet, ssiList, ssio, authorizations, executionHints);
     this.extent = extent;
     this.readaheadThreshold = readaheadThreshold;
     this.batchTimeOut = batchTimeOut;
diff --git a/shell/src/main/java/org/apache/accumulo/shell/ShellUtil.java b/shell/src/main/java/org/apache/accumulo/shell/ShellUtil.java
index 1333087362..6587ccd315 100644
--- a/shell/src/main/java/org/apache/accumulo/shell/ShellUtil.java
+++ b/shell/src/main/java/org/apache/accumulo/shell/ShellUtil.java
@@ -22,11 +22,18 @@
 import java.io.FileNotFoundException;
 import java.util.ArrayList;
 import java.util.Base64;
+import java.util.Collections;
 import java.util.List;
+import java.util.Map;
 import java.util.Scanner;
 
+import org.apache.commons.cli.CommandLine;
+import org.apache.commons.cli.Option;
 import org.apache.hadoop.io.Text;
 
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.ImmutableMap.Builder;
+
 public class ShellUtil {
 
   /**
@@ -55,4 +62,20 @@
     }
     return result;
   }
+
+  public static Map<String,String> parseMapOpt(CommandLine cl, Option opt) {
+    if (cl.hasOption(opt.getLongOpt())) {
+      Builder<String,String> builder = ImmutableMap.builder();
+      String[] keyVals = cl.getOptionValue(opt.getLongOpt()).split(",");
+      for (String keyVal : keyVals) {
+        String[] sa = keyVal.split("=");
+        builder.put(sa[0], sa[1]);
+      }
+
+      return builder.build();
+    } else {
+      return Collections.emptyMap();
+    }
+
+  }
 }
diff --git a/shell/src/main/java/org/apache/accumulo/shell/commands/CloneTableCommand.java b/shell/src/main/java/org/apache/accumulo/shell/commands/CloneTableCommand.java
index 46c0d6626f..1c6441db76 100644
--- a/shell/src/main/java/org/apache/accumulo/shell/commands/CloneTableCommand.java
+++ b/shell/src/main/java/org/apache/accumulo/shell/commands/CloneTableCommand.java
@@ -16,7 +16,6 @@
  */
 package org.apache.accumulo.shell.commands;
 
-import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Map;
 import java.util.Set;
@@ -27,6 +26,7 @@
 import org.apache.accumulo.core.client.TableNotFoundException;
 import org.apache.accumulo.shell.Shell;
 import org.apache.accumulo.shell.Shell.Command;
+import org.apache.accumulo.shell.ShellUtil;
 import org.apache.accumulo.shell.Token;
 import org.apache.commons.cli.CommandLine;
 import org.apache.commons.cli.Option;
@@ -43,18 +43,10 @@ public int execute(final String fullCommand, final CommandLine cl, final Shell s
       throws AccumuloException, AccumuloSecurityException, TableNotFoundException,
       TableExistsException {
 
-    final HashMap<String,String> props = new HashMap<>();
+    Map<String,String> props = ShellUtil.parseMapOpt(cl, setPropsOption);
     final HashSet<String> exclude = new HashSet<>();
     boolean flush = true;
 
-    if (cl.hasOption(setPropsOption.getOpt())) {
-      String[] keyVals = cl.getOptionValue(setPropsOption.getOpt()).split(",");
-      for (String keyVal : keyVals) {
-        String[] sa = keyVal.split("=");
-        props.put(sa[0], sa[1]);
-      }
-    }
-
     if (cl.hasOption(excludePropsOption.getOpt())) {
       String[] keys = cl.getOptionValue(excludePropsOption.getOpt()).split(",");
       for (String key : keys) {
diff --git a/shell/src/main/java/org/apache/accumulo/shell/commands/CompactCommand.java b/shell/src/main/java/org/apache/accumulo/shell/commands/CompactCommand.java
index dfb21e9f9e..d3b41a313b 100644
--- a/shell/src/main/java/org/apache/accumulo/shell/commands/CompactCommand.java
+++ b/shell/src/main/java/org/apache/accumulo/shell/commands/CompactCommand.java
@@ -29,6 +29,7 @@
 import org.apache.accumulo.core.client.admin.CompactionStrategyConfig;
 import org.apache.accumulo.core.compaction.CompactionSettings;
 import org.apache.accumulo.shell.Shell;
+import org.apache.accumulo.shell.ShellUtil;
 import org.apache.commons.cli.CommandLine;
 import org.apache.commons.cli.Option;
 import org.apache.commons.cli.Options;
@@ -155,16 +156,8 @@ public int execute(final String fullCommand, final CommandLine cl, final Shell s
 
       CompactionStrategyConfig csc = new CompactionStrategyConfig(
           cl.getOptionValue(strategyOpt.getOpt()));
-      if (cl.hasOption(strategyConfigOpt.getOpt())) {
-        Map<String,String> props = new HashMap<>();
-        String[] keyVals = cl.getOptionValue(strategyConfigOpt.getOpt()).split(",");
-        for (String keyVal : keyVals) {
-          String[] sa = keyVal.split("=");
-          props.put(sa[0], sa[1]);
-        }
 
-        csc.setOptions(props);
-      }
+      csc.setOptions(ShellUtil.parseMapOpt(cl, strategyConfigOpt));
 
       compactionConfig.setCompactionStrategy(csc);
     }
diff --git a/shell/src/main/java/org/apache/accumulo/shell/commands/CreateTableCommand.java b/shell/src/main/java/org/apache/accumulo/shell/commands/CreateTableCommand.java
index 596196c4ba..836ffbdf8a 100644
--- a/shell/src/main/java/org/apache/accumulo/shell/commands/CreateTableCommand.java
+++ b/shell/src/main/java/org/apache/accumulo/shell/commands/CreateTableCommand.java
@@ -48,7 +48,6 @@
 import org.apache.commons.cli.Option;
 import org.apache.commons.cli.OptionGroup;
 import org.apache.commons.cli.Options;
-import org.apache.commons.lang.StringUtils;
 import org.apache.hadoop.io.Text;
 
 public class CreateTableCommand extends Command {
@@ -71,7 +70,6 @@ public int execute(final String fullCommand, final CommandLine cl, final Shell s
       TableNotFoundException, IOException, ClassNotFoundException {
 
     final String testTableName = cl.getArgs()[0];
-    final HashMap<String,String> props = new HashMap<>();
     NewTableConfiguration ntc = new NewTableConfiguration();
 
     if (!testTableName.matches(Tables.VALID_NAME_REGEX)) {
@@ -110,13 +108,7 @@ public int execute(final String fullCommand, final CommandLine cl, final Shell s
       timeType = TimeType.LOGICAL;
     }
 
-    if (cl.hasOption(createTableOptInitProp.getOpt())) {
-      String[] keyVals = StringUtils.split(cl.getOptionValue(createTableOptInitProp.getOpt()), ',');
-      for (String keyVal : keyVals) {
-        String[] sa = StringUtils.split(keyVal, '=');
-        props.put(sa[0], sa[1]);
-      }
-    }
+    Map<String,String> props = ShellUtil.parseMapOpt(cl, createTableOptInitProp);
 
     // Set iterator if supplied
     if (cl.hasOption(createTableOptIteratorProps.getOpt())) {
diff --git a/shell/src/main/java/org/apache/accumulo/shell/commands/ScanCommand.java b/shell/src/main/java/org/apache/accumulo/shell/commands/ScanCommand.java
index 5be80819af..a986147572 100644
--- a/shell/src/main/java/org/apache/accumulo/shell/commands/ScanCommand.java
+++ b/shell/src/main/java/org/apache/accumulo/shell/commands/ScanCommand.java
@@ -47,6 +47,7 @@
 import org.apache.accumulo.shell.Shell.PrintFile;
 import org.apache.accumulo.shell.ShellCommandException;
 import org.apache.accumulo.shell.ShellCommandException.ErrorCode;
+import org.apache.accumulo.shell.ShellUtil;
 import org.apache.accumulo.start.classloader.vfs.AccumuloVFSClassLoader;
 import org.apache.commons.cli.CommandLine;
 import org.apache.commons.cli.Option;
@@ -66,6 +67,7 @@
   private Option profileOpt;
   private Option sampleOpt;
   private Option contextOpt;
+  private Option executionHintsOpt;
 
   protected void setupSampling(final String tableName, final CommandLine cl, final Shell shellState,
       ScannerBase scanner)
@@ -116,6 +118,8 @@ public int execute(final String fullCommand, final CommandLine cl, final Shell s
 
       setupSampling(tableName, cl, shellState, scanner);
 
+      scanner.setExecutionHints(ShellUtil.parseMapOpt(cl, executionHintsOpt));
+
       // output the records
 
       final FormatterConfig config = new FormatterConfig();
@@ -347,6 +351,7 @@ public Options getOptions() {
     outputFileOpt = new Option("o", "output", true, "local file to write the scan output to");
     sampleOpt = new Option(null, "sample", false, "Show sample");
     contextOpt = new Option("cc", "context", true, "name of the classloader context");
+    executionHintsOpt = new Option(null, "execution-hints", true, "Execution hints map");
 
     scanOptAuths.setArgName("comma-separated-authorizations");
     scanOptRow.setArgName("row");
@@ -358,6 +363,7 @@ public Options getOptions() {
     timeoutOption.setArgName("timeout");
     outputFileOpt.setArgName("file");
     contextOpt.setArgName("context");
+    executionHintsOpt.setArgName("<key>=<value>{,<key>=<value>}");
 
     profileOpt = new Option("pn", "profile", true, "iterator profile name");
     profileOpt.setArgName("profile");
@@ -389,6 +395,7 @@ public Options getOptions() {
     o.addOption(profileOpt);
     o.addOption(sampleOpt);
     o.addOption(contextOpt);
+    o.addOption(executionHintsOpt);
 
     return o;
   }
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/BloomFilterIT.java b/test/src/main/java/org/apache/accumulo/test/functional/BloomFilterIT.java
index 7a0f848796..350c862939 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/BloomFilterIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/BloomFilterIT.java
@@ -71,8 +71,8 @@ protected int defaultTimeoutSeconds() {
   public void test() throws Exception {
     Connector c = getConnector();
     final String readAhead = c.instanceOperations().getSystemConfiguration()
-        .get(Property.TSERV_READ_AHEAD_MAXCONCURRENT.getKey());
-    c.instanceOperations().setProperty(Property.TSERV_READ_AHEAD_MAXCONCURRENT.getKey(), "1");
+        .get(Property.TSERV_SCAN_EXECUTORS_DEFAULT_THREADS.getKey());
+    c.instanceOperations().setProperty(Property.TSERV_SCAN_EXECUTORS_DEFAULT_THREADS.getKey(), "1");
     try {
       Thread.sleep(1000);
       final String[] tables = getUniqueNames(4);
@@ -160,7 +160,7 @@ public void test() throws Exception {
         }
       }
     } finally {
-      c.instanceOperations().setProperty(Property.TSERV_READ_AHEAD_MAXCONCURRENT.getKey(),
+      c.instanceOperations().setProperty(Property.TSERV_SCAN_EXECUTORS_DEFAULT_THREADS.getKey(),
           readAhead);
     }
   }
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/SessionBlockVerifyIT.java b/test/src/main/java/org/apache/accumulo/test/functional/SessionBlockVerifyIT.java
index 5f34ac9a3b..f6631b112c 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/SessionBlockVerifyIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/SessionBlockVerifyIT.java
@@ -63,7 +63,7 @@ public void configureMiniCluster(MiniAccumuloConfigImpl cfg, Configuration hadoo
     Map<String,String> siteConfig = cfg.getSiteConfig();
     cfg.setNumTservers(1);
     siteConfig.put(Property.TSERV_SESSION_MAXIDLE.getKey(), getMaxIdleTimeString());
-    siteConfig.put(Property.TSERV_READ_AHEAD_MAXCONCURRENT.getKey(), "11");
+    siteConfig.put(Property.TSERV_SCAN_EXECUTORS_DEFAULT_THREADS.getKey(), "11");
     cfg.setSiteConfig(siteConfig);
   }
 
diff --git a/test/src/main/java/org/apache/accumulo/test/performance/NullTserver.java b/test/src/main/java/org/apache/accumulo/test/performance/NullTserver.java
index 7c103d4ee5..9e3197bdd4 100644
--- a/test/src/main/java/org/apache/accumulo/test/performance/NullTserver.java
+++ b/test/src/main/java/org/apache/accumulo/test/performance/NullTserver.java
@@ -150,7 +150,8 @@ public void splitTablet(TInfo tinfo, TCredentials credentials, TKeyExtent extent
     public InitialMultiScan startMultiScan(TInfo tinfo, TCredentials credentials,
         Map<TKeyExtent,List<TRange>> batch, List<TColumn> columns, List<IterInfo> ssiList,
         Map<String,Map<String,String>> ssio, List<ByteBuffer> authorizations, boolean waitForWrites,
-        TSamplerConfiguration tsc, long batchTimeOut, String context) {
+        TSamplerConfiguration tsc, long batchTimeOut, String context,
+        Map<String,String> executionHints) {
       return null;
     }
 
@@ -159,7 +160,7 @@ public InitialScan startScan(TInfo tinfo, TCredentials credentials, TKeyExtent e
         TRange range, List<TColumn> columns, int batchSize, List<IterInfo> ssiList,
         Map<String,Map<String,String>> ssio, List<ByteBuffer> authorizations, boolean waitForWrites,
         boolean isolated, long readaheadThreshold, TSamplerConfiguration tsc, long batchTimeOut,
-        String classLoaderContext) {
+        String classLoaderContext, Map<String,String> executionHints) {
       return null;
     }
 


 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services