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/05/05 23:35:44 UTC

[GitHub] keith-turner closed pull request #436: ACCUMULO-4813 New bulk import process and API

keith-turner closed pull request #436: ACCUMULO-4813 New bulk import process and API
URL: https://github.com/apache/accumulo/pull/436
 
 
   

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/core/pom.xml b/core/pom.xml
index 1643cd66d8..c8e6f84d1e 100644
--- a/core/pom.xml
+++ b/core/pom.xml
@@ -39,6 +39,10 @@
       <artifactId>auto-service</artifactId>
       <optional>true</optional>
     </dependency>
+    <dependency>
+      <groupId>com.google.code.gson</groupId>
+      <artifactId>gson</artifactId>
+    </dependency>
     <dependency>
       <groupId>com.google.guava</groupId>
       <artifactId>guava</artifactId>
diff --git a/core/src/main/java/org/apache/accumulo/core/Constants.java b/core/src/main/java/org/apache/accumulo/core/Constants.java
index ed1680b0bd..32705b1b5d 100644
--- a/core/src/main/java/org/apache/accumulo/core/Constants.java
+++ b/core/src/main/java/org/apache/accumulo/core/Constants.java
@@ -95,6 +95,8 @@
   public static final String ZTABLE_LOCKS = "/table_locks";
 
   public static final String BULK_PREFIX = "b-";
+  public static final String BULK_RENAME_FILE = "renames.json";
+  public static final String BULK_LOAD_MAPPING = "loadmap.json";
 
   public static final String CLONE_PREFIX = "c-";
   public static final byte[] CLONE_PREFIX_BYTES = CLONE_PREFIX.getBytes(UTF_8);
diff --git a/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperations.java b/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperations.java
index d86e6afebe..1cf2d817dd 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperations.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperations.java
@@ -24,6 +24,7 @@
 import java.util.Map.Entry;
 import java.util.Set;
 import java.util.SortedSet;
+import java.util.concurrent.Executor;
 import java.util.function.Predicate;
 
 import org.apache.accumulo.core.client.AccumuloException;
@@ -615,10 +616,77 @@ void setLocalityGroups(String tableName, Map<String,Set<Text>> groups)
    * @throws TableNotFoundException
    *           when the table no longer exists
    *
+   * @deprecated since 2.0.0 use {@link #addFilesTo(String)} instead.
    */
+  @Deprecated
   void importDirectory(String tableName, String dir, String failureDir, boolean setTime)
       throws TableNotFoundException, IOException, AccumuloException, AccumuloSecurityException;
 
+  /**
+   * @since 2.0.0
+   */
+  public static interface ImportSourceOptions {
+    ImportSourceOptions settingLogicalTime();
+
+    void load()
+        throws TableNotFoundException, IOException, AccumuloException, AccumuloSecurityException;
+  }
+
+  /**
+   * @since 2.0.0
+   */
+  public static interface ImportExecutorOptions extends ImportSourceOptions {
+    /**
+     * Files are examined to determine where to load them. This examination is done in the current
+     * process using multiple threads. If this property is not set, then the client property
+     * {@code bulk.threads} is used to create a thread pool.
+     *
+     * @param service
+     *          Use this executor to run file examination task
+     * @return ImportSourceOptions
+     */
+    ImportSourceOptions usingExecutor(Executor service);
+
+    /**
+     * Files are examined to determine where to load them. This examination is done in the current
+     * process using multiple threads. If this property is not set, then the client property
+     * {@code bulk.threads} is used to create a thread pool.
+     *
+     * @param numThreads
+     *          Create a thread pool with this many thread to run file examination task.
+     * @return ImportSourceOptions
+     */
+    ImportSourceOptions usingThreads(int numThreads);
+  }
+
+  /**
+   * @since 2.0.0
+   */
+  public static interface ImportSourceArguments {
+    /**
+     *
+     * @param directory
+     *          Load files from this directory
+     * @return ImportSourceOptions
+     */
+    ImportExecutorOptions from(String directory);
+  }
+
+  /**
+   * Bulk import the files in a directory into a table. Files can be created using
+   * {@link AccumuloFileOutputFormat} and {@link RFile#newWriter()}.
+   *
+   * <p>
+   * This new method of bulk import examines files in the current process outside of holding a table
+   * lock. The old bulk import method ({@link #importDirectory(String, String, String, boolean)})
+   * examines files on the server side while holding a table read lock.
+   *
+   * @since 2.0.0
+   */
+  default ImportSourceArguments addFilesTo(String tableName) {
+    throw new UnsupportedOperationException();
+  }
+
   /**
    * Initiates taking a table offline, but does not wait for action to complete
    *
diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/Bulk.java b/core/src/main/java/org/apache/accumulo/core/client/impl/Bulk.java
new file mode 100644
index 0000000000..935834f1f7
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/Bulk.java
@@ -0,0 +1,228 @@
+/*
+ * 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.client.impl;
+
+import java.io.Serializable;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.Objects;
+
+import org.apache.accumulo.core.data.impl.KeyExtent;
+import org.apache.accumulo.core.util.TextUtil;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.Text;
+
+import com.google.common.base.Preconditions;
+
+public class Bulk {
+
+  /**
+   * WARNING : do not change this class, its used for serialization to Json
+   */
+  public static class Mapping {
+    private Tablet tablet;
+    private Collection<FileInfo> files;
+
+    public Mapping(KeyExtent tablet, Files files) {
+      this.tablet = toTablet(tablet);
+      this.files = files.files.values();
+    }
+
+    public Tablet getTablet() {
+      return tablet;
+    }
+
+    public KeyExtent getKeyExtent(Table.ID tableId) {
+      return tablet.toKeyExtent(tableId);
+    }
+
+    public Files getFiles() {
+      return new Files(files);
+    }
+  }
+
+  /**
+   * WARNING : do not change this class, its used for serialization to Json
+   */
+  public static class Tablet {
+
+    private byte[] endRow;
+    private byte[] prevEndRow;
+
+    public Tablet(Text endRow, Text prevEndRow) {
+      this.endRow = endRow == null ? null : TextUtil.getBytes(endRow);
+      this.prevEndRow = prevEndRow == null ? null : TextUtil.getBytes(prevEndRow);
+    }
+
+    public KeyExtent toKeyExtent(Table.ID tableId) {
+      return Bulk.toKeyExtent(tableId, this);
+    }
+
+    public Text getEndRow() {
+      if (endRow == null) {
+        return null;
+      }
+      return new Text(endRow);
+    }
+
+    public Text getPrevEndRow() {
+      if (prevEndRow == null) {
+        return null;
+      }
+      return new Text(prevEndRow);
+    }
+
+    @Override
+    public String toString() {
+      return getEndRow().toString() + ";" + getPrevEndRow().toString();
+    }
+  }
+
+  /**
+   * WARNING : do not change this class, its used for serialization to Json
+   */
+  public static class FileInfo implements Serializable {
+    final String name;
+    final long estSize;
+    final long estEntries;
+
+    public FileInfo(String fileName, long estFileSize, long estNumEntries) {
+      this.name = fileName;
+      this.estSize = estFileSize;
+      this.estEntries = estNumEntries;
+    }
+
+    public FileInfo(Path path, long estSize) {
+      this(path.getName(), estSize, 0);
+    }
+
+    static FileInfo merge(FileInfo fi1, FileInfo fi2) {
+      Preconditions.checkArgument(fi1.name.equals(fi2.name));
+      return new FileInfo(fi1.name, fi1.estSize + fi2.estSize, fi1.estEntries + fi2.estEntries);
+    }
+
+    public String getFileName() {
+      return name;
+    }
+
+    public long getEstFileSize() {
+      return estSize;
+    }
+
+    public long getEstNumEntries() {
+      return estEntries;
+    }
+
+    @Override
+    public String toString() {
+      return String.format("file:%s  estSize:%d estEntries:%s", name, estSize, estEntries);
+    }
+
+    @Override
+    public boolean equals(Object o) {
+      if (o == this)
+        return true;
+      if (!(o instanceof FileInfo))
+        return false;
+      FileInfo other = (FileInfo) o;
+      return this.name.equals(other.name) && this.estSize == other.estSize
+          && this.estEntries == other.estEntries;
+    }
+
+    @Override
+    public int hashCode() {
+      return Objects.hash(name, estSize, estEntries);
+    }
+  }
+
+  public static class Files implements Iterable<FileInfo>, Serializable {
+    Map<String,FileInfo> files = new HashMap<>();
+
+    public Files(Collection<FileInfo> files) {
+      files.forEach(fi -> add(fi));
+    }
+
+    public Files() {}
+
+    public void add(FileInfo fi) {
+      if (files.putIfAbsent(fi.name, fi) != null) {
+        throw new IllegalArgumentException("File already present " + fi.name);
+      }
+    }
+
+    public FileInfo get(String fileName) {
+      return files.get(fileName);
+    }
+
+    public Files mapNames(Map<String,String> renames) {
+      Files renamed = new Files();
+
+      files.forEach((k, v) -> {
+        String newName = renames.get(k);
+        FileInfo nfi = new FileInfo(newName, v.estSize, v.estEntries);
+        renamed.files.put(newName, nfi);
+      });
+
+      return renamed;
+    }
+
+    void merge(Files other) {
+      other.files.forEach((k, v) -> {
+        files.merge(k, v, FileInfo::merge);
+      });
+    }
+
+    public int getSize() {
+      return this.files.size();
+    }
+
+    @Override
+    public Iterator<FileInfo> iterator() {
+      return files.values().iterator();
+    }
+
+    @Override
+    public boolean equals(Object o) {
+      if (o == this)
+        return true;
+      if (!(o instanceof Files))
+        return false;
+      Files other = (Files) o;
+      return this.files.equals(other.files);
+    }
+
+    @Override
+    public int hashCode() {
+      return files.hashCode();
+    }
+
+    @Override
+    public String toString() {
+      return files.toString();
+    }
+  }
+
+  public static Tablet toTablet(KeyExtent keyExtent) {
+    return new Tablet(keyExtent.getEndRow(), keyExtent.getPrevEndRow());
+  }
+
+  public static KeyExtent toKeyExtent(Table.ID tableId, Tablet tablet) {
+    return new KeyExtent(tableId, tablet.getEndRow(), tablet.getPrevEndRow());
+  }
+}
diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/BulkImport.java b/core/src/main/java/org/apache/accumulo/core/client/impl/BulkImport.java
new file mode 100644
index 0000000000..0c19cb0c7e
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/BulkImport.java
@@ -0,0 +1,382 @@
+/*
+ * 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.client.impl;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Objects;
+import java.util.Set;
+import java.util.SortedMap;
+import java.util.TreeMap;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CompletionException;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Executor;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+
+import org.apache.accumulo.core.Constants;
+import org.apache.accumulo.core.client.AccumuloException;
+import org.apache.accumulo.core.client.AccumuloSecurityException;
+import org.apache.accumulo.core.client.NamespaceExistsException;
+import org.apache.accumulo.core.client.NamespaceNotFoundException;
+import org.apache.accumulo.core.client.TableExistsException;
+import org.apache.accumulo.core.client.TableNotFoundException;
+import org.apache.accumulo.core.client.admin.TableOperations.ImportExecutorOptions;
+import org.apache.accumulo.core.client.admin.TableOperations.ImportSourceArguments;
+import org.apache.accumulo.core.client.admin.TableOperations.ImportSourceOptions;
+import org.apache.accumulo.core.client.impl.TabletLocator.TabletLocation;
+import org.apache.accumulo.core.conf.AccumuloConfiguration;
+import org.apache.accumulo.core.conf.ClientProperty;
+import org.apache.accumulo.core.conf.ConfigurationCopy;
+import org.apache.accumulo.core.conf.ConfigurationTypeHelper;
+import org.apache.accumulo.core.data.ByteSequence;
+import org.apache.accumulo.core.data.Key;
+import org.apache.accumulo.core.data.Range;
+import org.apache.accumulo.core.data.impl.KeyExtent;
+import org.apache.accumulo.core.file.FileOperations;
+import org.apache.accumulo.core.file.FileSKVIterator;
+import org.apache.accumulo.core.master.thrift.FateOperation;
+import org.apache.accumulo.core.util.CachedConfiguration;
+import org.apache.accumulo.core.volume.VolumeConfiguration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.Text;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Collections2;
+
+public class BulkImport implements ImportSourceArguments, ImportExecutorOptions {
+
+  private static final Logger log = LoggerFactory.getLogger(BulkImport.class);
+
+  private boolean setTime = false;
+  private Executor executor = null;
+  private String dir;
+  private int numThreads = -1;
+
+  private final ClientContext context;
+  private final String tableName;
+
+  BulkImport(String tableName, ClientContext context) {
+    this.context = context;
+    this.tableName = Objects.requireNonNull(tableName);
+  }
+
+  @Override
+  public ImportSourceOptions settingLogicalTime() {
+    this.setTime = true;
+    return this;
+  }
+
+  @Override
+  public void load()
+      throws TableNotFoundException, IOException, AccumuloException, AccumuloSecurityException {
+
+    Table.ID tableId = Tables.getTableId(context.getInstance(), tableName);
+
+    Map<String,String> props = context.getConnector().instanceOperations().getSystemConfiguration();
+    AccumuloConfiguration conf = new ConfigurationCopy(props);
+
+    FileSystem fs = VolumeConfiguration.getVolume(dir, CachedConfiguration.getInstance(), conf)
+        .getFileSystem();
+
+    Path srcPath = checkPath(fs, dir);
+
+    Executor executor;
+    ExecutorService service = null;
+
+    if (this.executor != null) {
+      executor = this.executor;
+    } else if (numThreads > 0) {
+      executor = service = Executors.newFixedThreadPool(numThreads);
+    } else {
+      String threads = context.getConfiguration().get(ClientProperty.BULK_LOAD_THREADS.getKey());
+      executor = service = Executors
+          .newFixedThreadPool(ConfigurationTypeHelper.getNumThreads(threads));
+    }
+
+    try {
+      SortedMap<KeyExtent,Bulk.Files> mappings = computeFileToTabletMappings(fs, tableId, srcPath,
+          executor, context);
+
+      BulkSerialize.writeLoadMapping(mappings, srcPath.toString(), p -> fs.create(p));
+
+      List<ByteBuffer> args = Arrays.asList(ByteBuffer.wrap(tableId.getUtf8()),
+          ByteBuffer.wrap(srcPath.toString().getBytes(UTF_8)),
+          ByteBuffer.wrap((setTime + "").getBytes(UTF_8)));
+      doFateOperation(FateOperation.TABLE_BULK_IMPORT2, args, Collections.emptyMap(), tableName);
+    } finally {
+      if (service != null) {
+        service.shutdown();
+      }
+    }
+  }
+
+  /**
+   * Check path of bulk directory and permissions
+   */
+  private Path checkPath(FileSystem fs, String dir) throws IOException, AccumuloException {
+    Path ret;
+
+    if (dir.contains(":")) {
+      ret = new Path(dir);
+    } else {
+      ret = fs.makeQualified(new Path(dir));
+    }
+
+    try {
+      if (!fs.getFileStatus(ret).isDirectory()) {
+        throw new AccumuloException("Bulk import directory " + dir + " is not a directory!");
+      }
+      Path tmpFile = new Path(ret, "isWritable");
+      if (fs.createNewFile(tmpFile))
+        fs.delete(tmpFile, true);
+      else
+        throw new AccumuloException("Bulk import directory " + dir + " is not writable.");
+    } catch (FileNotFoundException fnf) {
+      throw new AccumuloException(
+          "Bulk import directory " + dir + " does not exist or has bad permissions", fnf);
+    }
+    return ret;
+  }
+
+  @Override
+  public ImportSourceOptions usingExecutor(Executor service) {
+    this.executor = Objects.requireNonNull(service);
+    return this;
+  }
+
+  @Override
+  public ImportSourceOptions usingThreads(int numThreads) {
+    Preconditions.checkArgument(numThreads > 0, "Non positive number of threads given : %s",
+        numThreads);
+    this.numThreads = numThreads;
+    return this;
+  }
+
+  @Override
+  public ImportExecutorOptions from(String directory) {
+    this.dir = Objects.requireNonNull(directory);
+    return this;
+  }
+
+  private final static byte[] byte0 = {0};
+
+  private static class MLong {
+    public MLong(long i) {
+      l = i;
+    }
+
+    long l;
+  }
+
+  public static Map<KeyExtent,Long> estimateSizes(AccumuloConfiguration acuConf, Path mapFile,
+      long fileSize, Collection<KeyExtent> extents, FileSystem ns) throws IOException {
+
+    long totalIndexEntries = 0;
+    Map<KeyExtent,MLong> counts = new TreeMap<>();
+    for (KeyExtent keyExtent : extents)
+      counts.put(keyExtent, new MLong(0));
+
+    Text row = new Text();
+
+    FileSKVIterator index = FileOperations.getInstance().newIndexReaderBuilder()
+        .forFile(mapFile.toString(), ns, ns.getConf()).withTableConfiguration(acuConf).build();
+
+    try {
+      while (index.hasTop()) {
+        Key key = index.getTopKey();
+        totalIndexEntries++;
+        key.getRow(row);
+
+        // TODO this could use a binary search
+        for (Entry<KeyExtent,MLong> entry : counts.entrySet())
+          if (entry.getKey().contains(row))
+            entry.getValue().l++;
+
+        index.next();
+      }
+    } finally {
+      try {
+        if (index != null)
+          index.close();
+      } catch (IOException e) {
+        log.debug("Failed to close " + mapFile, e);
+      }
+    }
+
+    Map<KeyExtent,Long> results = new TreeMap<>();
+    for (KeyExtent keyExtent : extents) {
+      double numEntries = counts.get(keyExtent).l;
+      if (numEntries == 0)
+        numEntries = 1;
+      long estSize = (long) ((numEntries / totalIndexEntries) * fileSize);
+      results.put(keyExtent, estSize);
+    }
+    return results;
+  }
+
+  public static List<TabletLocation> findOverlappingTablets(ClientContext context,
+      TabletLocator locator, Text startRow, Text endRow, FileSKVIterator reader)
+      throws IOException, AccumuloException, AccumuloSecurityException, TableNotFoundException {
+    List<TabletLocation> result = new ArrayList<>();
+    Collection<ByteSequence> columnFamilies = Collections.emptyList();
+    Text row = startRow;
+    if (row == null)
+      row = new Text();
+    while (true) {
+      // log.debug(filename + " Seeking to row " + row);
+      reader.seek(new Range(row, null), columnFamilies, false);
+      if (!reader.hasTop()) {
+        // log.debug(filename + " not found");
+        break;
+      }
+      row = reader.getTopKey().getRow();
+      TabletLocation tabletLocation = locator.locateTablet(context, row, false, true);
+      // log.debug(filename + " found row " + row + " at location " + tabletLocation);
+      result.add(tabletLocation);
+      row = tabletLocation.tablet_extent.getEndRow();
+      if (row != null && (endRow == null || row.compareTo(endRow) < 0)) {
+        row = new Text(row);
+        row.append(byte0, 0, byte0.length);
+      } else
+        break;
+    }
+
+    return result;
+  }
+
+  public static List<TabletLocation> findOverlappingTablets(ClientContext context,
+      TabletLocator locator, Path file, FileSystem fs)
+      throws IOException, AccumuloException, AccumuloSecurityException, TableNotFoundException {
+    try (FileSKVIterator reader = FileOperations.getInstance().newReaderBuilder()
+        .forFile(file.toString(), fs, fs.getConf())
+        .withTableConfiguration(context.getConfiguration()).seekToBeginning().build()) {
+      return findOverlappingTablets(context, locator, null, null, reader);
+    }
+  }
+
+  public static SortedMap<KeyExtent,Bulk.Files> computeFileToTabletMappings(FileSystem fs,
+      Table.ID tableId, Path dirPath, Executor executor, ClientContext context) throws IOException {
+    TabletLocator locator = TabletLocator.getLocator(context, tableId);
+
+    FileStatus[] files = fs.listStatus(dirPath,
+        p -> !p.getName().equals(Constants.BULK_LOAD_MAPPING));
+
+    List<CompletableFuture<Map<KeyExtent,Bulk.FileInfo>>> futures = new ArrayList<>();
+
+    for (FileStatus fileStatus : files) {
+      CompletableFuture<Map<KeyExtent,Bulk.FileInfo>> future = CompletableFuture.supplyAsync(() -> {
+        try {
+          long t1 = System.currentTimeMillis();
+          List<TabletLocation> locations = findOverlappingTablets(context, locator,
+              fileStatus.getPath(), fs);
+          Collection<KeyExtent> extents = Collections2.transform(locations, l -> l.tablet_extent);
+          Map<KeyExtent,Long> estSizes = estimateSizes(context.getConfiguration(),
+              fileStatus.getPath(), fileStatus.getLen(), extents, fs);
+          Map<KeyExtent,Bulk.FileInfo> pathLocations = new HashMap<>();
+          for (TabletLocation location : locations) {
+            KeyExtent ke = location.tablet_extent;
+            pathLocations.put(ke,
+                new Bulk.FileInfo(fileStatus.getPath(), estSizes.getOrDefault(ke, 0L)));
+          }
+          long t2 = System.currentTimeMillis();
+          log.trace("Mapped {} to {} tablets in {}ms", fileStatus.getPath(), pathLocations.size(),
+              t2 - t1);
+          return pathLocations;
+        } catch (Exception e) {
+          throw new CompletionException(e);
+        }
+      }, executor);
+
+      futures.add(future);
+    }
+
+    SortedMap<KeyExtent,Bulk.Files> mappings = new TreeMap<>();
+
+    for (CompletableFuture<Map<KeyExtent,Bulk.FileInfo>> future : futures) {
+      try {
+        Map<KeyExtent,Bulk.FileInfo> pathMapping = future.get();
+        pathMapping.forEach((extent, path) -> {
+          mappings.computeIfAbsent(extent, k -> new Bulk.Files()).add(path);
+        });
+      } catch (InterruptedException e) {
+        Thread.currentThread().interrupt();
+        throw new RuntimeException(e);
+      } catch (ExecutionException e) {
+        throw new RuntimeException(e);
+      }
+    }
+
+    return mergeOverlapping(mappings);
+  }
+
+  // This method handles the case of splits happening while files are being examined. It merges
+  // smaller tablets into large tablets.
+  static SortedMap<KeyExtent,Bulk.Files> mergeOverlapping(
+      SortedMap<KeyExtent,Bulk.Files> mappings) {
+    List<KeyExtent> extents = new ArrayList<>(mappings.keySet());
+
+    for (KeyExtent ke : extents) {
+      Set<KeyExtent> overlapping = KeyExtent.findOverlapping(ke, mappings);
+      for (KeyExtent oke : overlapping) {
+        if (ke.equals(oke)) {
+          continue;
+        }
+
+        boolean containsPrevRow = ke.getPrevEndRow() == null || (oke.getPrevEndRow() != null
+            && ke.getPrevEndRow().compareTo(oke.getPrevEndRow()) <= 0);
+        boolean containsEndRow = ke.getEndRow() == null
+            || (oke.getEndRow() != null && ke.getEndRow().compareTo(oke.getEndRow()) >= 0);
+
+        if (containsPrevRow && containsEndRow) {
+          mappings.get(ke).merge(mappings.remove(oke));
+        } else {
+          throw new RuntimeException("TODO handle merges");
+        }
+      }
+    }
+
+    return mappings;
+  }
+
+  private String doFateOperation(FateOperation op, List<ByteBuffer> args, Map<String,String> opts,
+      String tableName) throws AccumuloSecurityException, AccumuloException {
+    try {
+      return new TableOperationsImpl(context).doFateOperation(op, args, opts, tableName);
+    } catch (TableExistsException | TableNotFoundException | NamespaceNotFoundException
+        | NamespaceExistsException e) {
+      // should not happen
+      throw new AssertionError(e);
+    }
+  }
+}
diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/BulkSerialize.java b/core/src/main/java/org/apache/accumulo/core/client/impl/BulkSerialize.java
new file mode 100644
index 0000000000..698b17f0c5
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/BulkSerialize.java
@@ -0,0 +1,206 @@
+/*
+ * 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.client.impl;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+
+import java.io.BufferedReader;
+import java.io.BufferedWriter;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.io.OutputStream;
+import java.io.OutputStreamWriter;
+import java.io.UncheckedIOException;
+import java.lang.reflect.Type;
+import java.util.AbstractMap;
+import java.util.Base64;
+import java.util.Base64.Decoder;
+import java.util.Base64.Encoder;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Set;
+import java.util.SortedMap;
+
+import org.apache.accumulo.core.Constants;
+import org.apache.accumulo.core.client.impl.Bulk.Files;
+import org.apache.accumulo.core.client.impl.Bulk.Mapping;
+import org.apache.accumulo.core.client.impl.Table.ID;
+import org.apache.accumulo.core.data.impl.KeyExtent;
+import org.apache.hadoop.fs.Path;
+
+import com.google.gson.Gson;
+import com.google.gson.GsonBuilder;
+import com.google.gson.JsonDeserializationContext;
+import com.google.gson.JsonDeserializer;
+import com.google.gson.JsonElement;
+import com.google.gson.JsonParseException;
+import com.google.gson.JsonPrimitive;
+import com.google.gson.JsonSerializationContext;
+import com.google.gson.JsonSerializer;
+import com.google.gson.reflect.TypeToken;
+import com.google.gson.stream.JsonReader;
+import com.google.gson.stream.JsonWriter;
+
+/**
+ * Place for all bulk import serialization code. For the objects being serialized see {@link Bulk}
+ */
+public class BulkSerialize {
+
+  private static class ByteArrayToBase64TypeAdapter
+      implements JsonSerializer<byte[]>, JsonDeserializer<byte[]> {
+
+    Decoder decoder = Base64.getUrlDecoder();
+    Encoder encoder = Base64.getUrlEncoder();
+
+    @Override
+    public byte[] deserialize(JsonElement json, Type typeOfT, JsonDeserializationContext context)
+        throws JsonParseException {
+      return decoder.decode(json.getAsString());
+    }
+
+    @Override
+    public JsonElement serialize(byte[] src, Type typeOfSrc, JsonSerializationContext context) {
+      return new JsonPrimitive(encoder.encodeToString(src));
+    }
+  }
+
+  private static Gson createGson() {
+    return new GsonBuilder()
+        .registerTypeHierarchyAdapter(byte[].class, new ByteArrayToBase64TypeAdapter()).create();
+  }
+
+  public interface Output {
+    OutputStream create(Path path) throws IOException;
+  }
+
+  public interface Input {
+    InputStream open(Path path) throws IOException;
+  }
+
+  /**
+   * Serialize bulk load mapping to {@link Constants.BULK_LOAD_MAPPING}
+   */
+  public static void writeLoadMapping(SortedMap<KeyExtent,Bulk.Files> loadMapping, String sourceDir,
+      Output output) throws IOException {
+    final Path lmFile = new Path(sourceDir, Constants.BULK_LOAD_MAPPING);
+
+    try (OutputStream fsOut = output.create(lmFile);
+        JsonWriter writer = new JsonWriter(
+            new BufferedWriter(new OutputStreamWriter(fsOut, UTF_8)))) {
+      Gson gson = createGson();
+      writer.setIndent("  ");
+      writer.beginArray();
+      Set<Entry<KeyExtent,Files>> es = loadMapping.entrySet();
+      for (Entry<KeyExtent,Files> entry : es) {
+        Mapping mapping = new Bulk.Mapping(entry.getKey(), entry.getValue());
+        gson.toJson(mapping, Mapping.class, writer);
+      }
+      writer.endArray();
+    }
+  }
+
+  public static class LoadMappingIterator
+      implements Iterator<Entry<KeyExtent,Bulk.Files>>, AutoCloseable {
+    private ID tableId;
+    private JsonReader reader;
+    private Gson gson = createGson();
+    private Map<String,String> renameMap;
+
+    private LoadMappingIterator(Table.ID tableId, JsonReader reader) {
+      this.tableId = tableId;
+      this.reader = reader;
+    }
+
+    private void setRenameMap(Map<String,String> renameMap) {
+      this.renameMap = renameMap;
+    }
+
+    @Override
+    public void close() throws Exception {
+      reader.close();
+    }
+
+    @Override
+    public boolean hasNext() {
+      try {
+        return reader.hasNext();
+      } catch (IOException e) {
+        throw new UncheckedIOException(e);
+      }
+    }
+
+    @Override
+    public Entry<KeyExtent,Files> next() {
+      Bulk.Mapping bm = gson.fromJson(reader, Bulk.Mapping.class);
+      if (renameMap != null) {
+        return new AbstractMap.SimpleEntry<>(bm.getKeyExtent(tableId),
+            bm.getFiles().mapNames(renameMap));
+      } else {
+        return new AbstractMap.SimpleEntry<>(bm.getKeyExtent(tableId), bm.getFiles());
+      }
+    }
+
+  }
+
+  /**
+   * Read Json array of Bulk.Mapping into LoadMappingIterator
+   */
+  public static LoadMappingIterator readLoadMapping(String bulkDir, Table.ID tableId, Input input)
+      throws IOException {
+    final Path lmFile = new Path(bulkDir, Constants.BULK_LOAD_MAPPING);
+    JsonReader reader = new JsonReader(
+        new BufferedReader(new InputStreamReader(input.open(lmFile), UTF_8)));
+    reader.beginArray();
+    return new LoadMappingIterator(tableId, reader);
+  }
+
+  public static void writeRenameMap(Map<String,String> oldToNewNameMap, String bulkDir,
+      Output output) throws IOException {
+    final Path renamingFile = new Path(bulkDir, Constants.BULK_RENAME_FILE);
+    try (OutputStream fsOut = output.create(renamingFile);
+        BufferedWriter writer = new BufferedWriter(new OutputStreamWriter(fsOut))) {
+      Gson gson = new GsonBuilder().create();
+      gson.toJson(oldToNewNameMap, writer);
+    }
+  }
+
+  public static Map<String,String> readRenameMap(String bulkDir, Input input) throws IOException {
+    final Path renamingFile = new Path(bulkDir, Constants.BULK_RENAME_FILE);
+    Map<String,String> oldToNewNameMap;
+    Gson gson = createGson();
+    try (InputStream fis = input.open(renamingFile);
+        BufferedReader reader = new BufferedReader(new InputStreamReader(fis))) {
+      oldToNewNameMap = gson.fromJson(reader, new TypeToken<Map<String,String>>() {}.getType());
+    }
+    return oldToNewNameMap;
+  }
+
+  /**
+   * Read in both maps and change all the file names in the mapping to the new names. This is needed
+   * because the load mapping file was written with the original file names before they were moved
+   * by BulkImportMove
+   */
+  public static LoadMappingIterator getUpdatedLoadMapping(String bulkDir, Table.ID tableId,
+      Input input) throws IOException {
+    Map<String,String> renames = readRenameMap(bulkDir, input);
+    LoadMappingIterator lmi = readLoadMapping(bulkDir, tableId, input);
+    lmi.setRenameMap(renames);
+    return lmi;
+  }
+}
diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/TableOperationsImpl.java b/core/src/main/java/org/apache/accumulo/core/client/impl/TableOperationsImpl.java
index 30ebcdcfb0..1f3fb2b387 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/impl/TableOperationsImpl.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/TableOperationsImpl.java
@@ -1947,4 +1947,9 @@ public void removeSummarizers(String tableName, Predicate<SummarizerConfiguratio
       throws AccumuloException, TableNotFoundException, AccumuloSecurityException {
     return new ArrayList<>(SummarizerConfiguration.fromTableProperties(getProperties(tableName)));
   }
+
+  @Override
+  public ImportSourceArguments addFilesTo(String tableName) {
+    return new BulkImport(tableName, context);
+  }
 }
diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/thrift/TableOperationExceptionType.java b/core/src/main/java/org/apache/accumulo/core/client/impl/thrift/TableOperationExceptionType.java
index 5f683278fb..998f91cb3d 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/impl/thrift/TableOperationExceptionType.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/thrift/TableOperationExceptionType.java
@@ -34,7 +34,8 @@
   OTHER(6),
   NAMESPACE_EXISTS(7),
   NAMESPACE_NOTFOUND(8),
-  INVALID_NAME(9);
+  INVALID_NAME(9),
+  BULK_BAD_LOAD_MAPPING(10);
 
   private final int value;
 
@@ -75,6 +76,8 @@ public static TableOperationExceptionType findByValue(int value) {
         return NAMESPACE_NOTFOUND;
       case 9:
         return INVALID_NAME;
+      case 10:
+        return BULK_BAD_LOAD_MAPPING;
       default:
         return null;
     }
diff --git a/core/src/main/java/org/apache/accumulo/core/conf/ClientProperty.java b/core/src/main/java/org/apache/accumulo/core/conf/ClientProperty.java
index e481f6a486..09507369a0 100644
--- a/core/src/main/java/org/apache/accumulo/core/conf/ClientProperty.java
+++ b/core/src/main/java/org/apache/accumulo/core/conf/ClientProperty.java
@@ -60,6 +60,12 @@
       "Change the" + " durability for the BatchWriter session. To use the table's durability"
           + " setting. use \"default\" which is the table's durability setting."),
 
+  // Bulk load
+  BULK_LOAD_THREADS("bulk.threads", "8C",
+      "The number of threads used to inspect bulk load files to determine where files go.  "
+          + "If the value ends with C, then it will be multiplied by the number of cores on the "
+          + "system. This property is only used by the bulk import API introduced in 2.0.0."),
+
   // SSL
   SSL_ENABLED("ssl.enabled", "false", "Enable SSL for client RPC"),
   SSL_KEYSTORE_PASSWORD("ssl.keystore.password", "", "Password used to encrypt keystore"),
diff --git a/core/src/main/java/org/apache/accumulo/core/conf/ConfigurationTypeHelper.java b/core/src/main/java/org/apache/accumulo/core/conf/ConfigurationTypeHelper.java
index d718c9c732..def6e5f78c 100644
--- a/core/src/main/java/org/apache/accumulo/core/conf/ConfigurationTypeHelper.java
+++ b/core/src/main/java/org/apache/accumulo/core/conf/ConfigurationTypeHelper.java
@@ -213,4 +213,22 @@ public static double getFraction(String str) {
 
     return instance;
   }
+
+  /**
+   * Get the number of threads from string property. If the value ends with C, then it will be
+   * multiplied by the number of cores.
+   */
+  public static int getNumThreads(String threads) {
+    if (threads == null) {
+      threads = ClientProperty.BULK_LOAD_THREADS.getDefaultValue();
+    }
+    int nThreads;
+    if (threads.toUpperCase().endsWith("C")) {
+      nThreads = Runtime.getRuntime().availableProcessors()
+          * Integer.parseInt(threads.substring(0, threads.length() - 1));
+    } else {
+      nThreads = Integer.parseInt(threads);
+    }
+    return nThreads;
+  }
 }
diff --git a/core/src/main/java/org/apache/accumulo/core/master/thrift/FateOperation.java b/core/src/main/java/org/apache/accumulo/core/master/thrift/FateOperation.java
index 0aa24d7b61..45ce87c602 100644
--- a/core/src/main/java/org/apache/accumulo/core/master/thrift/FateOperation.java
+++ b/core/src/main/java/org/apache/accumulo/core/master/thrift/FateOperation.java
@@ -40,7 +40,8 @@
   TABLE_CANCEL_COMPACT(12),
   NAMESPACE_CREATE(13),
   NAMESPACE_DELETE(14),
-  NAMESPACE_RENAME(15);
+  NAMESPACE_RENAME(15),
+  TABLE_BULK_IMPORT2(16);
 
   private final int value;
 
@@ -93,6 +94,8 @@ public static FateOperation findByValue(int value) {
         return NAMESPACE_DELETE;
       case 15:
         return NAMESPACE_RENAME;
+      case 16:
+        return TABLE_BULK_IMPORT2;
       default:
         return null;
     }
diff --git a/core/src/main/java/org/apache/accumulo/core/metadata/schema/MetadataScanner.java b/core/src/main/java/org/apache/accumulo/core/metadata/schema/MetadataScanner.java
index b3fbe35813..0516051794 100644
--- a/core/src/main/java/org/apache/accumulo/core/metadata/schema/MetadataScanner.java
+++ b/core/src/main/java/org/apache/accumulo/core/metadata/schema/MetadataScanner.java
@@ -67,6 +67,8 @@
   public static interface ColumnOptions {
     public ColumnOptions fetchFiles();
 
+    public ColumnOptions fetchLoaded();
+
     public ColumnOptions fetchLocation();
 
     public ColumnOptions fetchPrev();
@@ -125,6 +127,13 @@ public ColumnOptions fetchFiles() {
       return this;
     }
 
+    @Override
+    public ColumnOptions fetchLoaded() {
+      fetchedCols.add(FetchedColumns.LOADED);
+      families.add(MetadataSchema.TabletsSection.BulkFileColumnFamily.NAME);
+      return this;
+    }
+
     @Override
     public ColumnOptions fetchLocation() {
       fetchedCols.add(FetchedColumns.LOCATION);
diff --git a/core/src/main/java/org/apache/accumulo/core/metadata/schema/TabletMetadata.java b/core/src/main/java/org/apache/accumulo/core/metadata/schema/TabletMetadata.java
index 92f4e4e7c5..b1019710c6 100644
--- a/core/src/main/java/org/apache/accumulo/core/metadata/schema/TabletMetadata.java
+++ b/core/src/main/java/org/apache/accumulo/core/metadata/schema/TabletMetadata.java
@@ -24,6 +24,7 @@
 import java.util.List;
 import java.util.Map.Entry;
 import java.util.Objects;
+import java.util.Set;
 
 import org.apache.accumulo.core.client.RowIterator;
 import org.apache.accumulo.core.client.Scanner;
@@ -42,6 +43,7 @@
 import com.google.common.base.Preconditions;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableList.Builder;
+import com.google.common.collect.ImmutableSet;
 import com.google.common.collect.Iterators;
 
 public class TabletMetadata {
@@ -51,6 +53,7 @@
   private Text endRow;
   private Location location;
   private List<String> files;
+  private Set<String> loadedFiles;
   private EnumSet<FetchedColumns> fetchedColumns;
   private KeyExtent extent;
   private Location last;
@@ -60,7 +63,7 @@
   }
 
   public static enum FetchedColumns {
-    LOCATION, PREV_ROW, FILES, LAST
+    LOCATION, PREV_ROW, FILES, LAST, LOADED
   }
 
   public static class Location {
@@ -114,6 +117,12 @@ public Location getLocation() {
     return location;
   }
 
+  public Set<String> getLoaded() {
+    Preconditions.checkState(fetchedColumns.contains(FetchedColumns.LOADED),
+        "Requested loaded when it was not fetched");
+    return loadedFiles;
+  }
+
   public Location getLast() {
     Preconditions.checkState(fetchedColumns.contains(FetchedColumns.LAST),
         "Requested last when it was not fetched");
@@ -133,6 +142,7 @@ public static TabletMetadata convertRow(Iterator<Entry<Key,Value>> rowIter,
     TabletMetadata te = new TabletMetadata();
 
     Builder<String> filesBuilder = ImmutableList.builder();
+    final ImmutableSet.Builder<String> loadedFilesBuilder = ImmutableSet.builder();
     ByteSequence row = null;
 
     while (rowIter.hasNext()) {
@@ -154,9 +164,10 @@ public static TabletMetadata convertRow(Iterator<Entry<Key,Value>> rowIter,
       if (PREV_ROW_COLUMN.hasColumns(k)) {
         te.prevEndRow = KeyExtent.decodePrevEndRow(v);
       }
-
       if (fam.equals(DataFileColumnFamily.NAME)) {
         filesBuilder.add(k.getColumnQualifier().toString());
+      } else if (fam.equals(MetadataSchema.TabletsSection.BulkFileColumnFamily.NAME)) {
+        loadedFilesBuilder.add(k.getColumnQualifier().toString());
       } else if (fam.equals(CurrentLocationColumnFamily.NAME)) {
         if (te.location != null) {
           throw new IllegalArgumentException(
@@ -178,6 +189,7 @@ public static TabletMetadata convertRow(Iterator<Entry<Key,Value>> rowIter,
     }
 
     te.files = filesBuilder.build();
+    te.loadedFiles = loadedFilesBuilder.build();
     te.fetchedColumns = fetchedColumns;
     return te;
   }
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 c45d91f102..5b71a5ebd7 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
@@ -58,6 +58,8 @@
 
     public java.util.List<org.apache.accumulo.core.data.thrift.TKeyExtent> bulkImport(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, long tid, java.util.Map<org.apache.accumulo.core.data.thrift.TKeyExtent,java.util.Map<java.lang.String,org.apache.accumulo.core.data.thrift.MapFileInfo>> files, boolean setTime) throws org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException, org.apache.thrift.TException;
 
+    public void loadFiles(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, long tid, org.apache.accumulo.core.data.thrift.TKeyExtent keyExtent, java.lang.String dir, java.util.Map<java.lang.String,org.apache.accumulo.core.data.thrift.MapFileInfo> files, boolean setTime) throws org.apache.thrift.TException;
+
     public void splitTablet(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, org.apache.accumulo.core.data.thrift.TKeyExtent extent, java.nio.ByteBuffer splitPoint) throws org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException, NotServingTabletException, org.apache.thrift.TException;
 
     public void loadTablet(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, java.lang.String lock, org.apache.accumulo.core.data.thrift.TKeyExtent extent) throws org.apache.thrift.TException;
@@ -132,6 +134,8 @@
 
     public void bulkImport(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, long tid, java.util.Map<org.apache.accumulo.core.data.thrift.TKeyExtent,java.util.Map<java.lang.String,org.apache.accumulo.core.data.thrift.MapFileInfo>> files, boolean setTime, org.apache.thrift.async.AsyncMethodCallback<java.util.List<org.apache.accumulo.core.data.thrift.TKeyExtent>> resultHandler) throws org.apache.thrift.TException;
 
+    public void loadFiles(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, long tid, org.apache.accumulo.core.data.thrift.TKeyExtent keyExtent, java.lang.String dir, java.util.Map<java.lang.String,org.apache.accumulo.core.data.thrift.MapFileInfo> files, boolean setTime, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
+
     public void splitTablet(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, org.apache.accumulo.core.data.thrift.TKeyExtent extent, java.nio.ByteBuffer splitPoint, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
 
     public void loadTablet(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, java.lang.String lock, org.apache.accumulo.core.data.thrift.TKeyExtent extent, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
@@ -612,6 +616,24 @@ public void send_bulkImport(org.apache.accumulo.core.trace.thrift.TInfo tinfo, o
       throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "bulkImport failed: unknown result");
     }
 
+    public void loadFiles(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, long tid, org.apache.accumulo.core.data.thrift.TKeyExtent keyExtent, java.lang.String dir, java.util.Map<java.lang.String,org.apache.accumulo.core.data.thrift.MapFileInfo> files, boolean setTime) throws org.apache.thrift.TException
+    {
+      send_loadFiles(tinfo, credentials, tid, keyExtent, dir, files, setTime);
+    }
+
+    public void send_loadFiles(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, long tid, org.apache.accumulo.core.data.thrift.TKeyExtent keyExtent, java.lang.String dir, java.util.Map<java.lang.String,org.apache.accumulo.core.data.thrift.MapFileInfo> files, boolean setTime) throws org.apache.thrift.TException
+    {
+      loadFiles_args args = new loadFiles_args();
+      args.setTinfo(tinfo);
+      args.setCredentials(credentials);
+      args.setTid(tid);
+      args.setKeyExtent(keyExtent);
+      args.setDir(dir);
+      args.setFiles(files);
+      args.setSetTime(setTime);
+      sendBaseOneway("loadFiles", args);
+    }
+
     public void splitTablet(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, org.apache.accumulo.core.data.thrift.TKeyExtent extent, java.nio.ByteBuffer splitPoint) throws org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException, NotServingTabletException, org.apache.thrift.TException
     {
       send_splitTablet(tinfo, credentials, extent, splitPoint);
@@ -1721,6 +1743,56 @@ public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apa
       }
     }
 
+    public void loadFiles(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, long tid, org.apache.accumulo.core.data.thrift.TKeyExtent keyExtent, java.lang.String dir, java.util.Map<java.lang.String,org.apache.accumulo.core.data.thrift.MapFileInfo> files, boolean setTime, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
+      checkReady();
+      loadFiles_call method_call = new loadFiles_call(tinfo, credentials, tid, keyExtent, dir, files, setTime, resultHandler, this, ___protocolFactory, ___transport);
+      this.___currentMethod = method_call;
+      ___manager.call(method_call);
+    }
+
+    public static class loadFiles_call extends org.apache.thrift.async.TAsyncMethodCall<Void> {
+      private org.apache.accumulo.core.trace.thrift.TInfo tinfo;
+      private org.apache.accumulo.core.security.thrift.TCredentials credentials;
+      private long tid;
+      private org.apache.accumulo.core.data.thrift.TKeyExtent keyExtent;
+      private java.lang.String dir;
+      private java.util.Map<java.lang.String,org.apache.accumulo.core.data.thrift.MapFileInfo> files;
+      private boolean setTime;
+      public loadFiles_call(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, long tid, org.apache.accumulo.core.data.thrift.TKeyExtent keyExtent, java.lang.String dir, java.util.Map<java.lang.String,org.apache.accumulo.core.data.thrift.MapFileInfo> files, boolean setTime, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+        super(client, protocolFactory, transport, resultHandler, true);
+        this.tinfo = tinfo;
+        this.credentials = credentials;
+        this.tid = tid;
+        this.keyExtent = keyExtent;
+        this.dir = dir;
+        this.files = files;
+        this.setTime = setTime;
+      }
+
+      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("loadFiles", org.apache.thrift.protocol.TMessageType.ONEWAY, 0));
+        loadFiles_args args = new loadFiles_args();
+        args.setTinfo(tinfo);
+        args.setCredentials(credentials);
+        args.setTid(tid);
+        args.setKeyExtent(keyExtent);
+        args.setDir(dir);
+        args.setFiles(files);
+        args.setSetTime(setTime);
+        args.write(prot);
+        prot.writeMessageEnd();
+      }
+
+      public Void getResult() throws org.apache.thrift.TException {
+        if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
+          throw new java.lang.IllegalStateException("Method call not finished!");
+        }
+        org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
+        org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
+        return null;
+      }
+    }
+
     public void splitTablet(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, org.apache.accumulo.core.data.thrift.TKeyExtent extent, java.nio.ByteBuffer splitPoint, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
       checkReady();
       splitTablet_call method_call = new splitTablet_call(tinfo, credentials, extent, splitPoint, resultHandler, this, ___protocolFactory, ___transport);
@@ -2539,6 +2611,7 @@ protected Processor(I iface, java.util.Map<java.lang.String, org.apache.thrift.P
       processMap.put("invalidateConditionalUpdate", new invalidateConditionalUpdate());
       processMap.put("closeConditionalUpdate", new closeConditionalUpdate());
       processMap.put("bulkImport", new bulkImport());
+      processMap.put("loadFiles", new loadFiles());
       processMap.put("splitTablet", new splitTablet());
       processMap.put("loadTablet", new loadTablet());
       processMap.put("unloadTablet", new unloadTablet());
@@ -2924,6 +2997,25 @@ public bulkImport_result getResult(I iface, bulkImport_args args) throws org.apa
       }
     }
 
+    public static class loadFiles<I extends Iface> extends org.apache.thrift.ProcessFunction<I, loadFiles_args> {
+      public loadFiles() {
+        super("loadFiles");
+      }
+
+      public loadFiles_args getEmptyArgsInstance() {
+        return new loadFiles_args();
+      }
+
+      protected boolean isOneway() {
+        return true;
+      }
+
+      public org.apache.thrift.TBase getResult(I iface, loadFiles_args args) throws org.apache.thrift.TException {
+        iface.loadFiles(args.tinfo, args.credentials, args.tid, args.keyExtent, args.dir, args.files, args.setTime);
+        return null;
+      }
+    }
+
     public static class splitTablet<I extends Iface> extends org.apache.thrift.ProcessFunction<I, splitTablet_args> {
       public splitTablet() {
         super("splitTablet");
@@ -3392,6 +3484,7 @@ protected AsyncProcessor(I iface, java.util.Map<java.lang.String,  org.apache.th
       processMap.put("invalidateConditionalUpdate", new invalidateConditionalUpdate());
       processMap.put("closeConditionalUpdate", new closeConditionalUpdate());
       processMap.put("bulkImport", new bulkImport());
+      processMap.put("loadFiles", new loadFiles());
       processMap.put("splitTablet", new splitTablet());
       processMap.put("loadTablet", new loadTablet());
       processMap.put("unloadTablet", new unloadTablet());
@@ -4331,115 +4424,13 @@ public void start(I iface, bulkImport_args args, org.apache.thrift.async.AsyncMe
       }
     }
 
-    public static class splitTablet<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, splitTablet_args, Void> {
-      public splitTablet() {
-        super("splitTablet");
-      }
-
-      public splitTablet_args getEmptyArgsInstance() {
-        return new splitTablet_args();
-      }
-
-      public org.apache.thrift.async.AsyncMethodCallback<Void> getResultHandler(final org.apache.thrift.server.AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) {
-        final org.apache.thrift.AsyncProcessFunction fcall = this;
-        return new org.apache.thrift.async.AsyncMethodCallback<Void>() { 
-          public void onComplete(Void o) {
-            splitTablet_result result = new splitTablet_result();
-            try {
-              fcall.sendResponse(fb, result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
-            } catch (org.apache.thrift.transport.TTransportException e) {
-              _LOGGER.error("TTransportException writing to internal frame buffer", e);
-              fb.close();
-            } catch (java.lang.Exception e) {
-              _LOGGER.error("Exception writing to internal frame buffer", e);
-              onError(e);
-            }
-          }
-          public void onError(java.lang.Exception e) {
-            byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
-            org.apache.thrift.TSerializable msg;
-            splitTablet_result result = new splitTablet_result();
-            if (e instanceof org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException) {
-              result.sec = (org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException) e;
-              result.setSecIsSet(true);
-              msg = result;
-            } else if (e instanceof NotServingTabletException) {
-              result.nste = (NotServingTabletException) e;
-              result.setNsteIsSet(true);
-              msg = result;
-            } else if (e instanceof org.apache.thrift.transport.TTransportException) {
-              _LOGGER.error("TTransportException inside handler", e);
-              fb.close();
-              return;
-            } else if (e instanceof org.apache.thrift.TApplicationException) {
-              _LOGGER.error("TApplicationException inside handler", e);
-              msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
-              msg = (org.apache.thrift.TApplicationException)e;
-            } else {
-              _LOGGER.error("Exception inside handler", e);
-              msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
-              msg = new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage());
-            }
-            try {
-              fcall.sendResponse(fb,msg,msgType,seqid);
-            } catch (java.lang.Exception ex) {
-              _LOGGER.error("Exception writing to internal frame buffer", ex);
-              fb.close();
-            }
-          }
-        };
-      }
-
-      protected boolean isOneway() {
-        return false;
-      }
-
-      public void start(I iface, splitTablet_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
-        iface.splitTablet(args.tinfo, args.credentials, args.extent, args.splitPoint,resultHandler);
-      }
-    }
-
-    public static class loadTablet<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, loadTablet_args, Void> {
-      public loadTablet() {
-        super("loadTablet");
-      }
-
-      public loadTablet_args getEmptyArgsInstance() {
-        return new loadTablet_args();
-      }
-
-      public org.apache.thrift.async.AsyncMethodCallback<Void> getResultHandler(final org.apache.thrift.server.AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) {
-        final org.apache.thrift.AsyncProcessFunction fcall = this;
-        return new org.apache.thrift.async.AsyncMethodCallback<Void>() { 
-          public void onComplete(Void o) {
-          }
-          public void onError(java.lang.Exception e) {
-            if (e instanceof org.apache.thrift.transport.TTransportException) {
-              _LOGGER.error("TTransportException inside handler", e);
-              fb.close();
-            } else {
-              _LOGGER.error("Exception inside oneway handler", e);
-            }
-          }
-        };
-      }
-
-      protected boolean isOneway() {
-        return true;
-      }
-
-      public void start(I iface, loadTablet_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
-        iface.loadTablet(args.tinfo, args.credentials, args.lock, args.extent,resultHandler);
-      }
-    }
-
-    public static class unloadTablet<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, unloadTablet_args, Void> {
-      public unloadTablet() {
-        super("unloadTablet");
+    public static class loadFiles<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, loadFiles_args, Void> {
+      public loadFiles() {
+        super("loadFiles");
       }
 
-      public unloadTablet_args getEmptyArgsInstance() {
-        return new unloadTablet_args();
+      public loadFiles_args getEmptyArgsInstance() {
+        return new loadFiles_args();
       }
 
       public org.apache.thrift.async.AsyncMethodCallback<Void> getResultHandler(final org.apache.thrift.server.AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) {
@@ -4462,162 +4453,25 @@ protected boolean isOneway() {
         return true;
       }
 
-      public void start(I iface, unloadTablet_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
-        iface.unloadTablet(args.tinfo, args.credentials, args.lock, args.extent, args.goal, args.requestTime,resultHandler);
+      public void start(I iface, loadFiles_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
+        iface.loadFiles(args.tinfo, args.credentials, args.tid, args.keyExtent, args.dir, args.files, args.setTime,resultHandler);
       }
     }
 
-    public static class flush<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, flush_args, Void> {
-      public flush() {
-        super("flush");
-      }
-
-      public flush_args getEmptyArgsInstance() {
-        return new flush_args();
-      }
-
-      public org.apache.thrift.async.AsyncMethodCallback<Void> getResultHandler(final org.apache.thrift.server.AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) {
-        final org.apache.thrift.AsyncProcessFunction fcall = this;
-        return new org.apache.thrift.async.AsyncMethodCallback<Void>() { 
-          public void onComplete(Void o) {
-          }
-          public void onError(java.lang.Exception e) {
-            if (e instanceof org.apache.thrift.transport.TTransportException) {
-              _LOGGER.error("TTransportException inside handler", e);
-              fb.close();
-            } else {
-              _LOGGER.error("Exception inside oneway handler", e);
-            }
-          }
-        };
-      }
-
-      protected boolean isOneway() {
-        return true;
-      }
-
-      public void start(I iface, flush_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
-        iface.flush(args.tinfo, args.credentials, args.lock, args.tableId, args.startRow, args.endRow,resultHandler);
-      }
-    }
-
-    public static class flushTablet<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, flushTablet_args, Void> {
-      public flushTablet() {
-        super("flushTablet");
-      }
-
-      public flushTablet_args getEmptyArgsInstance() {
-        return new flushTablet_args();
-      }
-
-      public org.apache.thrift.async.AsyncMethodCallback<Void> getResultHandler(final org.apache.thrift.server.AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) {
-        final org.apache.thrift.AsyncProcessFunction fcall = this;
-        return new org.apache.thrift.async.AsyncMethodCallback<Void>() { 
-          public void onComplete(Void o) {
-          }
-          public void onError(java.lang.Exception e) {
-            if (e instanceof org.apache.thrift.transport.TTransportException) {
-              _LOGGER.error("TTransportException inside handler", e);
-              fb.close();
-            } else {
-              _LOGGER.error("Exception inside oneway handler", e);
-            }
-          }
-        };
-      }
-
-      protected boolean isOneway() {
-        return true;
-      }
-
-      public void start(I iface, flushTablet_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
-        iface.flushTablet(args.tinfo, args.credentials, args.lock, args.extent,resultHandler);
-      }
-    }
-
-    public static class chop<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, chop_args, Void> {
-      public chop() {
-        super("chop");
-      }
-
-      public chop_args getEmptyArgsInstance() {
-        return new chop_args();
-      }
-
-      public org.apache.thrift.async.AsyncMethodCallback<Void> getResultHandler(final org.apache.thrift.server.AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) {
-        final org.apache.thrift.AsyncProcessFunction fcall = this;
-        return new org.apache.thrift.async.AsyncMethodCallback<Void>() { 
-          public void onComplete(Void o) {
-          }
-          public void onError(java.lang.Exception e) {
-            if (e instanceof org.apache.thrift.transport.TTransportException) {
-              _LOGGER.error("TTransportException inside handler", e);
-              fb.close();
-            } else {
-              _LOGGER.error("Exception inside oneway handler", e);
-            }
-          }
-        };
-      }
-
-      protected boolean isOneway() {
-        return true;
-      }
-
-      public void start(I iface, chop_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
-        iface.chop(args.tinfo, args.credentials, args.lock, args.extent,resultHandler);
-      }
-    }
-
-    public static class compact<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, compact_args, Void> {
-      public compact() {
-        super("compact");
+    public static class splitTablet<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, splitTablet_args, Void> {
+      public splitTablet() {
+        super("splitTablet");
       }
 
-      public compact_args getEmptyArgsInstance() {
-        return new compact_args();
+      public splitTablet_args getEmptyArgsInstance() {
+        return new splitTablet_args();
       }
 
       public org.apache.thrift.async.AsyncMethodCallback<Void> getResultHandler(final org.apache.thrift.server.AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) {
         final org.apache.thrift.AsyncProcessFunction fcall = this;
         return new org.apache.thrift.async.AsyncMethodCallback<Void>() { 
           public void onComplete(Void o) {
-          }
-          public void onError(java.lang.Exception e) {
-            if (e instanceof org.apache.thrift.transport.TTransportException) {
-              _LOGGER.error("TTransportException inside handler", e);
-              fb.close();
-            } else {
-              _LOGGER.error("Exception inside oneway handler", e);
-            }
-          }
-        };
-      }
-
-      protected boolean isOneway() {
-        return true;
-      }
-
-      public void start(I iface, compact_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
-        iface.compact(args.tinfo, args.credentials, args.lock, args.tableId, args.startRow, args.endRow,resultHandler);
-      }
-    }
-
-    public static class getTabletServerStatus<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, getTabletServerStatus_args, org.apache.accumulo.core.master.thrift.TabletServerStatus> {
-      public getTabletServerStatus() {
-        super("getTabletServerStatus");
-      }
-
-      public getTabletServerStatus_args getEmptyArgsInstance() {
-        return new getTabletServerStatus_args();
-      }
-
-      public org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.master.thrift.TabletServerStatus> getResultHandler(final org.apache.thrift.server.AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) {
-        final org.apache.thrift.AsyncProcessFunction fcall = this;
-        return new org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.master.thrift.TabletServerStatus>() { 
-          public void onComplete(org.apache.accumulo.core.master.thrift.TabletServerStatus o) {
-            getTabletServerStatus_result result = new getTabletServerStatus_result();
-            result.success = o;
+            splitTablet_result result = new splitTablet_result();
             try {
               fcall.sendResponse(fb, result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
             } catch (org.apache.thrift.transport.TTransportException e) {
@@ -4631,7 +4485,280 @@ public void onComplete(org.apache.accumulo.core.master.thrift.TabletServerStatus
           public void onError(java.lang.Exception e) {
             byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
             org.apache.thrift.TSerializable msg;
-            getTabletServerStatus_result result = new getTabletServerStatus_result();
+            splitTablet_result result = new splitTablet_result();
+            if (e instanceof org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException) {
+              result.sec = (org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException) e;
+              result.setSecIsSet(true);
+              msg = result;
+            } else if (e instanceof NotServingTabletException) {
+              result.nste = (NotServingTabletException) e;
+              result.setNsteIsSet(true);
+              msg = result;
+            } else if (e instanceof org.apache.thrift.transport.TTransportException) {
+              _LOGGER.error("TTransportException inside handler", e);
+              fb.close();
+              return;
+            } else if (e instanceof org.apache.thrift.TApplicationException) {
+              _LOGGER.error("TApplicationException inside handler", e);
+              msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
+              msg = (org.apache.thrift.TApplicationException)e;
+            } else {
+              _LOGGER.error("Exception inside handler", e);
+              msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
+              msg = new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage());
+            }
+            try {
+              fcall.sendResponse(fb,msg,msgType,seqid);
+            } catch (java.lang.Exception ex) {
+              _LOGGER.error("Exception writing to internal frame buffer", ex);
+              fb.close();
+            }
+          }
+        };
+      }
+
+      protected boolean isOneway() {
+        return false;
+      }
+
+      public void start(I iface, splitTablet_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
+        iface.splitTablet(args.tinfo, args.credentials, args.extent, args.splitPoint,resultHandler);
+      }
+    }
+
+    public static class loadTablet<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, loadTablet_args, Void> {
+      public loadTablet() {
+        super("loadTablet");
+      }
+
+      public loadTablet_args getEmptyArgsInstance() {
+        return new loadTablet_args();
+      }
+
+      public org.apache.thrift.async.AsyncMethodCallback<Void> getResultHandler(final org.apache.thrift.server.AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) {
+        final org.apache.thrift.AsyncProcessFunction fcall = this;
+        return new org.apache.thrift.async.AsyncMethodCallback<Void>() { 
+          public void onComplete(Void o) {
+          }
+          public void onError(java.lang.Exception e) {
+            if (e instanceof org.apache.thrift.transport.TTransportException) {
+              _LOGGER.error("TTransportException inside handler", e);
+              fb.close();
+            } else {
+              _LOGGER.error("Exception inside oneway handler", e);
+            }
+          }
+        };
+      }
+
+      protected boolean isOneway() {
+        return true;
+      }
+
+      public void start(I iface, loadTablet_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
+        iface.loadTablet(args.tinfo, args.credentials, args.lock, args.extent,resultHandler);
+      }
+    }
+
+    public static class unloadTablet<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, unloadTablet_args, Void> {
+      public unloadTablet() {
+        super("unloadTablet");
+      }
+
+      public unloadTablet_args getEmptyArgsInstance() {
+        return new unloadTablet_args();
+      }
+
+      public org.apache.thrift.async.AsyncMethodCallback<Void> getResultHandler(final org.apache.thrift.server.AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) {
+        final org.apache.thrift.AsyncProcessFunction fcall = this;
+        return new org.apache.thrift.async.AsyncMethodCallback<Void>() { 
+          public void onComplete(Void o) {
+          }
+          public void onError(java.lang.Exception e) {
+            if (e instanceof org.apache.thrift.transport.TTransportException) {
+              _LOGGER.error("TTransportException inside handler", e);
+              fb.close();
+            } else {
+              _LOGGER.error("Exception inside oneway handler", e);
+            }
+          }
+        };
+      }
+
+      protected boolean isOneway() {
+        return true;
+      }
+
+      public void start(I iface, unloadTablet_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
+        iface.unloadTablet(args.tinfo, args.credentials, args.lock, args.extent, args.goal, args.requestTime,resultHandler);
+      }
+    }
+
+    public static class flush<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, flush_args, Void> {
+      public flush() {
+        super("flush");
+      }
+
+      public flush_args getEmptyArgsInstance() {
+        return new flush_args();
+      }
+
+      public org.apache.thrift.async.AsyncMethodCallback<Void> getResultHandler(final org.apache.thrift.server.AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) {
+        final org.apache.thrift.AsyncProcessFunction fcall = this;
+        return new org.apache.thrift.async.AsyncMethodCallback<Void>() { 
+          public void onComplete(Void o) {
+          }
+          public void onError(java.lang.Exception e) {
+            if (e instanceof org.apache.thrift.transport.TTransportException) {
+              _LOGGER.error("TTransportException inside handler", e);
+              fb.close();
+            } else {
+              _LOGGER.error("Exception inside oneway handler", e);
+            }
+          }
+        };
+      }
+
+      protected boolean isOneway() {
+        return true;
+      }
+
+      public void start(I iface, flush_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
+        iface.flush(args.tinfo, args.credentials, args.lock, args.tableId, args.startRow, args.endRow,resultHandler);
+      }
+    }
+
+    public static class flushTablet<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, flushTablet_args, Void> {
+      public flushTablet() {
+        super("flushTablet");
+      }
+
+      public flushTablet_args getEmptyArgsInstance() {
+        return new flushTablet_args();
+      }
+
+      public org.apache.thrift.async.AsyncMethodCallback<Void> getResultHandler(final org.apache.thrift.server.AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) {
+        final org.apache.thrift.AsyncProcessFunction fcall = this;
+        return new org.apache.thrift.async.AsyncMethodCallback<Void>() { 
+          public void onComplete(Void o) {
+          }
+          public void onError(java.lang.Exception e) {
+            if (e instanceof org.apache.thrift.transport.TTransportException) {
+              _LOGGER.error("TTransportException inside handler", e);
+              fb.close();
+            } else {
+              _LOGGER.error("Exception inside oneway handler", e);
+            }
+          }
+        };
+      }
+
+      protected boolean isOneway() {
+        return true;
+      }
+
+      public void start(I iface, flushTablet_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
+        iface.flushTablet(args.tinfo, args.credentials, args.lock, args.extent,resultHandler);
+      }
+    }
+
+    public static class chop<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, chop_args, Void> {
+      public chop() {
+        super("chop");
+      }
+
+      public chop_args getEmptyArgsInstance() {
+        return new chop_args();
+      }
+
+      public org.apache.thrift.async.AsyncMethodCallback<Void> getResultHandler(final org.apache.thrift.server.AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) {
+        final org.apache.thrift.AsyncProcessFunction fcall = this;
+        return new org.apache.thrift.async.AsyncMethodCallback<Void>() { 
+          public void onComplete(Void o) {
+          }
+          public void onError(java.lang.Exception e) {
+            if (e instanceof org.apache.thrift.transport.TTransportException) {
+              _LOGGER.error("TTransportException inside handler", e);
+              fb.close();
+            } else {
+              _LOGGER.error("Exception inside oneway handler", e);
+            }
+          }
+        };
+      }
+
+      protected boolean isOneway() {
+        return true;
+      }
+
+      public void start(I iface, chop_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
+        iface.chop(args.tinfo, args.credentials, args.lock, args.extent,resultHandler);
+      }
+    }
+
+    public static class compact<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, compact_args, Void> {
+      public compact() {
+        super("compact");
+      }
+
+      public compact_args getEmptyArgsInstance() {
+        return new compact_args();
+      }
+
+      public org.apache.thrift.async.AsyncMethodCallback<Void> getResultHandler(final org.apache.thrift.server.AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) {
+        final org.apache.thrift.AsyncProcessFunction fcall = this;
+        return new org.apache.thrift.async.AsyncMethodCallback<Void>() { 
+          public void onComplete(Void o) {
+          }
+          public void onError(java.lang.Exception e) {
+            if (e instanceof org.apache.thrift.transport.TTransportException) {
+              _LOGGER.error("TTransportException inside handler", e);
+              fb.close();
+            } else {
+              _LOGGER.error("Exception inside oneway handler", e);
+            }
+          }
+        };
+      }
+
+      protected boolean isOneway() {
+        return true;
+      }
+
+      public void start(I iface, compact_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
+        iface.compact(args.tinfo, args.credentials, args.lock, args.tableId, args.startRow, args.endRow,resultHandler);
+      }
+    }
+
+    public static class getTabletServerStatus<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, getTabletServerStatus_args, org.apache.accumulo.core.master.thrift.TabletServerStatus> {
+      public getTabletServerStatus() {
+        super("getTabletServerStatus");
+      }
+
+      public getTabletServerStatus_args getEmptyArgsInstance() {
+        return new getTabletServerStatus_args();
+      }
+
+      public org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.master.thrift.TabletServerStatus> getResultHandler(final org.apache.thrift.server.AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) {
+        final org.apache.thrift.AsyncProcessFunction fcall = this;
+        return new org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.master.thrift.TabletServerStatus>() { 
+          public void onComplete(org.apache.accumulo.core.master.thrift.TabletServerStatus o) {
+            getTabletServerStatus_result result = new getTabletServerStatus_result();
+            result.success = o;
+            try {
+              fcall.sendResponse(fb, result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
+            } catch (org.apache.thrift.transport.TTransportException e) {
+              _LOGGER.error("TTransportException writing to internal frame buffer", e);
+              fb.close();
+            } catch (java.lang.Exception e) {
+              _LOGGER.error("Exception writing to internal frame buffer", e);
+              onError(e);
+            }
+          }
+          public void onError(java.lang.Exception e) {
+            byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
+            org.apache.thrift.TSerializable msg;
+            getTabletServerStatus_result result = new getTabletServerStatus_result();
             if (e instanceof org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException) {
               result.sec = (org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException) e;
               result.setSecIsSet(true);
@@ -23689,6 +23816,1060 @@ public void read(org.apache.thrift.protocol.TProtocol prot, bulkImport_result st
     }
   }
 
+  public static class loadFiles_args implements org.apache.thrift.TBase<loadFiles_args, loadFiles_args._Fields>, java.io.Serializable, Cloneable, Comparable<loadFiles_args>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("loadFiles_args");
+
+    private static final org.apache.thrift.protocol.TField TINFO_FIELD_DESC = new org.apache.thrift.protocol.TField("tinfo", org.apache.thrift.protocol.TType.STRUCT, (short)1);
+    private static final org.apache.thrift.protocol.TField CREDENTIALS_FIELD_DESC = new org.apache.thrift.protocol.TField("credentials", org.apache.thrift.protocol.TType.STRUCT, (short)2);
+    private static final org.apache.thrift.protocol.TField TID_FIELD_DESC = new org.apache.thrift.protocol.TField("tid", org.apache.thrift.protocol.TType.I64, (short)3);
+    private static final org.apache.thrift.protocol.TField KEY_EXTENT_FIELD_DESC = new org.apache.thrift.protocol.TField("keyExtent", org.apache.thrift.protocol.TType.STRUCT, (short)4);
+    private static final org.apache.thrift.protocol.TField DIR_FIELD_DESC = new org.apache.thrift.protocol.TField("dir", org.apache.thrift.protocol.TType.STRING, (short)5);
+    private static final org.apache.thrift.protocol.TField FILES_FIELD_DESC = new org.apache.thrift.protocol.TField("files", org.apache.thrift.protocol.TType.MAP, (short)6);
+    private static final org.apache.thrift.protocol.TField SET_TIME_FIELD_DESC = new org.apache.thrift.protocol.TField("setTime", org.apache.thrift.protocol.TType.BOOL, (short)7);
+
+    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new loadFiles_argsStandardSchemeFactory();
+    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new loadFiles_argsTupleSchemeFactory();
+
+    public org.apache.accumulo.core.trace.thrift.TInfo tinfo; // required
+    public org.apache.accumulo.core.security.thrift.TCredentials credentials; // required
+    public long tid; // required
+    public org.apache.accumulo.core.data.thrift.TKeyExtent keyExtent; // required
+    public java.lang.String dir; // required
+    public java.util.Map<java.lang.String,org.apache.accumulo.core.data.thrift.MapFileInfo> files; // required
+    public boolean setTime; // required
+
+    /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+    public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+      TINFO((short)1, "tinfo"),
+      CREDENTIALS((short)2, "credentials"),
+      TID((short)3, "tid"),
+      KEY_EXTENT((short)4, "keyExtent"),
+      DIR((short)5, "dir"),
+      FILES((short)6, "files"),
+      SET_TIME((short)7, "setTime");
+
+      private static final java.util.Map<java.lang.String, _Fields> byName = new java.util.HashMap<java.lang.String, _Fields>();
+
+      static {
+        for (_Fields field : java.util.EnumSet.allOf(_Fields.class)) {
+          byName.put(field.getFieldName(), field);
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, or null if its not found.
+       */
+      public static _Fields findByThriftId(int fieldId) {
+        switch(fieldId) {
+          case 1: // TINFO
+            return TINFO;
+          case 2: // CREDENTIALS
+            return CREDENTIALS;
+          case 3: // TID
+            return TID;
+          case 4: // KEY_EXTENT
+            return KEY_EXTENT;
+          case 5: // DIR
+            return DIR;
+          case 6: // FILES
+            return FILES;
+          case 7: // SET_TIME
+            return SET_TIME;
+          default:
+            return null;
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, throwing an exception
+       * if it is not found.
+       */
+      public static _Fields findByThriftIdOrThrow(int fieldId) {
+        _Fields fields = findByThriftId(fieldId);
+        if (fields == null) throw new java.lang.IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+        return fields;
+      }
+
+      /**
+       * Find the _Fields constant that matches name, or null if its not found.
+       */
+      public static _Fields findByName(java.lang.String name) {
+        return byName.get(name);
+      }
+
+      private final short _thriftId;
+      private final java.lang.String _fieldName;
+
+      _Fields(short thriftId, java.lang.String fieldName) {
+        _thriftId = thriftId;
+        _fieldName = fieldName;
+      }
+
+      public short getThriftFieldId() {
+        return _thriftId;
+      }
+
+      public java.lang.String getFieldName() {
+        return _fieldName;
+      }
+    }
+
+    // isset id assignments
+    private static final int __TID_ISSET_ID = 0;
+    private static final int __SETTIME_ISSET_ID = 1;
+    private byte __isset_bitfield = 0;
+    public static final java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+    static {
+      java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+      tmpMap.put(_Fields.TINFO, new org.apache.thrift.meta_data.FieldMetaData("tinfo", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.trace.thrift.TInfo.class)));
+      tmpMap.put(_Fields.CREDENTIALS, new org.apache.thrift.meta_data.FieldMetaData("credentials", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.security.thrift.TCredentials.class)));
+      tmpMap.put(_Fields.TID, new org.apache.thrift.meta_data.FieldMetaData("tid", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64)));
+      tmpMap.put(_Fields.KEY_EXTENT, new org.apache.thrift.meta_data.FieldMetaData("keyExtent", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.data.thrift.TKeyExtent.class)));
+      tmpMap.put(_Fields.DIR, new org.apache.thrift.meta_data.FieldMetaData("dir", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+      tmpMap.put(_Fields.FILES, new org.apache.thrift.meta_data.FieldMetaData("files", 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.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.data.thrift.MapFileInfo.class))));
+      tmpMap.put(_Fields.SET_TIME, new org.apache.thrift.meta_data.FieldMetaData("setTime", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.BOOL)));
+      metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(loadFiles_args.class, metaDataMap);
+    }
+
+    public loadFiles_args() {
+    }
+
+    public loadFiles_args(
+      org.apache.accumulo.core.trace.thrift.TInfo tinfo,
+      org.apache.accumulo.core.security.thrift.TCredentials credentials,
+      long tid,
+      org.apache.accumulo.core.data.thrift.TKeyExtent keyExtent,
+      java.lang.String dir,
+      java.util.Map<java.lang.String,org.apache.accumulo.core.data.thrift.MapFileInfo> files,
+      boolean setTime)
+    {
+      this();
+      this.tinfo = tinfo;
+      this.credentials = credentials;
+      this.tid = tid;
+      setTidIsSet(true);
+      this.keyExtent = keyExtent;
+      this.dir = dir;
+      this.files = files;
+      this.setTime = setTime;
+      setSetTimeIsSet(true);
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public loadFiles_args(loadFiles_args other) {
+      __isset_bitfield = other.__isset_bitfield;
+      if (other.isSetTinfo()) {
+        this.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo(other.tinfo);
+      }
+      if (other.isSetCredentials()) {
+        this.credentials = new org.apache.accumulo.core.security.thrift.TCredentials(other.credentials);
+      }
+      this.tid = other.tid;
+      if (other.isSetKeyExtent()) {
+        this.keyExtent = new org.apache.accumulo.core.data.thrift.TKeyExtent(other.keyExtent);
+      }
+      if (other.isSetDir()) {
+        this.dir = other.dir;
+      }
+      if (other.isSetFiles()) {
+        java.util.Map<java.lang.String,org.apache.accumulo.core.data.thrift.MapFileInfo> __this__files = new java.util.HashMap<java.lang.String,org.apache.accumulo.core.data.thrift.MapFileInfo>(other.files.size());
+        for (java.util.Map.Entry<java.lang.String, org.apache.accumulo.core.data.thrift.MapFileInfo> other_element : other.files.entrySet()) {
+
+          java.lang.String other_element_key = other_element.getKey();
+          org.apache.accumulo.core.data.thrift.MapFileInfo other_element_value = other_element.getValue();
+
+          java.lang.String __this__files_copy_key = other_element_key;
+
+          org.apache.accumulo.core.data.thrift.MapFileInfo __this__files_copy_value = new org.apache.accumulo.core.data.thrift.MapFileInfo(other_element_value);
+
+          __this__files.put(__this__files_copy_key, __this__files_copy_value);
+        }
+        this.files = __this__files;
+      }
+      this.setTime = other.setTime;
+    }
+
+    public loadFiles_args deepCopy() {
+      return new loadFiles_args(this);
+    }
+
+    @Override
+    public void clear() {
+      this.tinfo = null;
+      this.credentials = null;
+      setTidIsSet(false);
+      this.tid = 0;
+      this.keyExtent = null;
+      this.dir = null;
+      this.files = null;
+      setSetTimeIsSet(false);
+      this.setTime = false;
+    }
+
+    public org.apache.accumulo.core.trace.thrift.TInfo getTinfo() {
+      return this.tinfo;
+    }
+
+    public loadFiles_args setTinfo(org.apache.accumulo.core.trace.thrift.TInfo tinfo) {
+      this.tinfo = tinfo;
+      return this;
+    }
+
+    public void unsetTinfo() {
+      this.tinfo = null;
+    }
+
+    /** Returns true if field tinfo is set (has been assigned a value) and false otherwise */
+    public boolean isSetTinfo() {
+      return this.tinfo != null;
+    }
+
+    public void setTinfoIsSet(boolean value) {
+      if (!value) {
+        this.tinfo = null;
+      }
+    }
+
+    public org.apache.accumulo.core.security.thrift.TCredentials getCredentials() {
+      return this.credentials;
+    }
+
+    public loadFiles_args setCredentials(org.apache.accumulo.core.security.thrift.TCredentials credentials) {
+      this.credentials = credentials;
+      return this;
+    }
+
+    public void unsetCredentials() {
+      this.credentials = null;
+    }
+
+    /** Returns true if field credentials is set (has been assigned a value) and false otherwise */
+    public boolean isSetCredentials() {
+      return this.credentials != null;
+    }
+
+    public void setCredentialsIsSet(boolean value) {
+      if (!value) {
+        this.credentials = null;
+      }
+    }
+
+    public long getTid() {
+      return this.tid;
+    }
+
+    public loadFiles_args setTid(long tid) {
+      this.tid = tid;
+      setTidIsSet(true);
+      return this;
+    }
+
+    public void unsetTid() {
+      __isset_bitfield = org.apache.thrift.EncodingUtils.clearBit(__isset_bitfield, __TID_ISSET_ID);
+    }
+
+    /** Returns true if field tid is set (has been assigned a value) and false otherwise */
+    public boolean isSetTid() {
+      return org.apache.thrift.EncodingUtils.testBit(__isset_bitfield, __TID_ISSET_ID);
+    }
+
+    public void setTidIsSet(boolean value) {
+      __isset_bitfield = org.apache.thrift.EncodingUtils.setBit(__isset_bitfield, __TID_ISSET_ID, value);
+    }
+
+    public org.apache.accumulo.core.data.thrift.TKeyExtent getKeyExtent() {
+      return this.keyExtent;
+    }
+
+    public loadFiles_args setKeyExtent(org.apache.accumulo.core.data.thrift.TKeyExtent keyExtent) {
+      this.keyExtent = keyExtent;
+      return this;
+    }
+
+    public void unsetKeyExtent() {
+      this.keyExtent = null;
+    }
+
+    /** Returns true if field keyExtent is set (has been assigned a value) and false otherwise */
+    public boolean isSetKeyExtent() {
+      return this.keyExtent != null;
+    }
+
+    public void setKeyExtentIsSet(boolean value) {
+      if (!value) {
+        this.keyExtent = null;
+      }
+    }
+
+    public java.lang.String getDir() {
+      return this.dir;
+    }
+
+    public loadFiles_args setDir(java.lang.String dir) {
+      this.dir = dir;
+      return this;
+    }
+
+    public void unsetDir() {
+      this.dir = null;
+    }
+
+    /** Returns true if field dir is set (has been assigned a value) and false otherwise */
+    public boolean isSetDir() {
+      return this.dir != null;
+    }
+
+    public void setDirIsSet(boolean value) {
+      if (!value) {
+        this.dir = null;
+      }
+    }
+
+    public int getFilesSize() {
+      return (this.files == null) ? 0 : this.files.size();
+    }
+
+    public void putToFiles(java.lang.String key, org.apache.accumulo.core.data.thrift.MapFileInfo val) {
+      if (this.files == null) {
+        this.files = new java.util.HashMap<java.lang.String,org.apache.accumulo.core.data.thrift.MapFileInfo>();
+      }
+      this.files.put(key, val);
+    }
+
+    public java.util.Map<java.lang.String,org.apache.accumulo.core.data.thrift.MapFileInfo> getFiles() {
+      return this.files;
+    }
+
+    public loadFiles_args setFiles(java.util.Map<java.lang.String,org.apache.accumulo.core.data.thrift.MapFileInfo> files) {
+      this.files = files;
+      return this;
+    }
+
+    public void unsetFiles() {
+      this.files = null;
+    }
+
+    /** Returns true if field files is set (has been assigned a value) and false otherwise */
+    public boolean isSetFiles() {
+      return this.files != null;
+    }
+
+    public void setFilesIsSet(boolean value) {
+      if (!value) {
+        this.files = null;
+      }
+    }
+
+    public boolean isSetTime() {
+      return this.setTime;
+    }
+
+    public loadFiles_args setSetTime(boolean setTime) {
+      this.setTime = setTime;
+      setSetTimeIsSet(true);
+      return this;
+    }
+
+    public void unsetSetTime() {
+      __isset_bitfield = org.apache.thrift.EncodingUtils.clearBit(__isset_bitfield, __SETTIME_ISSET_ID);
+    }
+
+    /** Returns true if field setTime is set (has been assigned a value) and false otherwise */
+    public boolean isSetSetTime() {
+      return org.apache.thrift.EncodingUtils.testBit(__isset_bitfield, __SETTIME_ISSET_ID);
+    }
+
+    public void setSetTimeIsSet(boolean value) {
+      __isset_bitfield = org.apache.thrift.EncodingUtils.setBit(__isset_bitfield, __SETTIME_ISSET_ID, value);
+    }
+
+    public void setFieldValue(_Fields field, java.lang.Object value) {
+      switch (field) {
+      case TINFO:
+        if (value == null) {
+          unsetTinfo();
+        } else {
+          setTinfo((org.apache.accumulo.core.trace.thrift.TInfo)value);
+        }
+        break;
+
+      case CREDENTIALS:
+        if (value == null) {
+          unsetCredentials();
+        } else {
+          setCredentials((org.apache.accumulo.core.security.thrift.TCredentials)value);
+        }
+        break;
+
+      case TID:
+        if (value == null) {
+          unsetTid();
+        } else {
+          setTid((java.lang.Long)value);
+        }
+        break;
+
+      case KEY_EXTENT:
+        if (value == null) {
+          unsetKeyExtent();
+        } else {
+          setKeyExtent((org.apache.accumulo.core.data.thrift.TKeyExtent)value);
+        }
+        break;
+
+      case DIR:
+        if (value == null) {
+          unsetDir();
+        } else {
+          setDir((java.lang.String)value);
+        }
+        break;
+
+      case FILES:
+        if (value == null) {
+          unsetFiles();
+        } else {
+          setFiles((java.util.Map<java.lang.String,org.apache.accumulo.core.data.thrift.MapFileInfo>)value);
+        }
+        break;
+
+      case SET_TIME:
+        if (value == null) {
+          unsetSetTime();
+        } else {
+          setSetTime((java.lang.Boolean)value);
+        }
+        break;
+
+      }
+    }
+
+    public java.lang.Object getFieldValue(_Fields field) {
+      switch (field) {
+      case TINFO:
+        return getTinfo();
+
+      case CREDENTIALS:
+        return getCredentials();
+
+      case TID:
+        return getTid();
+
+      case KEY_EXTENT:
+        return getKeyExtent();
+
+      case DIR:
+        return getDir();
+
+      case FILES:
+        return getFiles();
+
+      case SET_TIME:
+        return isSetTime();
+
+      }
+      throw new java.lang.IllegalStateException();
+    }
+
+    /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+    public boolean isSet(_Fields field) {
+      if (field == null) {
+        throw new java.lang.IllegalArgumentException();
+      }
+
+      switch (field) {
+      case TINFO:
+        return isSetTinfo();
+      case CREDENTIALS:
+        return isSetCredentials();
+      case TID:
+        return isSetTid();
+      case KEY_EXTENT:
+        return isSetKeyExtent();
+      case DIR:
+        return isSetDir();
+      case FILES:
+        return isSetFiles();
+      case SET_TIME:
+        return isSetSetTime();
+      }
+      throw new java.lang.IllegalStateException();
+    }
+
+    @Override
+    public boolean equals(java.lang.Object that) {
+      if (that == null)
+        return false;
+      if (that instanceof loadFiles_args)
+        return this.equals((loadFiles_args)that);
+      return false;
+    }
+
+    public boolean equals(loadFiles_args that) {
+      if (that == null)
+        return false;
+      if (this == that)
+        return true;
+
+      boolean this_present_tinfo = true && this.isSetTinfo();
+      boolean that_present_tinfo = true && that.isSetTinfo();
+      if (this_present_tinfo || that_present_tinfo) {
+        if (!(this_present_tinfo && that_present_tinfo))
+          return false;
+        if (!this.tinfo.equals(that.tinfo))
+          return false;
+      }
+
+      boolean this_present_credentials = true && this.isSetCredentials();
+      boolean that_present_credentials = true && that.isSetCredentials();
+      if (this_present_credentials || that_present_credentials) {
+        if (!(this_present_credentials && that_present_credentials))
+          return false;
+        if (!this.credentials.equals(that.credentials))
+          return false;
+      }
+
+      boolean this_present_tid = true;
+      boolean that_present_tid = true;
+      if (this_present_tid || that_present_tid) {
+        if (!(this_present_tid && that_present_tid))
+          return false;
+        if (this.tid != that.tid)
+          return false;
+      }
+
+      boolean this_present_keyExtent = true && this.isSetKeyExtent();
+      boolean that_present_keyExtent = true && that.isSetKeyExtent();
+      if (this_present_keyExtent || that_present_keyExtent) {
+        if (!(this_present_keyExtent && that_present_keyExtent))
+          return false;
+        if (!this.keyExtent.equals(that.keyExtent))
+          return false;
+      }
+
+      boolean this_present_dir = true && this.isSetDir();
+      boolean that_present_dir = true && that.isSetDir();
+      if (this_present_dir || that_present_dir) {
+        if (!(this_present_dir && that_present_dir))
+          return false;
+        if (!this.dir.equals(that.dir))
+          return false;
+      }
+
+      boolean this_present_files = true && this.isSetFiles();
+      boolean that_present_files = true && that.isSetFiles();
+      if (this_present_files || that_present_files) {
+        if (!(this_present_files && that_present_files))
+          return false;
+        if (!this.files.equals(that.files))
+          return false;
+      }
+
+      boolean this_present_setTime = true;
+      boolean that_present_setTime = true;
+      if (this_present_setTime || that_present_setTime) {
+        if (!(this_present_setTime && that_present_setTime))
+          return false;
+        if (this.setTime != that.setTime)
+          return false;
+      }
+
+      return true;
+    }
+
+    @Override
+    public int hashCode() {
+      int hashCode = 1;
+
+      hashCode = hashCode * 8191 + ((isSetTinfo()) ? 131071 : 524287);
+      if (isSetTinfo())
+        hashCode = hashCode * 8191 + tinfo.hashCode();
+
+      hashCode = hashCode * 8191 + ((isSetCredentials()) ? 131071 : 524287);
+      if (isSetCredentials())
+        hashCode = hashCode * 8191 + credentials.hashCode();
+
+      hashCode = hashCode * 8191 + org.apache.thrift.TBaseHelper.hashCode(tid);
+
+      hashCode = hashCode * 8191 + ((isSetKeyExtent()) ? 131071 : 524287);
+      if (isSetKeyExtent())
+        hashCode = hashCode * 8191 + keyExtent.hashCode();
+
+      hashCode = hashCode * 8191 + ((isSetDir()) ? 131071 : 524287);
+      if (isSetDir())
+        hashCode = hashCode * 8191 + dir.hashCode();
+
+      hashCode = hashCode * 8191 + ((isSetFiles()) ? 131071 : 524287);
+      if (isSetFiles())
+        hashCode = hashCode * 8191 + files.hashCode();
+
+      hashCode = hashCode * 8191 + ((setTime) ? 131071 : 524287);
+
+      return hashCode;
+    }
+
+    @Override
+    public int compareTo(loadFiles_args other) {
+      if (!getClass().equals(other.getClass())) {
+        return getClass().getName().compareTo(other.getClass().getName());
+      }
+
+      int lastComparison = 0;
+
+      lastComparison = java.lang.Boolean.valueOf(isSetTinfo()).compareTo(other.isSetTinfo());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetTinfo()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tinfo, other.tinfo);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      lastComparison = java.lang.Boolean.valueOf(isSetCredentials()).compareTo(other.isSetCredentials());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetCredentials()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.credentials, other.credentials);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      lastComparison = java.lang.Boolean.valueOf(isSetTid()).compareTo(other.isSetTid());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetTid()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tid, other.tid);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      lastComparison = java.lang.Boolean.valueOf(isSetKeyExtent()).compareTo(other.isSetKeyExtent());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetKeyExtent()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.keyExtent, other.keyExtent);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      lastComparison = java.lang.Boolean.valueOf(isSetDir()).compareTo(other.isSetDir());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetDir()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.dir, other.dir);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      lastComparison = java.lang.Boolean.valueOf(isSetFiles()).compareTo(other.isSetFiles());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetFiles()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.files, other.files);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      lastComparison = java.lang.Boolean.valueOf(isSetSetTime()).compareTo(other.isSetSetTime());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetSetTime()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.setTime, other.setTime);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      return 0;
+    }
+
+    public _Fields fieldForId(int fieldId) {
+      return _Fields.findByThriftId(fieldId);
+    }
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+      scheme(iprot).read(iprot, this);
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+      scheme(oprot).write(oprot, this);
+    }
+
+    @Override
+    public java.lang.String toString() {
+      java.lang.StringBuilder sb = new java.lang.StringBuilder("loadFiles_args(");
+      boolean first = true;
+
+      sb.append("tinfo:");
+      if (this.tinfo == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.tinfo);
+      }
+      first = false;
+      if (!first) sb.append(", ");
+      sb.append("credentials:");
+      if (this.credentials == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.credentials);
+      }
+      first = false;
+      if (!first) sb.append(", ");
+      sb.append("tid:");
+      sb.append(this.tid);
+      first = false;
+      if (!first) sb.append(", ");
+      sb.append("keyExtent:");
+      if (this.keyExtent == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.keyExtent);
+      }
+      first = false;
+      if (!first) sb.append(", ");
+      sb.append("dir:");
+      if (this.dir == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.dir);
+      }
+      first = false;
+      if (!first) sb.append(", ");
+      sb.append("files:");
+      if (this.files == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.files);
+      }
+      first = false;
+      if (!first) sb.append(", ");
+      sb.append("setTime:");
+      sb.append(this.setTime);
+      first = false;
+      sb.append(")");
+      return sb.toString();
+    }
+
+    public void validate() throws org.apache.thrift.TException {
+      // check for required fields
+      // check for sub-struct validity
+      if (tinfo != null) {
+        tinfo.validate();
+      }
+      if (credentials != null) {
+        credentials.validate();
+      }
+      if (keyExtent != null) {
+        keyExtent.validate();
+      }
+    }
+
+    private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+      try {
+        write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+      } catch (org.apache.thrift.TException te) {
+        throw new java.io.IOException(te);
+      }
+    }
+
+    private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, java.lang.ClassNotFoundException {
+      try {
+        // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor.
+        __isset_bitfield = 0;
+        read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+      } catch (org.apache.thrift.TException te) {
+        throw new java.io.IOException(te);
+      }
+    }
+
+    private static class loadFiles_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public loadFiles_argsStandardScheme getScheme() {
+        return new loadFiles_argsStandardScheme();
+      }
+    }
+
+    private static class loadFiles_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme<loadFiles_args> {
+
+      public void read(org.apache.thrift.protocol.TProtocol iprot, loadFiles_args struct) throws org.apache.thrift.TException {
+        org.apache.thrift.protocol.TField schemeField;
+        iprot.readStructBegin();
+        while (true)
+        {
+          schemeField = iprot.readFieldBegin();
+          if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+            break;
+          }
+          switch (schemeField.id) {
+            case 1: // TINFO
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+                struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+                struct.tinfo.read(iprot);
+                struct.setTinfoIsSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            case 2: // CREDENTIALS
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+                struct.credentials = new org.apache.accumulo.core.security.thrift.TCredentials();
+                struct.credentials.read(iprot);
+                struct.setCredentialsIsSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            case 3: // TID
+              if (schemeField.type == org.apache.thrift.protocol.TType.I64) {
+                struct.tid = iprot.readI64();
+                struct.setTidIsSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            case 4: // KEY_EXTENT
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+                struct.keyExtent = new org.apache.accumulo.core.data.thrift.TKeyExtent();
+                struct.keyExtent.read(iprot);
+                struct.setKeyExtentIsSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            case 5: // DIR
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+                struct.dir = iprot.readString();
+                struct.setDirIsSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            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)
+                  {
+                    _key301 = iprot.readString();
+                    _val302 = new org.apache.accumulo.core.data.thrift.MapFileInfo();
+                    _val302.read(iprot);
+                    struct.files.put(_key301, _val302);
+                  }
+                  iprot.readMapEnd();
+                }
+                struct.setFilesIsSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            case 7: // SET_TIME
+              if (schemeField.type == org.apache.thrift.protocol.TType.BOOL) {
+                struct.setTime = iprot.readBool();
+                struct.setSetTimeIsSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            default:
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+          }
+          iprot.readFieldEnd();
+        }
+        iprot.readStructEnd();
+
+        // check for required fields of primitive type, which can't be checked in the validate method
+        struct.validate();
+      }
+
+      public void write(org.apache.thrift.protocol.TProtocol oprot, loadFiles_args struct) throws org.apache.thrift.TException {
+        struct.validate();
+
+        oprot.writeStructBegin(STRUCT_DESC);
+        if (struct.tinfo != null) {
+          oprot.writeFieldBegin(TINFO_FIELD_DESC);
+          struct.tinfo.write(oprot);
+          oprot.writeFieldEnd();
+        }
+        if (struct.credentials != null) {
+          oprot.writeFieldBegin(CREDENTIALS_FIELD_DESC);
+          struct.credentials.write(oprot);
+          oprot.writeFieldEnd();
+        }
+        oprot.writeFieldBegin(TID_FIELD_DESC);
+        oprot.writeI64(struct.tid);
+        oprot.writeFieldEnd();
+        if (struct.keyExtent != null) {
+          oprot.writeFieldBegin(KEY_EXTENT_FIELD_DESC);
+          struct.keyExtent.write(oprot);
+          oprot.writeFieldEnd();
+        }
+        if (struct.dir != null) {
+          oprot.writeFieldBegin(DIR_FIELD_DESC);
+          oprot.writeString(struct.dir);
+          oprot.writeFieldEnd();
+        }
+        if (struct.files != null) {
+          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())
+            {
+              oprot.writeString(_iter304.getKey());
+              _iter304.getValue().write(oprot);
+            }
+            oprot.writeMapEnd();
+          }
+          oprot.writeFieldEnd();
+        }
+        oprot.writeFieldBegin(SET_TIME_FIELD_DESC);
+        oprot.writeBool(struct.setTime);
+        oprot.writeFieldEnd();
+        oprot.writeFieldStop();
+        oprot.writeStructEnd();
+      }
+
+    }
+
+    private static class loadFiles_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public loadFiles_argsTupleScheme getScheme() {
+        return new loadFiles_argsTupleScheme();
+      }
+    }
+
+    private static class loadFiles_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme<loadFiles_args> {
+
+      @Override
+      public void write(org.apache.thrift.protocol.TProtocol prot, loadFiles_args struct) throws org.apache.thrift.TException {
+        org.apache.thrift.protocol.TTupleProtocol oprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
+        java.util.BitSet optionals = new java.util.BitSet();
+        if (struct.isSetTinfo()) {
+          optionals.set(0);
+        }
+        if (struct.isSetCredentials()) {
+          optionals.set(1);
+        }
+        if (struct.isSetTid()) {
+          optionals.set(2);
+        }
+        if (struct.isSetKeyExtent()) {
+          optionals.set(3);
+        }
+        if (struct.isSetDir()) {
+          optionals.set(4);
+        }
+        if (struct.isSetFiles()) {
+          optionals.set(5);
+        }
+        if (struct.isSetSetTime()) {
+          optionals.set(6);
+        }
+        oprot.writeBitSet(optionals, 7);
+        if (struct.isSetTinfo()) {
+          struct.tinfo.write(oprot);
+        }
+        if (struct.isSetCredentials()) {
+          struct.credentials.write(oprot);
+        }
+        if (struct.isSetTid()) {
+          oprot.writeI64(struct.tid);
+        }
+        if (struct.isSetKeyExtent()) {
+          struct.keyExtent.write(oprot);
+        }
+        if (struct.isSetDir()) {
+          oprot.writeString(struct.dir);
+        }
+        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())
+            {
+              oprot.writeString(_iter305.getKey());
+              _iter305.getValue().write(oprot);
+            }
+          }
+        }
+        if (struct.isSetSetTime()) {
+          oprot.writeBool(struct.setTime);
+        }
+      }
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol prot, loadFiles_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(7);
+        if (incoming.get(0)) {
+          struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+          struct.tinfo.read(iprot);
+          struct.setTinfoIsSet(true);
+        }
+        if (incoming.get(1)) {
+          struct.credentials = new org.apache.accumulo.core.security.thrift.TCredentials();
+          struct.credentials.read(iprot);
+          struct.setCredentialsIsSet(true);
+        }
+        if (incoming.get(2)) {
+          struct.tid = iprot.readI64();
+          struct.setTidIsSet(true);
+        }
+        if (incoming.get(3)) {
+          struct.keyExtent = new org.apache.accumulo.core.data.thrift.TKeyExtent();
+          struct.keyExtent.read(iprot);
+          struct.setKeyExtentIsSet(true);
+        }
+        if (incoming.get(4)) {
+          struct.dir = iprot.readString();
+          struct.setDirIsSet(true);
+        }
+        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)
+            {
+              _key307 = iprot.readString();
+              _val308 = new org.apache.accumulo.core.data.thrift.MapFileInfo();
+              _val308.read(iprot);
+              struct.files.put(_key307, _val308);
+            }
+          }
+          struct.setFilesIsSet(true);
+        }
+        if (incoming.get(6)) {
+          struct.setTime = iprot.readBool();
+          struct.setSetTimeIsSet(true);
+        }
+      }
+    }
+
+    private static <S extends org.apache.thrift.scheme.IScheme> S scheme(org.apache.thrift.protocol.TProtocol proto) {
+      return (org.apache.thrift.scheme.StandardScheme.class.equals(proto.getScheme()) ? STANDARD_SCHEME_FACTORY : TUPLE_SCHEME_FACTORY).getScheme();
+    }
+  }
+
   public static class splitTablet_args implements org.apache.thrift.TBase<splitTablet_args, splitTablet_args._Fields>, java.io.Serializable, Cloneable, Comparable<splitTablet_args>   {
     private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("splitTablet_args");
 
@@ -31602,14 +32783,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 _list300 = iprot.readListBegin();
-                  struct.success = new java.util.ArrayList<TabletStats>(_list300.size);
-                  TabletStats _elem301;
-                  for (int _i302 = 0; _i302 < _list300.size; ++_i302)
+                  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)
                   {
-                    _elem301 = new TabletStats();
-                    _elem301.read(iprot);
-                    struct.success.add(_elem301);
+                    _elem311 = new TabletStats();
+                    _elem311.read(iprot);
+                    struct.success.add(_elem311);
                   }
                   iprot.readListEnd();
                 }
@@ -31646,9 +32827,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 _iter303 : struct.success)
+            for (TabletStats _iter313 : struct.success)
             {
-              _iter303.write(oprot);
+              _iter313.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -31687,9 +32868,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, getTabletStats_resu
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (TabletStats _iter304 : struct.success)
+            for (TabletStats _iter314 : struct.success)
             {
-              _iter304.write(oprot);
+              _iter314.write(oprot);
             }
           }
         }
@@ -31704,14 +32885,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 _list305 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.success = new java.util.ArrayList<TabletStats>(_list305.size);
-            TabletStats _elem306;
-            for (int _i307 = 0; _i307 < _list305.size; ++_i307)
+            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)
             {
-              _elem306 = new TabletStats();
-              _elem306.read(iprot);
-              struct.success.add(_elem306);
+              _elem316 = new TabletStats();
+              _elem316.read(iprot);
+              struct.success.add(_elem316);
             }
           }
           struct.setSuccessIsSet(true);
@@ -35073,14 +36254,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 _list308 = iprot.readListBegin();
-                  struct.success = new java.util.ArrayList<ActiveScan>(_list308.size);
-                  ActiveScan _elem309;
-                  for (int _i310 = 0; _i310 < _list308.size; ++_i310)
+                  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)
                   {
-                    _elem309 = new ActiveScan();
-                    _elem309.read(iprot);
-                    struct.success.add(_elem309);
+                    _elem319 = new ActiveScan();
+                    _elem319.read(iprot);
+                    struct.success.add(_elem319);
                   }
                   iprot.readListEnd();
                 }
@@ -35117,9 +36298,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 _iter311 : struct.success)
+            for (ActiveScan _iter321 : struct.success)
             {
-              _iter311.write(oprot);
+              _iter321.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -35158,9 +36339,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, getActiveScans_resu
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (ActiveScan _iter312 : struct.success)
+            for (ActiveScan _iter322 : struct.success)
             {
-              _iter312.write(oprot);
+              _iter322.write(oprot);
             }
           }
         }
@@ -35175,14 +36356,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 _list313 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.success = new java.util.ArrayList<ActiveScan>(_list313.size);
-            ActiveScan _elem314;
-            for (int _i315 = 0; _i315 < _list313.size; ++_i315)
+            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)
             {
-              _elem314 = new ActiveScan();
-              _elem314.read(iprot);
-              struct.success.add(_elem314);
+              _elem324 = new ActiveScan();
+              _elem324.read(iprot);
+              struct.success.add(_elem324);
             }
           }
           struct.setSuccessIsSet(true);
@@ -36071,14 +37252,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 _list316 = iprot.readListBegin();
-                  struct.success = new java.util.ArrayList<ActiveCompaction>(_list316.size);
-                  ActiveCompaction _elem317;
-                  for (int _i318 = 0; _i318 < _list316.size; ++_i318)
+                  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)
                   {
-                    _elem317 = new ActiveCompaction();
-                    _elem317.read(iprot);
-                    struct.success.add(_elem317);
+                    _elem327 = new ActiveCompaction();
+                    _elem327.read(iprot);
+                    struct.success.add(_elem327);
                   }
                   iprot.readListEnd();
                 }
@@ -36115,9 +37296,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 _iter319 : struct.success)
+            for (ActiveCompaction _iter329 : struct.success)
             {
-              _iter319.write(oprot);
+              _iter329.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -36156,9 +37337,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, getActiveCompaction
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (ActiveCompaction _iter320 : struct.success)
+            for (ActiveCompaction _iter330 : struct.success)
             {
-              _iter320.write(oprot);
+              _iter330.write(oprot);
             }
           }
         }
@@ -36173,14 +37354,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 _list321 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.success = new java.util.ArrayList<ActiveCompaction>(_list321.size);
-            ActiveCompaction _elem322;
-            for (int _i323 = 0; _i323 < _list321.size; ++_i323)
+            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)
             {
-              _elem322 = new ActiveCompaction();
-              _elem322.read(iprot);
-              struct.success.add(_elem322);
+              _elem332 = new ActiveCompaction();
+              _elem332.read(iprot);
+              struct.success.add(_elem332);
             }
           }
           struct.setSuccessIsSet(true);
@@ -36695,13 +37876,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 _list324 = iprot.readListBegin();
-                  struct.filenames = new java.util.ArrayList<java.lang.String>(_list324.size);
-                  java.lang.String _elem325;
-                  for (int _i326 = 0; _i326 < _list324.size; ++_i326)
+                  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)
                   {
-                    _elem325 = iprot.readString();
-                    struct.filenames.add(_elem325);
+                    _elem335 = iprot.readString();
+                    struct.filenames.add(_elem335);
                   }
                   iprot.readListEnd();
                 }
@@ -36739,9 +37920,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 _iter327 : struct.filenames)
+            for (java.lang.String _iter337 : struct.filenames)
             {
-              oprot.writeString(_iter327);
+              oprot.writeString(_iter337);
             }
             oprot.writeListEnd();
           }
@@ -36784,9 +37965,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 _iter328 : struct.filenames)
+            for (java.lang.String _iter338 : struct.filenames)
             {
-              oprot.writeString(_iter328);
+              oprot.writeString(_iter338);
             }
           }
         }
@@ -36808,13 +37989,13 @@ public void read(org.apache.thrift.protocol.TProtocol prot, removeLogs_args stru
         }
         if (incoming.get(2)) {
           {
-            org.apache.thrift.protocol.TList _list329 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.filenames = new java.util.ArrayList<java.lang.String>(_list329.size);
-            java.lang.String _elem330;
-            for (int _i331 = 0; _i331 < _list329.size; ++_i331)
+            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)
             {
-              _elem330 = iprot.readString();
-              struct.filenames.add(_elem330);
+              _elem340 = iprot.readString();
+              struct.filenames.add(_elem340);
             }
           }
           struct.setFilenamesIsSet(true);
@@ -37614,13 +38795,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 _list332 = iprot.readListBegin();
-                  struct.success = new java.util.ArrayList<java.lang.String>(_list332.size);
-                  java.lang.String _elem333;
-                  for (int _i334 = 0; _i334 < _list332.size; ++_i334)
+                  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)
                   {
-                    _elem333 = iprot.readString();
-                    struct.success.add(_elem333);
+                    _elem343 = iprot.readString();
+                    struct.success.add(_elem343);
                   }
                   iprot.readListEnd();
                 }
@@ -37648,9 +38829,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 _iter335 : struct.success)
+            for (java.lang.String _iter345 : struct.success)
             {
-              oprot.writeString(_iter335);
+              oprot.writeString(_iter345);
             }
             oprot.writeListEnd();
           }
@@ -37681,9 +38862,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, getActiveLogs_resul
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (java.lang.String _iter336 : struct.success)
+            for (java.lang.String _iter346 : struct.success)
             {
-              oprot.writeString(_iter336);
+              oprot.writeString(_iter346);
             }
           }
         }
@@ -37695,13 +38876,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 _list337 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.success = new java.util.ArrayList<java.lang.String>(_list337.size);
-            java.lang.String _elem338;
-            for (int _i339 = 0; _i339 < _list337.size; ++_i339)
+            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)
             {
-              _elem338 = iprot.readString();
-              struct.success.add(_elem338);
+              _elem348 = iprot.readString();
+              struct.success.add(_elem348);
             }
           }
           struct.setSuccessIsSet(true);
@@ -40734,26 +41915,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 _map340 = iprot.readMapBegin();
-                  struct.files = new java.util.HashMap<java.lang.String,java.util.List<org.apache.accumulo.core.data.thrift.TRowRange>>(2*_map340.size);
-                  java.lang.String _key341;
-                  java.util.List<org.apache.accumulo.core.data.thrift.TRowRange> _val342;
-                  for (int _i343 = 0; _i343 < _map340.size; ++_i343)
+                  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)
                   {
-                    _key341 = iprot.readString();
+                    _key351 = iprot.readString();
                     {
-                      org.apache.thrift.protocol.TList _list344 = iprot.readListBegin();
-                      _val342 = new java.util.ArrayList<org.apache.accumulo.core.data.thrift.TRowRange>(_list344.size);
-                      org.apache.accumulo.core.data.thrift.TRowRange _elem345;
-                      for (int _i346 = 0; _i346 < _list344.size; ++_i346)
+                      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)
                       {
-                        _elem345 = new org.apache.accumulo.core.data.thrift.TRowRange();
-                        _elem345.read(iprot);
-                        _val342.add(_elem345);
+                        _elem355 = new org.apache.accumulo.core.data.thrift.TRowRange();
+                        _elem355.read(iprot);
+                        _val352.add(_elem355);
                       }
                       iprot.readListEnd();
                     }
-                    struct.files.put(_key341, _val342);
+                    struct.files.put(_key351, _val352);
                   }
                   iprot.readMapEnd();
                 }
@@ -40796,14 +41977,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>> _iter347 : struct.files.entrySet())
+            for (java.util.Map.Entry<java.lang.String, java.util.List<org.apache.accumulo.core.data.thrift.TRowRange>> _iter357 : struct.files.entrySet())
             {
-              oprot.writeString(_iter347.getKey());
+              oprot.writeString(_iter357.getKey());
               {
-                oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, _iter347.getValue().size()));
-                for (org.apache.accumulo.core.data.thrift.TRowRange _iter348 : _iter347.getValue())
+                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())
                 {
-                  _iter348.write(oprot);
+                  _iter358.write(oprot);
                 }
                 oprot.writeListEnd();
               }
@@ -40855,14 +42036,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>> _iter349 : struct.files.entrySet())
+            for (java.util.Map.Entry<java.lang.String, java.util.List<org.apache.accumulo.core.data.thrift.TRowRange>> _iter359 : struct.files.entrySet())
             {
-              oprot.writeString(_iter349.getKey());
+              oprot.writeString(_iter359.getKey());
               {
-                oprot.writeI32(_iter349.getValue().size());
-                for (org.apache.accumulo.core.data.thrift.TRowRange _iter350 : _iter349.getValue())
+                oprot.writeI32(_iter359.getValue().size());
+                for (org.apache.accumulo.core.data.thrift.TRowRange _iter360 : _iter359.getValue())
                 {
-                  _iter350.write(oprot);
+                  _iter360.write(oprot);
                 }
               }
             }
@@ -40891,25 +42072,25 @@ public void read(org.apache.thrift.protocol.TProtocol prot, startGetSummariesFro
         }
         if (incoming.get(3)) {
           {
-            org.apache.thrift.protocol.TMap _map351 = 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*_map351.size);
-            java.lang.String _key352;
-            java.util.List<org.apache.accumulo.core.data.thrift.TRowRange> _val353;
-            for (int _i354 = 0; _i354 < _map351.size; ++_i354)
+            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)
             {
-              _key352 = iprot.readString();
+              _key362 = iprot.readString();
               {
-                org.apache.thrift.protocol.TList _list355 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-                _val353 = new java.util.ArrayList<org.apache.accumulo.core.data.thrift.TRowRange>(_list355.size);
-                org.apache.accumulo.core.data.thrift.TRowRange _elem356;
-                for (int _i357 = 0; _i357 < _list355.size; ++_i357)
+                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)
                 {
-                  _elem356 = new org.apache.accumulo.core.data.thrift.TRowRange();
-                  _elem356.read(iprot);
-                  _val353.add(_elem356);
+                  _elem366 = new org.apache.accumulo.core.data.thrift.TRowRange();
+                  _elem366.read(iprot);
+                  _val363.add(_elem366);
                 }
               }
-              struct.files.put(_key352, _val353);
+              struct.files.put(_key362, _val363);
             }
           }
           struct.setFilesIsSet(true);
diff --git a/core/src/main/thrift/client.thrift b/core/src/main/thrift/client.thrift
index 4ab3e48473..37c2195da5 100644
--- a/core/src/main/thrift/client.thrift
+++ b/core/src/main/thrift/client.thrift
@@ -51,6 +51,7 @@ enum TableOperationExceptionType {
   NAMESPACE_EXISTS
   NAMESPACE_NOTFOUND
   INVALID_NAME
+  BULK_BAD_LOAD_MAPPING
 }
 
 enum ConfigurationType {
@@ -110,7 +111,9 @@ service ClientService {
   string getInstanceId()
   string getZooKeepers()
 
+  // deprecated for new bulkImport
   list<string> bulkImportFiles(1:trace.TInfo tinfo, 8:security.TCredentials credentials, 3:i64 tid, 4:string tableId, 5:list<string> files, 6:string errorDir, 7:bool setTime) throws (1:ThriftSecurityException sec, 2:ThriftTableOperationException tope);
+
   // ensures that nobody is working on the transaction id above
   bool isActive(1:trace.TInfo tinfo, 2:i64 tid)
 
diff --git a/core/src/main/thrift/master.thrift b/core/src/main/thrift/master.thrift
index 2117c34bbf..761ef1e1eb 100644
--- a/core/src/main/thrift/master.thrift
+++ b/core/src/main/thrift/master.thrift
@@ -158,6 +158,7 @@ enum FateOperation {
   NAMESPACE_CREATE
   NAMESPACE_DELETE
   NAMESPACE_RENAME
+  TABLE_BULK_IMPORT2
 }
 
 service FateService {
diff --git a/core/src/main/thrift/tabletserver.thrift b/core/src/main/thrift/tabletserver.thrift
index b56449ff7b..2b7372f01b 100644
--- a/core/src/main/thrift/tabletserver.thrift
+++ b/core/src/main/thrift/tabletserver.thrift
@@ -211,6 +211,8 @@ service TabletClientService extends client.ClientService {
   // on success, returns an empty list
   list<data.TKeyExtent> bulkImport(3:trace.TInfo tinfo, 1:security.TCredentials credentials, 4:i64 tid, 2:data.TabletFiles files, 5:bool setTime) throws (1:client.ThriftSecurityException sec),
 
+  oneway void loadFiles(1:trace.TInfo tinfo, 2:security.TCredentials credentials, 3:i64 tid, 4:data.TKeyExtent keyExtent, 5:string dir, 6:map<string, data.MapFileInfo> files, 7:bool setTime);
+
   void splitTablet(4:trace.TInfo tinfo, 1:security.TCredentials credentials, 2:data.TKeyExtent extent, 3:binary splitPoint) throws (1:client.ThriftSecurityException sec, 2:NotServingTabletException nste)
  
   oneway void loadTablet(5:trace.TInfo tinfo, 1:security.TCredentials credentials, 4:string lock, 2:data.TKeyExtent extent),
diff --git a/core/src/test/java/org/apache/accumulo/core/client/impl/BulkSerializeTest.java b/core/src/test/java/org/apache/accumulo/core/client/impl/BulkSerializeTest.java
new file mode 100644
index 0000000000..34f453dbcc
--- /dev/null
+++ b/core/src/test/java/org/apache/accumulo/core/client/impl/BulkSerializeTest.java
@@ -0,0 +1,149 @@
+/*
+ * 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.client.impl;
+
+import static org.junit.Assert.assertEquals;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.SortedMap;
+import java.util.TreeMap;
+
+import org.apache.accumulo.core.Constants;
+import org.apache.accumulo.core.client.impl.Bulk.FileInfo;
+import org.apache.accumulo.core.client.impl.Bulk.Files;
+import org.apache.accumulo.core.client.impl.BulkSerialize.Input;
+import org.apache.accumulo.core.client.impl.BulkSerialize.LoadMappingIterator;
+import org.apache.accumulo.core.client.impl.Table.ID;
+import org.apache.accumulo.core.data.impl.KeyExtent;
+import org.apache.hadoop.io.Text;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class BulkSerializeTest {
+
+  @Test
+  public void writeReadLoadMapping() throws Exception {
+    Table.ID tableId = Table.ID.of("3");
+    SortedMap<KeyExtent,Bulk.Files> mapping = generateMapping(tableId);
+
+    ByteArrayOutputStream baos = new ByteArrayOutputStream();
+
+    BulkSerialize.writeLoadMapping(mapping, "/some/dir", p -> baos);
+
+    ByteArrayInputStream bais = new ByteArrayInputStream(baos.toByteArray());
+
+    try (LoadMappingIterator lmi = BulkSerialize.readLoadMapping("/some/dir", tableId, p -> bais)) {
+      SortedMap<KeyExtent,Bulk.Files> readMapping = new TreeMap<>();
+      lmi.forEachRemaining(e -> readMapping.put(e.getKey(), e.getValue()));
+      Assert.assertEquals(mapping, readMapping);
+    }
+  }
+
+  @Test
+  public void writeReadRenames() throws Exception {
+
+    Map<String,String> renames = new HashMap<>();
+    for (String f : "f1 f2 f3 f4 f5".split(" "))
+      renames.put("old_" + f + ".rf", "new_" + f + ".rf");
+
+    ByteArrayOutputStream baos = new ByteArrayOutputStream();
+
+    BulkSerialize.writeRenameMap(renames, "/some/dir", p -> baos);
+
+    ByteArrayInputStream bais = new ByteArrayInputStream(baos.toByteArray());
+
+    Map<String,String> readMap = BulkSerialize.readRenameMap("/some/dir", p -> bais);
+
+    assertEquals("Read renames file wrong size", renames.size(), readMap.size());
+    assertEquals("Read renames file different from what was written.", renames, readMap);
+  }
+
+  @Test
+  public void testRemap() throws Exception {
+    Table.ID tableId = Table.ID.of("3");
+    SortedMap<KeyExtent,Bulk.Files> mapping = generateMapping(tableId);
+
+    SortedMap<KeyExtent,Bulk.Files> newNameMapping = new TreeMap<>();
+
+    Map<String,String> nameMap = new HashMap<>();
+
+    mapping.forEach((extent, files) -> {
+      Files newFiles = new Files();
+      files.forEach(fi -> {
+        newFiles.add(new FileInfo("N" + fi.name, fi.estSize, fi.estEntries));
+        nameMap.put(fi.name, "N" + fi.name);
+      });
+
+      newNameMapping.put(extent, newFiles);
+    });
+
+    ByteArrayOutputStream mappingBaos = new ByteArrayOutputStream();
+    ByteArrayOutputStream nameBaos = new ByteArrayOutputStream();
+
+    BulkSerialize.writeRenameMap(nameMap, "/some/dir", p -> nameBaos);
+    BulkSerialize.writeLoadMapping(mapping, "/some/dir", p -> mappingBaos);
+
+    Input input = p -> {
+      if (p.getName().equals(Constants.BULK_LOAD_MAPPING)) {
+        return new ByteArrayInputStream(mappingBaos.toByteArray());
+      } else if (p.getName().equals(Constants.BULK_RENAME_FILE)) {
+        return new ByteArrayInputStream(nameBaos.toByteArray());
+      } else {
+        throw new IllegalArgumentException("bad path " + p);
+      }
+    };
+
+    try (LoadMappingIterator lmi = BulkSerialize.getUpdatedLoadMapping("/some/dir", tableId,
+        input)) {
+      SortedMap<KeyExtent,Bulk.Files> actual = new TreeMap<>();
+      lmi.forEachRemaining(e -> actual.put(e.getKey(), e.getValue()));
+      Assert.assertEquals(newNameMapping, actual);
+    }
+
+  }
+
+  public SortedMap<KeyExtent,Bulk.Files> generateMapping(ID tableId) {
+    SortedMap<KeyExtent,Bulk.Files> mapping = new TreeMap<>();
+    Bulk.Files testFiles = new Bulk.Files();
+    Bulk.Files testFiles2 = new Bulk.Files();
+    Bulk.Files testFiles3 = new Bulk.Files();
+    long c = 0L;
+    for (String f : "f1 f2 f3".split(" ")) {
+      c++;
+      testFiles.add(new Bulk.FileInfo(f, c, c));
+    }
+    c = 0L;
+    for (String f : "g1 g2 g3".split(" ")) {
+      c++;
+      testFiles2.add(new Bulk.FileInfo(f, c, c));
+    }
+    for (String f : "h1 h2 h3".split(" ")) {
+      c++;
+      testFiles3.add(new Bulk.FileInfo(f, c, c));
+    }
+
+    // add out of order to test sorting
+    mapping.put(new KeyExtent(tableId, new Text("d"), new Text("c")), testFiles);
+    mapping.put(new KeyExtent(tableId, new Text("c"), new Text("b")), testFiles2);
+    mapping.put(new KeyExtent(tableId, new Text("b"), new Text("a")), testFiles3);
+
+    return mapping;
+  }
+}
diff --git a/fate/src/main/java/org/apache/accumulo/fate/Fate.java b/fate/src/main/java/org/apache/accumulo/fate/Fate.java
index 4af32e5b47..28e0ebdbce 100644
--- a/fate/src/main/java/org/apache/accumulo/fate/Fate.java
+++ b/fate/src/main/java/org/apache/accumulo/fate/Fate.java
@@ -70,6 +70,7 @@ public void run() {
             Repo<T> prevOp = null;
             try {
               deferTime = op.isReady(tid, environment);
+
               if (deferTime == 0) {
                 prevOp = op;
                 op = op.call(tid, environment);
diff --git a/server/base/src/main/java/org/apache/accumulo/server/client/ClientServiceHandler.java b/server/base/src/main/java/org/apache/accumulo/server/client/ClientServiceHandler.java
index e3adec6d9f..70707cf479 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/client/ClientServiceHandler.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/client/ClientServiceHandler.java
@@ -61,6 +61,7 @@
 import org.apache.accumulo.core.security.thrift.TCredentials;
 import org.apache.accumulo.core.trace.thrift.TInfo;
 import org.apache.accumulo.server.AccumuloServerContext;
+import org.apache.accumulo.server.client.deprecated.BulkImporter;
 import org.apache.accumulo.server.conf.ServerConfigurationFactory;
 import org.apache.accumulo.server.fs.VolumeManager;
 import org.apache.accumulo.server.security.AuditedSecurityOperation;
diff --git a/server/base/src/main/java/org/apache/accumulo/server/client/BulkImporter.java b/server/base/src/main/java/org/apache/accumulo/server/client/deprecated/BulkImporter.java
similarity index 99%
rename from server/base/src/main/java/org/apache/accumulo/server/client/BulkImporter.java
rename to server/base/src/main/java/org/apache/accumulo/server/client/deprecated/BulkImporter.java
index 93646c47d1..37cb82fd23 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/client/BulkImporter.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/client/deprecated/BulkImporter.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.accumulo.server.client;
+package org.apache.accumulo.server.client.deprecated;
 
 import static org.apache.accumulo.fate.util.UtilWaitThread.sleepUninterruptibly;
 
@@ -75,10 +75,12 @@
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+@Deprecated
 public class BulkImporter {
 
   private static final Logger log = LoggerFactory.getLogger(BulkImporter.class);
 
+  @Deprecated
   public static List<String> bulkLoad(ClientContext context, long tid, String tableId,
       List<String> files, String errorDir, boolean setTime) throws IOException, AccumuloException,
       AccumuloSecurityException, ThriftTableOperationException {
diff --git a/server/base/src/main/java/org/apache/accumulo/server/util/FileUtil.java b/server/base/src/main/java/org/apache/accumulo/server/util/FileUtil.java
index 9be303be0f..bf704f40b3 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/util/FileUtil.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/util/FileUtil.java
@@ -23,12 +23,12 @@
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
-import java.util.Map.Entry;
 import java.util.Random;
 import java.util.Set;
 import java.util.SortedMap;
 import java.util.TreeMap;
 
+import org.apache.accumulo.core.client.impl.BulkImport;
 import org.apache.accumulo.core.conf.AccumuloConfiguration;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.data.Key;
@@ -543,59 +543,12 @@ else if (prevEndRow == null || key.compareRow(prevEndRow) > 0)
 
   }
 
-  private static class MLong {
-    public MLong(long i) {
-      l = i;
-    }
-
-    long l;
-  }
-
   public static Map<KeyExtent,Long> estimateSizes(AccumuloConfiguration acuConf, Path mapFile,
       long fileSize, List<KeyExtent> extents, Configuration conf, VolumeManager fs)
       throws IOException {
 
-    long totalIndexEntries = 0;
-    Map<KeyExtent,MLong> counts = new TreeMap<>();
-    for (KeyExtent keyExtent : extents)
-      counts.put(keyExtent, new MLong(0));
-
-    Text row = new Text();
     FileSystem ns = fs.getVolumeByPath(mapFile).getFileSystem();
-    FileSKVIterator index = FileOperations.getInstance().newIndexReaderBuilder()
-        .forFile(mapFile.toString(), ns, ns.getConf()).withTableConfiguration(acuConf).build();
-
-    try {
-      while (index.hasTop()) {
-        Key key = index.getTopKey();
-        totalIndexEntries++;
-        key.getRow(row);
-
-        for (Entry<KeyExtent,MLong> entry : counts.entrySet())
-          if (entry.getKey().contains(row))
-            entry.getValue().l++;
-
-        index.next();
-      }
-    } finally {
-      try {
-        if (index != null)
-          index.close();
-      } catch (IOException e) {
-        // continue with next file
-        log.error("{}", e.getMessage(), e);
-      }
-    }
-
-    Map<KeyExtent,Long> results = new TreeMap<>();
-    for (KeyExtent keyExtent : extents) {
-      double numEntries = counts.get(keyExtent).l;
-      if (numEntries == 0)
-        numEntries = 1;
-      long estSize = (long) ((numEntries / totalIndexEntries) * fileSize);
-      results.put(keyExtent, estSize);
-    }
-    return results;
+    return BulkImport.estimateSizes(acuConf, mapFile, fileSize, extents, ns);
   }
 
   public static Collection<String> toPathStrings(Collection<FileRef> refs) {
diff --git a/server/base/src/test/java/org/apache/accumulo/server/client/BulkImporterTest.java b/server/base/src/test/java/org/apache/accumulo/server/client/deprecated/BulkImporterTest.java
similarity index 98%
rename from server/base/src/test/java/org/apache/accumulo/server/client/BulkImporterTest.java
rename to server/base/src/test/java/org/apache/accumulo/server/client/deprecated/BulkImporterTest.java
index a525e57028..df9e1f6a1c 100644
--- a/server/base/src/test/java/org/apache/accumulo/server/client/BulkImporterTest.java
+++ b/server/base/src/test/java/org/apache/accumulo/server/client/deprecated/BulkImporterTest.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.accumulo.server.client;
+package org.apache.accumulo.server.client.deprecated;
 
 import java.util.Collection;
 import java.util.Collections;
@@ -40,6 +40,7 @@
 import org.apache.accumulo.core.file.FileOperations;
 import org.apache.accumulo.core.file.FileSKVWriter;
 import org.apache.accumulo.core.util.CachedConfiguration;
+import org.apache.accumulo.server.client.deprecated.BulkImporter;
 import org.apache.accumulo.server.fs.VolumeManager;
 import org.apache.accumulo.server.fs.VolumeManagerImpl;
 import org.apache.commons.lang.NotImplementedException;
diff --git a/server/master/src/main/java/org/apache/accumulo/master/FateServiceHandler.java b/server/master/src/main/java/org/apache/accumulo/master/FateServiceHandler.java
index 2c64242953..0777e13075 100644
--- a/server/master/src/main/java/org/apache/accumulo/master/FateServiceHandler.java
+++ b/server/master/src/main/java/org/apache/accumulo/master/FateServiceHandler.java
@@ -56,7 +56,6 @@
 import org.apache.accumulo.core.util.ByteBufferUtil;
 import org.apache.accumulo.core.util.Validator;
 import org.apache.accumulo.fate.ReadOnlyTStore.TStatus;
-import org.apache.accumulo.master.tableOps.BulkImport;
 import org.apache.accumulo.master.tableOps.CancelCompactions;
 import org.apache.accumulo.master.tableOps.ChangeTableState;
 import org.apache.accumulo.master.tableOps.CloneTable;
@@ -71,6 +70,7 @@
 import org.apache.accumulo.master.tableOps.RenameTable;
 import org.apache.accumulo.master.tableOps.TableRangeOp;
 import org.apache.accumulo.master.tableOps.TraceRepo;
+import org.apache.accumulo.master.tableOps.bulkVer2.PrepBulkImport;
 import org.apache.accumulo.server.client.ClientServiceHandler;
 import org.apache.accumulo.server.master.state.MergeInfo;
 import org.apache.accumulo.server.util.TablePropUtil;
@@ -412,7 +412,9 @@ public String invalidMessage(String argument) {
 
         master.updateBulkImportStatus(dir, BulkImportState.INITIAL);
         master.fate.seedTransaction(opid,
-            new TraceRepo<>(new BulkImport(tableId, dir, failDir, setTime)), autoCleanup);
+            new TraceRepo<>(new org.apache.accumulo.master.tableOps.bulkVer1.BulkImport(tableId,
+                dir, failDir, setTime)),
+            autoCleanup);
         break;
       }
       case TABLE_COMPACT: {
@@ -515,6 +517,30 @@ public String invalidMessage(String argument) {
             autoCleanup);
         break;
       }
+      case TABLE_BULK_IMPORT2: {
+        TableOperation tableOp = TableOperation.BULK_IMPORT;
+        Table.ID tableId = validateTableIdArgument(arguments.get(0), tableOp, NOT_ROOT_ID);
+        String dir = ByteBufferUtil.toString(arguments.get(1));
+
+        boolean setTime = Boolean.parseBoolean(ByteBufferUtil.toString(arguments.get(2)));
+
+        Namespace.ID namespaceId = getNamespaceIdFromTableId(tableOp, tableId);
+
+        final boolean canBulkImport;
+        try {
+          canBulkImport = master.security.canBulkImport(c, tableId, namespaceId);
+        } catch (ThriftSecurityException e) {
+          throwIfTableMissingSecurityException(e, tableId, "", TableOperation.BULK_IMPORT);
+          throw e;
+        }
+
+        if (!canBulkImport)
+          throw new ThriftSecurityException(c.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED);
+
+        master.fate.seedTransaction(opid,
+            new TraceRepo<>(new PrepBulkImport(tableId, dir, setTime)), autoCleanup);
+        break;
+      }
       default:
         throw new UnsupportedOperationException();
     }
diff --git a/server/master/src/main/java/org/apache/accumulo/master/tableOps/BulkImport.java b/server/master/src/main/java/org/apache/accumulo/master/tableOps/bulkVer1/BulkImport.java
similarity index 98%
rename from server/master/src/main/java/org/apache/accumulo/master/tableOps/BulkImport.java
rename to server/master/src/main/java/org/apache/accumulo/master/tableOps/bulkVer1/BulkImport.java
index 325211b79f..9b26786692 100644
--- a/server/master/src/main/java/org/apache/accumulo/master/tableOps/BulkImport.java
+++ b/server/master/src/main/java/org/apache/accumulo/master/tableOps/bulkVer1/BulkImport.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.accumulo.master.tableOps;
+package org.apache.accumulo.master.tableOps.bulkVer1;
 
 import static org.apache.accumulo.fate.util.UtilWaitThread.sleepUninterruptibly;
 
@@ -39,6 +39,8 @@
 import org.apache.accumulo.core.util.SimpleThreadPool;
 import org.apache.accumulo.fate.Repo;
 import org.apache.accumulo.master.Master;
+import org.apache.accumulo.master.tableOps.MasterRepo;
+import org.apache.accumulo.master.tableOps.Utils;
 import org.apache.accumulo.server.ServerConstants;
 import org.apache.accumulo.server.fs.VolumeManager;
 import org.apache.accumulo.server.tablets.UniqueNameAllocator;
@@ -69,7 +71,6 @@
  * server checks the Arbitrator to see if the request is still valid.
  *
  */
-
 public class BulkImport extends MasterRepo {
   public static final String FAILURES_TXT = "failures.txt";
 
diff --git a/server/master/src/main/java/org/apache/accumulo/master/tableOps/CleanUpBulkImport.java b/server/master/src/main/java/org/apache/accumulo/master/tableOps/bulkVer1/CleanUpBulkImport.java
similarity index 85%
rename from server/master/src/main/java/org/apache/accumulo/master/tableOps/CleanUpBulkImport.java
rename to server/master/src/main/java/org/apache/accumulo/master/tableOps/bulkVer1/CleanUpBulkImport.java
index a7dc5cf5ed..4dd6c6d35e 100644
--- a/server/master/src/main/java/org/apache/accumulo/master/tableOps/CleanUpBulkImport.java
+++ b/server/master/src/main/java/org/apache/accumulo/master/tableOps/bulkVer1/CleanUpBulkImport.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.accumulo.master.tableOps;
+package org.apache.accumulo.master.tableOps.bulkVer1;
 
 import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.client.Connector;
@@ -22,13 +22,15 @@
 import org.apache.accumulo.core.master.thrift.BulkImportState;
 import org.apache.accumulo.fate.Repo;
 import org.apache.accumulo.master.Master;
+import org.apache.accumulo.master.tableOps.MasterRepo;
+import org.apache.accumulo.master.tableOps.Utils;
 import org.apache.accumulo.server.util.MetadataTableUtil;
 import org.apache.accumulo.server.zookeeper.TransactionWatcher.ZooArbitrator;
 import org.apache.hadoop.fs.Path;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-class CleanUpBulkImport extends MasterRepo {
+public class CleanUpBulkImport extends MasterRepo {
 
   private static final long serialVersionUID = 1L;
 
@@ -39,7 +41,7 @@
   private String bulk;
   private String error;
 
-  public CleanUpBulkImport(Table.ID tableId, String source, String bulk, String error) {
+  CleanUpBulkImport(Table.ID tableId, String source, String bulk, String error) {
     this.tableId = tableId;
     this.source = source;
     this.bulk = bulk;
@@ -49,7 +51,7 @@ public CleanUpBulkImport(Table.ID tableId, String source, String bulk, String er
   @Override
   public Repo<Master> call(long tid, Master master) throws Exception {
     master.updateBulkImportStatus(source, BulkImportState.CLEANUP);
-    log.debug("removing the bulk processing flag file in " + bulk);
+    log.debug("removing the bulkDir processing flag file in " + bulk);
     Path bulkDir = new Path(bulk);
     MetadataTableUtil.removeBulkLoadInProgressFlag(master,
         "/" + bulkDir.getParent().getName() + "/" + bulkDir.getName());
@@ -61,7 +63,7 @@ public CleanUpBulkImport(Table.ID tableId, String source, String bulk, String er
     Utils.unreserveHdfsDirectory(source, tid);
     Utils.unreserveHdfsDirectory(error, tid);
     Utils.getReadLock(tableId, tid).unlock();
-    log.debug("completing bulk import transaction " + tid);
+    log.debug("completing bulkDir import transaction " + tid);
     ZooArbitrator.cleanup(Constants.BULK_ARBITRATOR_TYPE, tid);
     master.removeBulkImportStatus(source);
     return null;
diff --git a/server/master/src/main/java/org/apache/accumulo/master/tableOps/CompleteBulkImport.java b/server/master/src/main/java/org/apache/accumulo/master/tableOps/bulkVer1/CompleteBulkImport.java
similarity index 90%
rename from server/master/src/main/java/org/apache/accumulo/master/tableOps/CompleteBulkImport.java
rename to server/master/src/main/java/org/apache/accumulo/master/tableOps/bulkVer1/CompleteBulkImport.java
index bcfadd5902..5c85f7af03 100644
--- a/server/master/src/main/java/org/apache/accumulo/master/tableOps/CompleteBulkImport.java
+++ b/server/master/src/main/java/org/apache/accumulo/master/tableOps/bulkVer1/CompleteBulkImport.java
@@ -14,15 +14,16 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.accumulo.master.tableOps;
+package org.apache.accumulo.master.tableOps.bulkVer1;
 
 import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.client.impl.Table;
 import org.apache.accumulo.fate.Repo;
 import org.apache.accumulo.master.Master;
+import org.apache.accumulo.master.tableOps.MasterRepo;
 import org.apache.accumulo.server.zookeeper.TransactionWatcher.ZooArbitrator;
 
-class CompleteBulkImport extends MasterRepo {
+public class CompleteBulkImport extends MasterRepo {
 
   private static final long serialVersionUID = 1L;
 
diff --git a/server/master/src/main/java/org/apache/accumulo/master/tableOps/CopyFailed.java b/server/master/src/main/java/org/apache/accumulo/master/tableOps/bulkVer1/CopyFailed.java
similarity index 98%
rename from server/master/src/main/java/org/apache/accumulo/master/tableOps/CopyFailed.java
rename to server/master/src/main/java/org/apache/accumulo/master/tableOps/bulkVer1/CopyFailed.java
index 311a2abd0c..356e3d698b 100644
--- a/server/master/src/main/java/org/apache/accumulo/master/tableOps/CopyFailed.java
+++ b/server/master/src/main/java/org/apache/accumulo/master/tableOps/bulkVer1/CopyFailed.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.accumulo.master.tableOps;
+package org.apache.accumulo.master.tableOps.bulkVer1;
 
 import static java.nio.charset.StandardCharsets.UTF_8;
 
@@ -39,6 +39,7 @@
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.fate.Repo;
 import org.apache.accumulo.master.Master;
+import org.apache.accumulo.master.tableOps.MasterRepo;
 import org.apache.accumulo.server.fs.FileRef;
 import org.apache.accumulo.server.fs.VolumeManager;
 import org.apache.accumulo.server.master.LiveTServerSet.TServerConnection;
diff --git a/server/master/src/main/java/org/apache/accumulo/master/tableOps/LoadFiles.java b/server/master/src/main/java/org/apache/accumulo/master/tableOps/bulkVer1/LoadFiles.java
similarity index 98%
rename from server/master/src/main/java/org/apache/accumulo/master/tableOps/LoadFiles.java
rename to server/master/src/main/java/org/apache/accumulo/master/tableOps/bulkVer1/LoadFiles.java
index 225bed8d01..58fb7aa606 100644
--- a/server/master/src/main/java/org/apache/accumulo/master/tableOps/LoadFiles.java
+++ b/server/master/src/main/java/org/apache/accumulo/master/tableOps/bulkVer1/LoadFiles.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.accumulo.master.tableOps;
+package org.apache.accumulo.master.tableOps.bulkVer1;
 
 import static java.nio.charset.StandardCharsets.UTF_8;
 import static org.apache.accumulo.fate.util.UtilWaitThread.sleepUninterruptibly;
@@ -49,6 +49,7 @@
 import org.apache.accumulo.core.util.SimpleThreadPool;
 import org.apache.accumulo.fate.Repo;
 import org.apache.accumulo.master.Master;
+import org.apache.accumulo.master.tableOps.MasterRepo;
 import org.apache.accumulo.server.fs.VolumeManager;
 import org.apache.accumulo.server.master.state.TServerInstance;
 import org.apache.hadoop.fs.FSDataOutputStream;
diff --git a/server/master/src/main/java/org/apache/accumulo/master/tableOps/bulkVer2/BulkImportMove.java b/server/master/src/main/java/org/apache/accumulo/master/tableOps/bulkVer2/BulkImportMove.java
new file mode 100644
index 0000000000..29839f89b9
--- /dev/null
+++ b/server/master/src/main/java/org/apache/accumulo/master/tableOps/bulkVer2/BulkImportMove.java
@@ -0,0 +1,136 @@
+/*
+ * 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.master.tableOps.bulkVer2;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.accumulo.core.Constants;
+import org.apache.accumulo.core.client.impl.AcceptableThriftTableOperationException;
+import org.apache.accumulo.core.client.impl.BulkSerialize;
+import org.apache.accumulo.core.client.impl.thrift.TableOperation;
+import org.apache.accumulo.core.client.impl.thrift.TableOperationExceptionType;
+import org.apache.accumulo.core.conf.Property;
+import org.apache.accumulo.core.util.SimpleThreadPool;
+import org.apache.accumulo.fate.Repo;
+import org.apache.accumulo.master.Master;
+import org.apache.accumulo.master.tableOps.MasterRepo;
+import org.apache.accumulo.server.fs.VolumeManager;
+import org.apache.accumulo.server.util.MetadataTableUtil;
+import org.apache.accumulo.server.zookeeper.TransactionWatcher.ZooArbitrator;
+import org.apache.hadoop.fs.Path;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/*
+ * Bulk import makes requests of tablet servers, and those requests can take a
+ * long time. Our communications to the tablet server may fail, so we won't know
+ * the status of the request. The master will repeat failed requests so now
+ * there are multiple requests to the tablet server. The tablet server will not
+ * execute the request multiple times, so long as the marker it wrote in the
+ * metadata table stays there. The master needs to know when all requests have
+ * finished so it can remove the markers. Did it start? Did it finish? We can see
+ * that *a* request completed by seeing the flag written into the metadata
+ * table, but we won't know if some other rogue thread is still waiting to start
+ * a thread and repeat the operation.
+ *
+ * The master can ask the tablet server if it has any requests still running.
+ * Except the tablet server might have some thread about to start a request, but
+ * before it has made any bookkeeping about the request. To prevent problems
+ * like this, an Arbitrator is used. Before starting any new request, the tablet
+ * server checks the Arbitrator to see if the request is still valid.
+ *
+ */
+class BulkImportMove extends MasterRepo {
+
+  private static final long serialVersionUID = 1L;
+
+  private static final Logger log = LoggerFactory.getLogger(BulkImportMove.class);
+
+  private final BulkInfo bulkInfo;
+
+  public BulkImportMove(BulkInfo bulkInfo) {
+    this.bulkInfo = bulkInfo;
+  }
+
+  @Override
+  public Repo<Master> call(long tid, Master master) throws Exception {
+    final Path bulkDir = new Path(bulkInfo.bulkDir);
+    final Path sourceDir = new Path(bulkInfo.sourceDir);
+    log.debug(" tid {} sourceDir {}", tid, sourceDir);
+
+    VolumeManager fs = master.getFileSystem();
+
+    ZooArbitrator.start(Constants.BULK_ARBITRATOR_TYPE, tid);
+
+    try {
+      Map<String,String> oldToNewNameMap = BulkSerialize.readRenameMap(bulkDir.toString(),
+          p -> fs.open(p));
+      moveFiles(String.format("%016x", tid), sourceDir, bulkDir, master, fs, oldToNewNameMap);
+
+      return new LoadFiles(bulkInfo);
+    } catch (Exception ex) {
+      throw new AcceptableThriftTableOperationException(bulkInfo.tableId.canonicalID(), null,
+          TableOperation.BULK_IMPORT, TableOperationExceptionType.BULK_BAD_INPUT_DIRECTORY,
+          bulkInfo.sourceDir + ": " + ex);
+    }
+  }
+
+  /**
+   * For every entry in renames, move the file from the key path to the value path
+   */
+  private void moveFiles(String fmtTid, Path sourceDir, Path bulkDir, Master master,
+      final VolumeManager fs, Map<String,String> renames) throws Exception {
+    MetadataTableUtil.addBulkLoadInProgressFlag(master,
+        "/" + bulkDir.getParent().getName() + "/" + bulkDir.getName());
+
+    int workerCount = master.getConfiguration().getCount(Property.MASTER_BULK_RENAME_THREADS);
+    SimpleThreadPool workers = new SimpleThreadPool(workerCount, "bulkDir move");
+    List<Future<Boolean>> results = new ArrayList<>();
+
+    for (Map.Entry<String,String> renameEntry : renames.entrySet()) {
+      results.add(workers.submit(() -> {
+        final Path originalPath = new Path(sourceDir, renameEntry.getKey());
+        Path newPath = new Path(bulkDir, renameEntry.getValue());
+        Boolean success = fs.rename(originalPath, newPath);
+        if (success && log.isTraceEnabled())
+          log.trace("tid {} moved {} to {}", fmtTid, originalPath, newPath);
+        return success;
+      }));
+    }
+    workers.shutdown();
+    while (!workers.awaitTermination(1000L, TimeUnit.MILLISECONDS)) {}
+
+    for (Future<Boolean> future : results) {
+      try {
+        if (!future.get()) {
+          throw new AcceptableThriftTableOperationException(bulkInfo.tableId.canonicalID(), null,
+              TableOperation.BULK_IMPORT, TableOperationExceptionType.OTHER,
+              "Failed to move files from " + bulkInfo.sourceDir);
+        }
+      } catch (ExecutionException ee) {
+        throw new AcceptableThriftTableOperationException(bulkInfo.tableId.canonicalID(), null,
+            TableOperation.BULK_IMPORT, TableOperationExceptionType.OTHER,
+            ee.getCause().getMessage());
+      }
+    }
+  }
+}
diff --git a/server/master/src/main/java/org/apache/accumulo/master/tableOps/bulkVer2/BulkInfo.java b/server/master/src/main/java/org/apache/accumulo/master/tableOps/bulkVer2/BulkInfo.java
new file mode 100644
index 0000000000..6c94e20f59
--- /dev/null
+++ b/server/master/src/main/java/org/apache/accumulo/master/tableOps/bulkVer2/BulkInfo.java
@@ -0,0 +1,33 @@
+/*
+ * 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.master.tableOps.bulkVer2;
+
+import java.io.Serializable;
+
+import org.apache.accumulo.core.client.impl.Table;
+
+/**
+ * Package private class to hold all the information used for bulk import2
+ */
+class BulkInfo implements Serializable {
+  private static final long serialVersionUID = 1L;
+
+  Table.ID tableId;
+  String sourceDir;
+  String bulkDir;
+  boolean setTime;
+}
diff --git a/server/master/src/main/java/org/apache/accumulo/master/tableOps/bulkVer2/CleanUpBulkImport.java b/server/master/src/main/java/org/apache/accumulo/master/tableOps/bulkVer2/CleanUpBulkImport.java
new file mode 100644
index 0000000000..c912f51417
--- /dev/null
+++ b/server/master/src/main/java/org/apache/accumulo/master/tableOps/bulkVer2/CleanUpBulkImport.java
@@ -0,0 +1,76 @@
+/*
+ * 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.master.tableOps.bulkVer2;
+
+import java.io.IOException;
+
+import org.apache.accumulo.core.Constants;
+import org.apache.accumulo.core.client.Connector;
+import org.apache.accumulo.core.client.impl.Table;
+import org.apache.accumulo.fate.Repo;
+import org.apache.accumulo.master.Master;
+import org.apache.accumulo.master.tableOps.MasterRepo;
+import org.apache.accumulo.master.tableOps.Utils;
+import org.apache.accumulo.server.util.MetadataTableUtil;
+import org.apache.accumulo.server.zookeeper.TransactionWatcher.ZooArbitrator;
+import org.apache.hadoop.fs.Path;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class CleanUpBulkImport extends MasterRepo {
+
+  private static final long serialVersionUID = 1L;
+
+  private static final Logger log = LoggerFactory.getLogger(CleanUpBulkImport.class);
+
+  private Table.ID tableId;
+  private String source;
+  private String bulk;
+
+  public CleanUpBulkImport(Table.ID tableId, String source, String bulk) {
+    this.tableId = tableId;
+    this.source = source;
+    this.bulk = bulk;
+  }
+
+  @Override
+  public Repo<Master> call(long tid, Master master) throws Exception {
+    log.debug("removing the bulkDir processing flag file in " + bulk);
+    Path bulkDir = new Path(bulk);
+    MetadataTableUtil.removeBulkLoadInProgressFlag(master,
+        "/" + bulkDir.getParent().getName() + "/" + bulkDir.getName());
+    MetadataTableUtil.addDeleteEntry(master, tableId, bulkDir.toString());
+    log.debug("removing the metadata table markers for loaded files");
+    Connector conn = master.getConnector();
+    MetadataTableUtil.removeBulkLoadEntries(conn, tableId, tid);
+    Utils.unreserveHdfsDirectory(source, tid);
+    Utils.getReadLock(tableId, tid).unlock();
+    // delete json renames and mapping files
+    Path renamingFile = new Path(bulkDir, Constants.BULK_RENAME_FILE);
+    Path mappingFile = new Path(bulkDir, Constants.BULK_LOAD_MAPPING);
+    try {
+      master.getFileSystem().delete(renamingFile);
+      master.getFileSystem().delete(mappingFile);
+    } catch (IOException ioe) {
+      log.debug("Failed to delete renames and/or loadmap", ioe);
+    }
+
+    log.debug("completing bulkDir import transaction " + tid);
+    ZooArbitrator.cleanup(Constants.BULK_ARBITRATOR_TYPE, tid);
+    return null;
+  }
+}
diff --git a/server/master/src/main/java/org/apache/accumulo/master/tableOps/bulkVer2/CompleteBulkImport.java b/server/master/src/main/java/org/apache/accumulo/master/tableOps/bulkVer2/CompleteBulkImport.java
new file mode 100644
index 0000000000..02a9848017
--- /dev/null
+++ b/server/master/src/main/java/org/apache/accumulo/master/tableOps/bulkVer2/CompleteBulkImport.java
@@ -0,0 +1,45 @@
+/*
+ * 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.master.tableOps.bulkVer2;
+
+import org.apache.accumulo.core.Constants;
+import org.apache.accumulo.core.client.impl.Table;
+import org.apache.accumulo.fate.Repo;
+import org.apache.accumulo.master.Master;
+import org.apache.accumulo.master.tableOps.MasterRepo;
+import org.apache.accumulo.server.zookeeper.TransactionWatcher.ZooArbitrator;
+
+public class CompleteBulkImport extends MasterRepo {
+
+  private static final long serialVersionUID = 1L;
+
+  private Table.ID tableId;
+  private String source;
+  private String bulk;
+
+  public CompleteBulkImport(Table.ID tableId, String source, String bulk) {
+    this.tableId = tableId;
+    this.source = source;
+    this.bulk = bulk;
+  }
+
+  @Override
+  public Repo<Master> call(long tid, Master master) throws Exception {
+    ZooArbitrator.stop(Constants.BULK_ARBITRATOR_TYPE, tid);
+    return new CleanUpBulkImport(tableId, source, bulk);
+  }
+}
diff --git a/server/master/src/main/java/org/apache/accumulo/master/tableOps/bulkVer2/LoadFiles.java b/server/master/src/main/java/org/apache/accumulo/master/tableOps/bulkVer2/LoadFiles.java
new file mode 100644
index 0000000000..bfd7fdd58b
--- /dev/null
+++ b/server/master/src/main/java/org/apache/accumulo/master/tableOps/bulkVer2/LoadFiles.java
@@ -0,0 +1,210 @@
+/*
+ * 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.master.tableOps.bulkVer2;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.accumulo.core.client.AccumuloException;
+import org.apache.accumulo.core.client.AccumuloSecurityException;
+import org.apache.accumulo.core.client.TableNotFoundException;
+import org.apache.accumulo.core.client.impl.Bulk;
+import org.apache.accumulo.core.client.impl.Bulk.Files;
+import org.apache.accumulo.core.client.impl.BulkSerialize;
+import org.apache.accumulo.core.client.impl.BulkSerialize.LoadMappingIterator;
+import org.apache.accumulo.core.client.impl.Table;
+import org.apache.accumulo.core.conf.Property;
+import org.apache.accumulo.core.data.impl.KeyExtent;
+import org.apache.accumulo.core.data.thrift.MapFileInfo;
+import org.apache.accumulo.core.metadata.schema.MetadataScanner;
+import org.apache.accumulo.core.metadata.schema.TabletMetadata;
+import org.apache.accumulo.core.rpc.ThriftUtil;
+import org.apache.accumulo.core.tabletserver.thrift.TabletClientService;
+import org.apache.accumulo.core.trace.Tracer;
+import org.apache.accumulo.core.util.HostAndPort;
+import org.apache.accumulo.core.util.MapCounter;
+import org.apache.accumulo.fate.Repo;
+import org.apache.accumulo.master.Master;
+import org.apache.accumulo.master.tableOps.MasterRepo;
+import org.apache.accumulo.server.fs.VolumeManager;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.Text;
+import org.apache.thrift.TException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Make asynchronous load calls to each overlapping Tablet. This RepO does its work on the isReady
+ * and will return a linear sleep value based on the largest number of Tablets on a TabletServer.
+ */
+class LoadFiles extends MasterRepo {
+
+  private static final long serialVersionUID = 1L;
+
+  private static final Logger log = LoggerFactory.getLogger(LoadFiles.class);
+
+  private final BulkInfo bulkInfo;
+
+  public LoadFiles(BulkInfo bulkInfo) {
+    this.bulkInfo = bulkInfo;
+  }
+
+  @Override
+  public long isReady(long tid, Master master) throws Exception {
+    if (master.onlineTabletServers().size() == 0) {
+      log.warn("There are no tablet server to process bulkDir import, waiting (tid = " + tid + ")");
+      return 100;
+    }
+    VolumeManager fs = master.getFileSystem();
+    final Path bulkDir = new Path(bulkInfo.bulkDir);
+    try (LoadMappingIterator lmi = BulkSerialize.getUpdatedLoadMapping(bulkDir.toString(),
+        bulkInfo.tableId, p -> fs.open(p))) {
+      return loadFiles(bulkInfo.tableId, bulkDir, lmi, master, tid);
+    }
+  }
+
+  @Override
+  public Repo<Master> call(final long tid, final Master master) throws Exception {
+    return new CompleteBulkImport(bulkInfo.tableId, bulkInfo.sourceDir, bulkInfo.bulkDir);
+  }
+
+  static boolean equals(Text t1, Text t2) {
+    if (t1 == null || t2 == null)
+      return t1 == t2;
+
+    return t1.equals(t2);
+  }
+
+  /**
+   * Make asynchronous load calls to each overlapping Tablet in the bulk mapping. Return a sleep
+   * time to isReady based on a factor of the TabletServer with the most Tablets. This method will
+   * scan the metadata table getting Tablet range and location information. It will return 0 when
+   * all files have been loaded.
+   */
+  private long loadFiles(Table.ID tableId, Path bulkDir, LoadMappingIterator lmi, Master master,
+      long tid) throws AccumuloSecurityException, TableNotFoundException, AccumuloException {
+
+    Map.Entry<KeyExtent,Bulk.Files> loadMapEntry = lmi.next();
+
+    Text startRow = loadMapEntry.getKey().getPrevEndRow();
+
+    Iterable<TabletMetadata> tableMetadata = MetadataScanner.builder().from(master)
+        .overUserTableId(tableId, startRow, null).fetchPrev().fetchLocation().fetchLoaded().build();
+
+    long timeInMillis = master.getConfiguration().getTimeInMillis(Property.MASTER_BULK_TIMEOUT);
+    Iterator<TabletMetadata> tabletIter = tableMetadata.iterator();
+
+    List<TabletMetadata> tablets = new ArrayList<>();
+    TabletMetadata currentTablet = tabletIter.next();
+    HostAndPort server = null;
+
+    // track how many tablets were sent load messages per tablet server
+    MapCounter<HostAndPort> tabletsPerServer = new MapCounter<>();
+
+    String fmtTid = String.format("%016x", tid);
+
+    int locationLess = 0;
+
+    long t1 = System.currentTimeMillis();
+    while (true) {
+      if (loadMapEntry == null) {
+        if (!lmi.hasNext()) {
+          break;
+        }
+        loadMapEntry = lmi.next();
+      }
+      KeyExtent fke = loadMapEntry.getKey();
+      Files files = loadMapEntry.getValue();
+      loadMapEntry = null;
+
+      tablets.clear();
+
+      while (!equals(currentTablet.getPrevEndRow(), fke.getPrevEndRow())) {
+        currentTablet = tabletIter.next();
+      }
+      tablets.add(currentTablet);
+
+      while (!equals(currentTablet.getEndRow(), fke.getEndRow())) {
+        currentTablet = tabletIter.next();
+        tablets.add(currentTablet);
+      }
+
+      for (TabletMetadata tablet : tablets) {
+        // send files to tablet sever
+        // ideally there should only be one tablet location to send all the files
+
+        TabletMetadata.Location location = tablet.getLocation();
+        if (location == null) {
+          locationLess++;
+          continue;
+        } else {
+          server = location.getHostAndPort();
+        }
+
+        Set<String> loadedFiles = tablet.getLoaded();
+
+        Map<String,MapFileInfo> thriftImports = new HashMap<>();
+
+        for (final Bulk.FileInfo fileInfo : files) {
+          String fullPath = new Path(bulkDir, fileInfo.getFileName()).toString();
+
+          if (!loadedFiles.contains(fullPath)) {
+            thriftImports.put(fileInfo.getFileName(), new MapFileInfo(fileInfo.getEstFileSize()));
+          }
+        }
+
+        if (thriftImports.size() > 0) {
+          // must always increment this even if there is a comms failure, because it indicates there
+          // is work to do
+          tabletsPerServer.increment(server, 1);
+          log.trace("tid {} asking {} to bulk import {} files", fmtTid, server,
+              thriftImports.size());
+          TabletClientService.Client client = null;
+          try {
+            client = ThriftUtil.getTServerClient(server, master, timeInMillis);
+            client.loadFiles(Tracer.traceInfo(), master.rpcCreds(), tid, fke.toThrift(),
+                bulkDir.toString(), thriftImports, bulkInfo.setTime);
+          } catch (TException ex) {
+            log.debug("rpc failed server: " + server + ", tid:" + fmtTid + " " + ex.getMessage(),
+                ex);
+          } finally {
+            ThriftUtil.returnClient(client);
+          }
+        }
+      }
+    }
+    long t2 = System.currentTimeMillis();
+
+    long sleepTime = 0;
+    if (tabletsPerServer.size() > 0) {
+      // find which tablet server had the most load messages sent to it and sleep 13ms for each load
+      // message
+      sleepTime = Collections.max(tabletsPerServer.values()) * 13;
+    }
+
+    if (locationLess > 0) {
+      sleepTime = Math.max(100, Math.max(2 * (t2 - t1), sleepTime));
+    }
+
+    return sleepTime;
+  }
+}
diff --git a/server/master/src/main/java/org/apache/accumulo/master/tableOps/bulkVer2/PrepBulkImport.java b/server/master/src/main/java/org/apache/accumulo/master/tableOps/bulkVer2/PrepBulkImport.java
new file mode 100644
index 0000000000..48b3e242e9
--- /dev/null
+++ b/server/master/src/main/java/org/apache/accumulo/master/tableOps/bulkVer2/PrepBulkImport.java
@@ -0,0 +1,256 @@
+/*
+ * 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.master.tableOps.bulkVer2;
+
+import static org.apache.accumulo.fate.util.UtilWaitThread.sleepUninterruptibly;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.accumulo.core.Constants;
+import org.apache.accumulo.core.client.impl.AcceptableThriftTableOperationException;
+import org.apache.accumulo.core.client.impl.BulkSerialize;
+import org.apache.accumulo.core.client.impl.BulkSerialize.LoadMappingIterator;
+import org.apache.accumulo.core.client.impl.Table;
+import org.apache.accumulo.core.client.impl.Tables;
+import org.apache.accumulo.core.client.impl.thrift.TableOperation;
+import org.apache.accumulo.core.client.impl.thrift.TableOperationExceptionType;
+import org.apache.accumulo.core.data.impl.KeyExtent;
+import org.apache.accumulo.core.file.FileOperations;
+import org.apache.accumulo.core.master.state.tables.TableState;
+import org.apache.accumulo.core.metadata.schema.MetadataScanner;
+import org.apache.accumulo.core.metadata.schema.TabletMetadata;
+import org.apache.accumulo.fate.Repo;
+import org.apache.accumulo.master.Master;
+import org.apache.accumulo.master.tableOps.MasterRepo;
+import org.apache.accumulo.master.tableOps.Utils;
+import org.apache.accumulo.server.ServerConstants;
+import org.apache.accumulo.server.fs.VolumeManager;
+import org.apache.accumulo.server.tablets.UniqueNameAllocator;
+import org.apache.accumulo.server.zookeeper.TransactionWatcher;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.Text;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.collect.Iterators;
+
+/**
+ * Prepare bulk import directory. This REPO creates a bulk directory in Accumulo, list all the files
+ * in the original directory and creates a renaming file for moving the files (which happens next in
+ * BulkImportMove). The renaming file has a mapping of originalPath -> newPath. The newPath will be
+ * the bulk directory in Accumulo. The renaming file is called {@link Constants.BULK_RENAME_FILE}
+ * and is written to the {@link Constants.BULK_PREFIX} bulk directory generated here.
+ *
+ * @since 2.0.0
+ */
+public class PrepBulkImport extends MasterRepo {
+
+  private static final long serialVersionUID = 1L;
+
+  private static final Logger log = LoggerFactory.getLogger(PrepBulkImport.class);
+
+  private final BulkInfo bulkInfo;
+
+  public PrepBulkImport(Table.ID tableId, String sourceDir, boolean setTime) {
+    BulkInfo info = new BulkInfo();
+    info.tableId = tableId;
+    info.sourceDir = sourceDir;
+    info.setTime = setTime;
+    this.bulkInfo = info;
+  }
+
+  @Override
+  public long isReady(long tid, Master master) throws Exception {
+    if (!Utils.getReadLock(bulkInfo.tableId, tid).tryLock())
+      return 100;
+
+    if (master.onlineTabletServers().size() == 0)
+      return 500;
+    Tables.clearCache(master.getInstance());
+    if (Tables.getTableState(master.getInstance(), bulkInfo.tableId) == TableState.ONLINE) {
+      return Utils.reserveHdfsDirectory(bulkInfo.sourceDir, tid);
+    } else {
+      throw new AcceptableThriftTableOperationException(bulkInfo.tableId.canonicalID(), null,
+          TableOperation.BULK_IMPORT, TableOperationExceptionType.OFFLINE, null);
+    }
+  }
+
+  static boolean equals(Text t1, Text t2) {
+    return LoadFiles.equals(t1, t2);
+  }
+
+  @VisibleForTesting
+  static interface TabletIterFactory {
+    Iterator<KeyExtent> newTabletIter(Text startRow) throws Exception;
+  }
+
+  @VisibleForTesting
+  static void checkForMerge(String tableId, Iterator<KeyExtent> lmi,
+      TabletIterFactory tabletIterFactory) throws Exception {
+    KeyExtent currentRange = lmi.next();
+
+    Text startRow = currentRange.getPrevEndRow();
+
+    Iterator<KeyExtent> tabletIter = tabletIterFactory.newTabletIter(startRow);
+
+    KeyExtent currentTablet = tabletIter.next();
+
+    if (!tabletIter.hasNext() && equals(currentTablet.getPrevEndRow(), currentRange.getPrevEndRow())
+        && equals(currentTablet.getEndRow(), currentRange.getEndRow()))
+      currentRange = null;
+
+    while (tabletIter.hasNext()) {
+
+      if (currentRange == null) {
+        if (!lmi.hasNext()) {
+          break;
+        }
+        currentRange = lmi.next();
+      }
+
+      while (!equals(currentTablet.getPrevEndRow(), currentRange.getPrevEndRow())
+          && tabletIter.hasNext()) {
+        currentTablet = tabletIter.next();
+      }
+
+      boolean matchedPrevRow = equals(currentTablet.getPrevEndRow(), currentRange.getPrevEndRow());
+
+      while (!equals(currentTablet.getEndRow(), currentRange.getEndRow()) && tabletIter.hasNext()) {
+        currentTablet = tabletIter.next();
+      }
+
+      if (!matchedPrevRow || !equals(currentTablet.getEndRow(), currentRange.getEndRow())) {
+        break;
+      }
+
+      currentRange = null;
+    }
+
+    if (currentRange != null || lmi.hasNext()) {
+      // a merge happened between the time the mapping was generated and the table lock was
+      // acquired
+      throw new AcceptableThriftTableOperationException(tableId, null, TableOperation.BULK_IMPORT,
+          TableOperationExceptionType.OTHER, "Concurrent merge happened"); // TODO need to handle
+                                                                           // this on the client
+                                                                           // side
+    }
+  }
+
+  private void checkForMerge(final Master master) throws Exception {
+
+    VolumeManager fs = master.getFileSystem();
+    final Path bulkDir = new Path(bulkInfo.sourceDir);
+    try (LoadMappingIterator lmi = BulkSerialize.readLoadMapping(bulkDir.toString(),
+        bulkInfo.tableId, p -> fs.open(p))) {
+
+      Iterators.transform(lmi, entry -> entry.getKey());
+
+      TabletIterFactory tabletIterFactory = startRow -> {
+        Iterable<TabletMetadata> tableMetadata = MetadataScanner.builder().from(master)
+            .overUserTableId(bulkInfo.tableId, startRow, null).build();
+        return Iterators.transform(tableMetadata.iterator(), tm -> tm.getExtent());
+      };
+
+      checkForMerge(bulkInfo.tableId.canonicalID(),
+          Iterators.transform(lmi, entry -> entry.getKey()), tabletIterFactory);
+    }
+  }
+
+  @Override
+  public Repo<Master> call(final long tid, final Master master) throws Exception {
+    // now that table lock is acquired check that all splits in load mapping exists in table
+    checkForMerge(master);
+
+    VolumeManager fs = master.getFileSystem();
+    final UniqueNameAllocator namer = UniqueNameAllocator.getInstance();
+    Path sourceDir = new Path(bulkInfo.sourceDir);
+    FileStatus[] files = fs.listStatus(sourceDir);
+
+    Path bulkDir = createNewBulkDir(fs, bulkInfo.tableId);
+    Path mappingFile = new Path(sourceDir, Constants.BULK_LOAD_MAPPING);
+
+    Map<String,String> oldToNewNameMap = new HashMap<>();
+    for (FileStatus file : files) {
+      final FileStatus fileStatus = file;
+      final Path originalPath = fileStatus.getPath();
+      String fileNameParts[] = originalPath.getName().split("\\.");
+      String extension = "";
+      boolean invalidFileName;
+      if (fileNameParts.length > 1) {
+        extension = fileNameParts[fileNameParts.length - 1];
+        invalidFileName = !FileOperations.getValidExtensions().contains(extension);
+      } else {
+        invalidFileName = true;
+      }
+      if (invalidFileName) {
+        log.warn("{} does not have a valid extension, ignoring", fileStatus.getPath());
+        continue;
+      }
+
+      String newName = "I" + namer.getNextName() + "." + extension;
+      Path newPath = new Path(bulkDir, newName);
+      oldToNewNameMap.put(originalPath.getName(), newPath.getName());
+    }
+
+    // also have to move mapping file
+    Path newMappingFile = new Path(bulkDir, mappingFile.getName());
+    oldToNewNameMap.put(mappingFile.getName(), newMappingFile.getName());
+    BulkSerialize.writeRenameMap(oldToNewNameMap, bulkDir.toString(), p -> fs.create(p));
+
+    bulkInfo.bulkDir = bulkDir.toString();
+    // return the next step, which will move files
+    return new BulkImportMove(bulkInfo);
+  }
+
+  private Path createNewBulkDir(VolumeManager fs, Table.ID tableId) throws IOException {
+    Path tempPath = fs.matchingFileSystem(new Path(bulkInfo.sourceDir),
+        ServerConstants.getTablesDirs());
+    if (tempPath == null)
+      throw new IOException(bulkInfo.sourceDir + " is not in a volume configured for Accumulo");
+
+    String tableDir = tempPath.toString();
+    if (tableDir == null)
+      throw new IOException(bulkInfo.sourceDir + " is not in a volume configured for Accumulo");
+    Path directory = new Path(tableDir + "/" + tableId);
+    fs.mkdirs(directory);
+
+    UniqueNameAllocator namer = UniqueNameAllocator.getInstance();
+    while (true) {
+      Path newBulkDir = new Path(directory, Constants.BULK_PREFIX + namer.getNextName());
+      if (fs.mkdirs(newBulkDir))
+        return newBulkDir;
+      log.warn("Failed to create {} for unknown reason", newBulkDir);
+
+      sleepUninterruptibly(3, TimeUnit.SECONDS);
+    }
+  }
+
+  @Override
+  public void undo(long tid, Master environment) throws Exception {
+    // unreserve sourceDir/error directories
+    Utils.unreserveHdfsDirectory(bulkInfo.sourceDir, tid);
+    Utils.getReadLock(bulkInfo.tableId, tid).unlock();
+    TransactionWatcher.ZooArbitrator.cleanup(Constants.BULK_ARBITRATOR_TYPE, tid);
+  }
+
+}
diff --git a/server/master/src/test/java/org/apache/accumulo/master/tableOps/bulkVer2/PrepBulkImportTest.java b/server/master/src/test/java/org/apache/accumulo/master/tableOps/bulkVer2/PrepBulkImportTest.java
new file mode 100644
index 0000000000..a5adbd95b7
--- /dev/null
+++ b/server/master/src/test/java/org/apache/accumulo/master/tableOps/bulkVer2/PrepBulkImportTest.java
@@ -0,0 +1,181 @@
+/*
+ * 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.master.tableOps.bulkVer2;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import org.apache.accumulo.core.client.impl.AcceptableThriftTableOperationException;
+import org.apache.accumulo.core.client.impl.Table;
+import org.apache.accumulo.core.data.impl.KeyExtent;
+import org.apache.accumulo.master.tableOps.bulkVer2.PrepBulkImport.TabletIterFactory;
+import org.apache.hadoop.io.Text;
+import org.junit.Assert;
+import org.junit.Test;
+
+import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.Iterables;
+import com.google.common.collect.Sets;
+
+public class PrepBulkImportTest {
+
+  KeyExtent nke(String prev, String end) {
+    Text per = prev == null ? null : new Text(prev);
+    Text er = end == null ? null : new Text(end);
+
+    return new KeyExtent(Table.ID.of("1"), er, per);
+  }
+
+  List<KeyExtent> createExtents(Iterable<String> rowsIter) {
+    List<KeyExtent> extents = new ArrayList<>();
+
+    List<String> rows = new ArrayList<>();
+    rowsIter.forEach(rows::add);
+
+    Collections.sort(rows);
+    for (int i = 0; i < rows.size(); i++) {
+      extents.add(nke(i == 0 ? null : rows.get(i - 1), rows.get(i)));
+
+    }
+
+    extents.add(nke(rows.size() == 0 ? null : rows.get(rows.size() - 1), null));
+
+    return extents;
+  }
+
+  Iterable<List<KeyExtent>> powerSet(KeyExtent... extents) {
+    Set<Set<KeyExtent>> powerSet = Sets.powerSet(ImmutableSet.copyOf(extents));
+
+    return Iterables.transform(powerSet, set -> {
+      List<KeyExtent> list = new ArrayList<>(set);
+
+      Collections.sort(list);
+
+      return list;
+    });
+  }
+
+  public void runTest(List<KeyExtent> loadRanges, List<KeyExtent> tabletRanges) throws Exception {
+    TabletIterFactory tabletIterFactory = startRow -> {
+      int start = -1;
+
+      if (startRow == null) {
+        start = 0;
+      } else {
+        for (int i = 0; i < tabletRanges.size(); i++) {
+          if (tabletRanges.get(i).contains(startRow)) {
+            start = i;
+            break;
+          }
+        }
+      }
+
+      return tabletRanges.subList(start, tabletRanges.size()).iterator();
+    };
+
+    PrepBulkImport.checkForMerge("1", loadRanges.iterator(), tabletIterFactory);
+  }
+
+  static String toRangeStrings(Collection<KeyExtent> extents) {
+    return extents.stream().map(ke -> "(" + ke.getPrevEndRow() + "," + ke.getEndRow() + "]")
+        .collect(Collectors.joining(","));
+  }
+
+  public void runExceptionTest(List<KeyExtent> loadRanges, List<KeyExtent> tabletRanges) {
+    try {
+      runTest(loadRanges, tabletRanges);
+      Assert.fail("expected " + toRangeStrings(loadRanges) + " to fail against "
+          + toRangeStrings(tabletRanges));
+    } catch (Exception e) {
+      Assert.assertTrue(e instanceof AcceptableThriftTableOperationException);
+    }
+  }
+
+  @Test
+  public void testSingleTablet() throws Exception {
+    runTest(Arrays.asList(nke(null, null)), Arrays.asList(nke(null, null)));
+
+    for (List<KeyExtent> loadRanges : powerSet(nke(null, "b"), nke("b", "k"), nke("k", "r"),
+        nke("r", null))) {
+      if (loadRanges.isEmpty()) {
+        continue;
+      }
+      runExceptionTest(loadRanges, Arrays.asList(nke(null, null)));
+    }
+  }
+
+  @Test
+  public void testNominal() throws Exception {
+
+    for (List<KeyExtent> loadRanges : powerSet(nke(null, "b"), nke("b", "m"), nke("m", "r"),
+        nke("r", "v"), nke("v", null))) {
+
+      if (loadRanges.isEmpty()) {
+        loadRanges = Arrays.asList(nke(null, null));
+      }
+
+      List<String> requiredRows = Arrays.asList("b", "m", "r", "v");
+      for (Set<String> otherRows : Sets.powerSet(ImmutableSet.of("a", "c", "q", "t", "x"))) {
+        runTest(loadRanges, createExtents(Iterables.concat(requiredRows, otherRows)));
+      }
+    }
+  }
+
+  @Test
+  public void testException() {
+    for (List<KeyExtent> loadRanges : powerSet(nke(null, "b"), nke("b", "m"), nke("m", "r"),
+        nke("r", "v"), nke("v", null))) {
+
+      if (loadRanges.isEmpty()) {
+        continue;
+      }
+
+      Set<String> rows = new HashSet<>();
+      for (KeyExtent ke : loadRanges) {
+        if (ke.getPrevEndRow() != null) {
+          rows.add(ke.getPrevEndRow().toString());
+        }
+        if (ke.getEndRow() != null) {
+          rows.add(ke.getEndRow().toString());
+        }
+      }
+
+      for (String row : rows) {
+        Set<String> rows2 = new HashSet<>(rows);
+        rows2.remove(row);
+        // test will all but one of the rows in the load mapping
+        for (Set<String> otherRows : Sets.powerSet(ImmutableSet.of("a", "c", "q", "t", "x"))) {
+          runExceptionTest(loadRanges, createExtents(Iterables.concat(rows2, otherRows)));
+        }
+      }
+
+      if (rows.size() > 1) {
+        // test with none of the rows in the load mapping
+        for (Set<String> otherRows : Sets.powerSet(ImmutableSet.of("a", "c", "q", "t", "x"))) {
+          runExceptionTest(loadRanges, createExtents(otherRows));
+        }
+      }
+    }
+
+  }
+}
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 52aaf055dc..09813ba2c5 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
@@ -513,6 +513,34 @@ private static long jitter(long ms) {
       return failures;
     }
 
+    @Override
+    public void loadFiles(TInfo tinfo, TCredentials credentials, long tid, TKeyExtent tke,
+        String dir, Map<String,MapFileInfo> fileMap, boolean setTime)
+        throws ThriftSecurityException {
+      if (!security.canPerformSystemActions(credentials))
+        throw new ThriftSecurityException(credentials.getPrincipal(),
+            SecurityErrorCode.PERMISSION_DENIED);
+
+      Map<FileRef,MapFileInfo> fileRefMap = new HashMap<>();
+      for (Entry<String,MapFileInfo> mapping : fileMap.entrySet()) {
+        Path path = new Path(dir, mapping.getKey());
+        FileSystem ns = fs.getVolumeByPath(path).getFileSystem();
+        path = ns.makeQualified(path);
+        fileRefMap.put(new FileRef(path.toString(), path), mapping.getValue());
+      }
+
+      Tablet importTablet = onlineTablets.get(new KeyExtent(tke));
+
+      if (importTablet != null) {
+        try {
+          importTablet.importMapFiles(tid, fileRefMap, setTime);
+        } catch (IOException ioe) {
+          log.info("files {} not imported to {}: {}", fileMap.keySet(), new KeyExtent(tke),
+              ioe.getMessage());
+        }
+      }
+    }
+
     @Override
     public InitialScan startScan(TInfo tinfo, TCredentials credentials, TKeyExtent textent,
         TRange range, List<TColumn> columns, int batchSize, List<IterInfo> ssiList,
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Tablet.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Tablet.java
index b3139e77c1..2d27ce0dbc 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Tablet.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Tablet.java
@@ -2394,7 +2394,7 @@ public void importMapFiles(long tid, Map<FileRef,MapFileInfo> fileMap, boolean s
       if (alreadyImported != null) {
         for (FileRef entry : alreadyImported) {
           if (fileMap.remove(entry) != null) {
-            log.info("Ignoring import of bulk file already imported: " + entry);
+            log.trace("Ignoring import of bulk file already imported: {}", entry);
           }
         }
       }
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/BulkIT.java b/test/src/main/java/org/apache/accumulo/test/functional/BulkIT.java
index f51ae1bda7..1301c09484 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/BulkIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/BulkIT.java
@@ -16,9 +16,14 @@
  */
 package org.apache.accumulo.test.functional;
 
+import java.io.IOException;
+
 import org.apache.accumulo.core.cli.BatchWriterOpts;
 import org.apache.accumulo.core.cli.ScannerOpts;
+import org.apache.accumulo.core.client.AccumuloException;
+import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.Connector;
+import org.apache.accumulo.core.client.TableNotFoundException;
 import org.apache.accumulo.core.util.CachedConfiguration;
 import org.apache.accumulo.harness.AccumuloClusterHarness;
 import org.apache.accumulo.test.TestIngest;
@@ -61,11 +66,18 @@ public void restoreConf() {
   public void test() throws Exception {
     runTest(getConnector(), getCluster().getFileSystem(), getCluster().getTemporaryPath(),
         getAdminPrincipal(), getUniqueNames(1)[0], this.getClass().getName(),
-        testName.getMethodName());
+        testName.getMethodName(), false);
+  }
+
+  @Test
+  public void testOld() throws Exception {
+    runTest(getConnector(), getCluster().getFileSystem(), getCluster().getTemporaryPath(),
+        getAdminPrincipal(), getUniqueNames(1)[0], this.getClass().getName(),
+        testName.getMethodName(), true);
   }
 
   static void runTest(Connector c, FileSystem fs, Path basePath, String principal, String tableName,
-      String filePrefix, String dirSuffix) throws Exception {
+      String filePrefix, String dirSuffix, boolean useOld) throws Exception {
     c.tableOperations().create(tableName);
 
     Path base = new Path(basePath, "testBulkFail_" + dirSuffix);
@@ -97,9 +109,7 @@ static void runTest(Connector c, FileSystem fs, Path basePath, String principal,
     // create an rfile with one entry, there was a bug with this:
     TestIngest.ingest(c, fs, opts, BWOPTS);
 
-    // Make sure the server can modify the files
-    c.tableOperations().importDirectory(tableName, files.toString(), bulkFailures.toString(),
-        false);
+    bulkLoad(c, tableName, bulkFailures, files, useOld);
     VerifyIngest.Opts vopts = new VerifyIngest.Opts();
     vopts.setTableName(tableName);
     vopts.random = 56;
@@ -114,4 +124,18 @@ static void runTest(Connector c, FileSystem fs, Path basePath, String principal,
     VerifyIngest.verifyIngest(c, vopts, SOPTS);
   }
 
+  @SuppressWarnings("deprecation")
+  private static void bulkLoad(Connector c, String tableName, Path bulkFailures, Path files,
+      boolean useOld)
+      throws TableNotFoundException, IOException, AccumuloException, AccumuloSecurityException {
+    // Make sure the server can modify the files
+    if (useOld) {
+      c.tableOperations().importDirectory(tableName, files.toString(), bulkFailures.toString(),
+          false);
+    } else {
+      c.tableOperations().addFilesTo(tableName).from(files.toString()).load();
+    }
+
+  }
+
 }
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/BulkLoadIT.java b/test/src/main/java/org/apache/accumulo/test/functional/BulkLoadIT.java
new file mode 100644
index 0000000000..d97d1f8c1d
--- /dev/null
+++ b/test/src/main/java/org/apache/accumulo/test/functional/BulkLoadIT.java
@@ -0,0 +1,268 @@
+/*
+ * 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.test.functional;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.junit.Assert.fail;
+
+import java.io.FileNotFoundException;
+import java.util.Iterator;
+import java.util.Map.Entry;
+import java.util.SortedSet;
+import java.util.TreeSet;
+
+import org.apache.accumulo.core.client.AccumuloException;
+import org.apache.accumulo.core.client.Connector;
+import org.apache.accumulo.core.client.Scanner;
+import org.apache.accumulo.core.conf.AccumuloConfiguration;
+import org.apache.accumulo.core.data.Key;
+import org.apache.accumulo.core.data.Value;
+import org.apache.accumulo.core.file.FileOperations;
+import org.apache.accumulo.core.file.FileSKVWriter;
+import org.apache.accumulo.core.file.rfile.RFile;
+import org.apache.accumulo.core.security.Authorizations;
+import org.apache.accumulo.harness.AccumuloClusterHarness;
+import org.apache.accumulo.minicluster.MemoryUnit;
+import org.apache.accumulo.minicluster.ServerType;
+import org.apache.accumulo.minicluster.impl.MiniAccumuloConfigImpl;
+import org.apache.accumulo.server.conf.ServerConfigurationFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RawLocalFileSystem;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.io.Text;
+import org.junit.Test;
+
+/**
+ * Tests new bulk import technique. If the old technique ever gets removed this will replace
+ * {@link BulkFileIT}
+ *
+ * @since 2.0
+ */
+public class BulkLoadIT extends AccumuloClusterHarness {
+
+  @Override
+  public void configureMiniCluster(MiniAccumuloConfigImpl cfg, Configuration conf) {
+    cfg.setMemory(ServerType.TABLET_SERVER, 128 * 4, MemoryUnit.MEGABYTE);
+
+    // use raw local file system
+    conf.set("fs.file.impl", RawLocalFileSystem.class.getName());
+  }
+
+  @Override
+  protected int defaultTimeoutSeconds() {
+    return 4 * 60;
+  }
+
+  @Test
+  public void testSingleTabletSingleFile() throws Exception {
+    Connector c = getConnector();
+    String tableName = getUniqueNames(1)[0];
+    c.tableOperations().create(tableName);
+    addSplits(tableName, "0333");
+
+    Configuration conf = new Configuration();
+    AccumuloConfiguration aconf = new ServerConfigurationFactory(c.getInstance())
+        .getSystemConfiguration();
+    FileSystem fs = getCluster().getFileSystem();
+    String rootPath = cluster.getTemporaryPath().toString();
+
+    String dir = rootPath + "/testSingleTabletSingleFileNoSplits-" + getUniqueNames(1)[0];
+    Path bulkDir = new Path(dir);
+
+    fs.delete(bulkDir, true);
+
+    FileSKVWriter writer1 = FileOperations.getInstance().newWriterBuilder()
+        .forFile(dir + "/f1." + RFile.EXTENSION, fs, conf).withTableConfiguration(aconf).build();
+    writer1.startDefaultLocalityGroup();
+    writeData(writer1, 0, 332);
+    writer1.close();
+
+    c.tableOperations().addFilesTo(tableName).from(dir).load();
+  }
+
+  @Test
+  public void testSingleTabletSingleFileNoSplits() throws Exception {
+    Connector c = getConnector();
+    String tableName = getUniqueNames(1)[0];
+    c.tableOperations().create(tableName);
+
+    Configuration conf = new Configuration();
+    AccumuloConfiguration aconf = new ServerConfigurationFactory(c.getInstance())
+        .getSystemConfiguration();
+    FileSystem fs = getCluster().getFileSystem();
+    String rootPath = cluster.getTemporaryPath().toString();
+
+    String dir = rootPath + "/testSingleTabletSingleFileNoSplits-" + getUniqueNames(1)[0];
+    Path bulkDir = new Path(dir);
+
+    fs.delete(bulkDir, true);
+
+    FileSKVWriter writer1 = FileOperations.getInstance().newWriterBuilder()
+        .forFile(dir + "/f1." + RFile.EXTENSION, fs, conf).withTableConfiguration(aconf).build();
+    writer1.startDefaultLocalityGroup();
+    writeData(writer1, 0, 333);
+    writer1.close();
+
+    c.tableOperations().addFilesTo(tableName).from(dir).load();
+  }
+
+  @Test
+  public void testBadPermissions() throws Exception {
+    Connector c = getConnector();
+    String tableName = getUniqueNames(1)[0];
+    c.tableOperations().create(tableName);
+    addSplits(tableName, "0333");
+
+    Configuration conf = new Configuration();
+    AccumuloConfiguration aconf = new ServerConfigurationFactory(c.getInstance())
+        .getSystemConfiguration();
+    FileSystem fs = getCluster().getFileSystem();
+    String rootPath = cluster.getTemporaryPath().toString();
+
+    String dir = rootPath + "/testBadPermissions-" + getUniqueNames(1)[0];
+    Path bulkDir = new Path(dir);
+
+    fs.delete(bulkDir, true);
+
+    FileSKVWriter writer1 = FileOperations.getInstance().newWriterBuilder()
+        .forFile(dir + "/f1." + RFile.EXTENSION, fs, conf).withTableConfiguration(aconf).build();
+    writer1.startDefaultLocalityGroup();
+    writeData(writer1, 0, 333);
+    writer1.close();
+
+    Path rFilePath = new Path(dir, "f1." + RFile.EXTENSION);
+    FsPermission originalPerms = fs.getFileStatus(rFilePath).getPermission();
+    fs.setPermission(rFilePath, FsPermission.valueOf("----------"));
+    try {
+      c.tableOperations().addFilesTo(tableName).from(dir).load();
+    } catch (Exception e) {
+      Throwable cause = e.getCause();
+      if (!(cause instanceof FileNotFoundException)
+          && !(cause.getCause() instanceof FileNotFoundException))
+        fail("Expected FileNotFoundException but threw " + e.getCause());
+    } finally {
+      fs.setPermission(rFilePath, originalPerms);
+    }
+
+    originalPerms = fs.getFileStatus(bulkDir).getPermission();
+    fs.setPermission(bulkDir, FsPermission.valueOf("dr--r--r--"));
+    try {
+      c.tableOperations().addFilesTo(tableName).from(dir).load();
+    } catch (AccumuloException ae) {
+      if (!(ae.getCause() instanceof FileNotFoundException))
+        fail("Expected FileNotFoundException but threw " + ae.getCause());
+    } finally {
+      fs.setPermission(bulkDir, originalPerms);
+    }
+  }
+
+  @Test
+  public void testBulkFile() throws Exception {
+    Connector c = getConnector();
+    String tableName = getUniqueNames(1)[0];
+    c.tableOperations().create(tableName);
+    addSplits(tableName, "0333 0666 0999 1333 1666");
+
+    Configuration conf = new Configuration();
+    AccumuloConfiguration aconf = new ServerConfigurationFactory(c.getInstance())
+        .getSystemConfiguration();
+    FileSystem fs = getCluster().getFileSystem();
+
+    String rootPath = cluster.getTemporaryPath().toString();
+
+    String dir = rootPath + "/testBulkFile-" + getUniqueNames(1)[0];
+
+    fs.delete(new Path(dir), true);
+
+    // TODO verify that data gets loaded in appropriate Tablets
+    // 1 Tablet 0333-null
+    FileSKVWriter writer1 = FileOperations.getInstance().newWriterBuilder()
+        .forFile(dir + "/f1." + RFile.EXTENSION, fs, conf).withTableConfiguration(aconf).build();
+    writer1.startDefaultLocalityGroup();
+    writeData(writer1, 0, 333);
+    writer1.close();
+
+    // 2 Tablets 0666-0334, 0999-0667
+    FileSKVWriter writer2 = FileOperations.getInstance().newWriterBuilder()
+        .forFile(dir + "/f2." + RFile.EXTENSION, fs, conf).withTableConfiguration(aconf).build();
+    writer2.startDefaultLocalityGroup();
+    writeData(writer2, 334, 999);
+    writer2.close();
+
+    // 2 Tablets 1333-1000, 1666-1334
+    FileSKVWriter writer3 = FileOperations.getInstance().newWriterBuilder()
+        .forFile(dir + "/f3." + RFile.EXTENSION, fs, conf).withTableConfiguration(aconf).build();
+    writer3.startDefaultLocalityGroup();
+    // writeData(writer3, 1000, 1999);
+    writeData(writer3, 1000, 1499);
+    writer3.close();
+
+    // 2 Tablets 1666-1334, >1666
+    FileSKVWriter writer4 = FileOperations.getInstance().newWriterBuilder()
+        .forFile(dir + "/f4." + RFile.EXTENSION, fs, conf).withTableConfiguration(aconf).build();
+    writer4.startDefaultLocalityGroup();
+    writeData(writer4, 1500, 1999);
+    writer4.close();
+
+    // TODO test c.tableOperations().offline(tableName, true);
+    c.tableOperations().addFilesTo(tableName).from(dir).load();
+
+    verifyData(tableName, 0, 1999);
+  }
+
+  private void addSplits(String tableName, String splitString) throws Exception {
+    SortedSet<Text> splits = new TreeSet<>();
+    for (String split : splitString.split(" "))
+      splits.add(new Text(split));
+    getConnector().tableOperations().addSplits(tableName, splits);
+  }
+
+  private void verifyData(String table, int s, int e) throws Exception {
+    try (Scanner scanner = getConnector().createScanner(table, Authorizations.EMPTY)) {
+
+      Iterator<Entry<Key,Value>> iter = scanner.iterator();
+
+      for (int i = s; i <= e; i++) {
+        if (!iter.hasNext())
+          throw new Exception("row " + i + " not found");
+
+        Entry<Key,Value> entry = iter.next();
+
+        String row = String.format("%04d", i);
+
+        if (!entry.getKey().getRow().equals(new Text(row)))
+          throw new Exception("unexpected row " + entry.getKey() + " " + i);
+
+        if (Integer.parseInt(entry.getValue().toString()) != i)
+          throw new Exception("unexpected value " + entry + " " + i);
+      }
+
+      if (iter.hasNext())
+        throw new Exception("found more than expected " + iter.next());
+    }
+  }
+
+  private void writeData(FileSKVWriter w, int s, int e) throws Exception {
+    for (int i = s; i <= e; i++) {
+      w.append(new Key(new Text(String.format("%04d", i))),
+          new Value(Integer.toString(i).getBytes(UTF_8)));
+    }
+  }
+
+}
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/SslIT.java b/test/src/main/java/org/apache/accumulo/test/functional/SslIT.java
index f98645a5bf..def280b605 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/SslIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/SslIT.java
@@ -63,7 +63,7 @@ public void adminStop() throws Exception {
   public void bulk() throws Exception {
     BulkIT.runTest(getConnector(), cluster.getFileSystem(),
         new Path(getCluster().getConfig().getDir().getAbsolutePath(), "tmp"), "root",
-        getUniqueNames(1)[0], this.getClass().getName(), testName.getMethodName());
+        getUniqueNames(1)[0], this.getClass().getName(), testName.getMethodName(), true);
   }
 
   @Test
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 080e658a95..0f69f4cd9d 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
@@ -122,6 +122,10 @@ public UpdateErrors closeUpdate(TInfo tinfo, long updateID) {
       return null;
     }
 
+    @Override
+    public void loadFiles(TInfo tinfo, TCredentials credentials, long tid, TKeyExtent tke,
+        String dir, Map<String,MapFileInfo> fileMap, boolean setTime) {}
+
     @Override
     public void closeMultiScan(TInfo tinfo, long scanID) {}
 


 

----------------------------------------------------------------
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