You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@accumulo.apache.org by ct...@apache.org on 2023/05/10 14:08:01 UTC

[accumulo] branch main updated: Remove broken support for map files (#3378)

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

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


The following commit(s) were added to refs/heads/main by this push:
     new fdabdc5133 Remove broken support for map files (#3378)
fdabdc5133 is described below

commit fdabdc5133dd262316bac5d71a832f332831a61a
Author: Christopher Tubbs <ct...@apache.org>
AuthorDate: Wed May 10 10:07:54 2023 -0400

    Remove broken support for map files (#3378)
    
    * Remove support for Hadoop map files for tablet data, as this hasn't
      worked since 1.5.0 anyway. Now, an error should be thrown, as the file
      type is no longer recognized. Replaced special handling for map file
      extension or directory with a logged warning, error, or allow an
      exception to be thrown, depending on the situation.
    * Rename 'mapFile' variables and related methods to the more generic
      'dataFile' or equivalent, to reduce confusion, since we no longer use
      MyMapFile or Hadoop MapFile, and only use RFile for storing tablet
      data files. This was done once before when RFile was initially added
      to the code base, which is why many of our variables and types already
      used the term 'dataFile', but evidently, not all of it had been
      renamed, or some of that terminology leaked back in. Some of it was
      added as part of recent Compaction changes, but there may have been
      earlier times this language was added back in. This change here resets
      us to the generic language for tablet data files.
    * Rename and relocate tabletingest-specific Thrift struct MapFileInfo to
      DataFileInfo under the tabletingest.thrift namespace. This only
      affects the bulk load Manager FATE operation and tablet servers. End
      user clients did not use this API, so it will not affect them.
    * Rename loadMapFile variable to loadMappingFile to make it clear that
      it's the file that stores the mapping of files for bulk loads, rather
      than a file in the MapFile format.
    * Add new validator to `table.file.type` value to prevent it from being
      anything other than the currently supported 'rf' extension for RFiles,
      which is also the current default (and the only value that would have
      worked anyway).
---
 .../java/org/apache/accumulo/core/Constants.java   |   1 -
 .../accumulo/core/clientImpl/bulk/BulkImport.java  |   6 +-
 .../core/clientImpl/bulk/LoadMappingIterator.java  |   4 +-
 .../org/apache/accumulo/core/conf/Property.java    |   2 +-
 .../apache/accumulo/core/conf/PropertyType.java    |   7 +-
 .../accumulo/core/file/BloomFilterLayer.java       |   2 +-
 .../accumulo/core/file/DispatchingFileFactory.java |  10 +-
 .../apache/accumulo/core/file/FileOperations.java  |   3 +-
 .../accumulo/core/file/map/MapFileOperations.java  | 186 ---------------------
 .../apache/accumulo/core/file/map/MapFileUtil.java |  52 ------
 .../core/iteratorsImpl/system/MapFileIterator.java | 176 -------------------
 .../thrift/DataFileInfo.java}                      |  58 +++----
 .../thrift/TabletIngestClientService.java          |  86 +++++-----
 core/src/main/thrift/data.thrift                   |   4 -
 core/src/main/thrift/tabletingest.thrift           |   6 +-
 .../accumulo/core/conf/PropertyTypeTest.java       |   7 +
 .../iterators/user/IndexedDocIteratorTest.java     |   2 +-
 .../accumulo/server/compaction/FileCompactor.java  |  22 +--
 .../org/apache/accumulo/server/util/FileUtil.java  |  90 +++++-----
 .../server/util/RemoveEntriesForMissingFiles.java  |   2 +-
 .../manager/tableOps/bulkVer2/LoadFiles.java       |  12 +-
 .../accumulo/manager/tableOps/delete/CleanUp.java  |   2 +-
 .../tableOps/tableImport/MapImportFileNames.java   |   6 +-
 .../accumulo/tserver/TabletClientHandler.java      |  10 +-
 .../org/apache/accumulo/tserver/TabletServer.java  |   4 +-
 .../tserver/TabletServerResourceManager.java       |   6 +-
 .../accumulo/tserver/tablet/CompactableImpl.java   |   4 +-
 .../accumulo/tserver/tablet/DatafileManager.java   |  10 +-
 .../tserver/tablet/MinorCompactionTask.java        |   2 +-
 .../accumulo/tserver/tablet/MinorCompactor.java    |   2 +-
 .../accumulo/tserver/tablet/ScanDataSource.java    |   8 +-
 .../apache/accumulo/tserver/tablet/Scanner.java    |   2 +-
 .../org/apache/accumulo/tserver/tablet/Tablet.java |  36 ++--
 .../apache/accumulo/tserver/tablet/TabletBase.java |   2 +-
 .../accumulo/test/functional/BulkFailureIT.java    |  39 +----
 .../test/functional/BulkSplitOptimizationIT.java   |   4 +-
 .../test/functional/FunctionalTestUtils.java       |   2 +-
 .../apache/accumulo/test/functional/MaxOpenIT.java |   2 +-
 .../accumulo/test/functional/SplitRecoveryIT.java  |  24 +--
 .../accumulo/test/performance/NullTserver.java     |   4 +-
 .../test/performance/scan/CollectTabletStats.java  |  11 +-
 41 files changed, 240 insertions(+), 678 deletions(-)

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 5d37166538..3c8da35361 100644
--- a/core/src/main/java/org/apache/accumulo/core/Constants.java
+++ b/core/src/main/java/org/apache/accumulo/core/Constants.java
@@ -116,7 +116,6 @@ public class Constants {
 
   public static final int MAX_DATA_TO_PRINT = 64;
   public static final String CORE_PACKAGE_NAME = "org.apache.accumulo.core";
-  public static final String MAPFILE_EXTENSION = "map";
   public static final String GENERATED_TABLET_DIRECTORY_PREFIX = "t-";
 
   public static final String EXPORT_METADATA_FILE = "metadata.bin";
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/bulk/BulkImport.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/bulk/BulkImport.java
index 8b9516ad7a..b788d63e98 100644
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/bulk/BulkImport.java
+++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/bulk/BulkImport.java
@@ -261,7 +261,7 @@ public class BulkImport implements ImportDestinationArguments, ImportMappingOpti
     long l;
   }
 
-  public static Map<KeyExtent,Long> estimateSizes(AccumuloConfiguration acuConf, Path mapFile,
+  public static Map<KeyExtent,Long> estimateSizes(AccumuloConfiguration acuConf, Path dataFile,
       long fileSize, Collection<KeyExtent> extents, FileSystem ns, Cache<String,Long> fileLenCache,
       CryptoService cs) throws IOException {
 
@@ -278,7 +278,7 @@ public class BulkImport implements ImportDestinationArguments, ImportMappingOpti
     Text row = new Text();
 
     FileSKVIterator index = FileOperations.getInstance().newIndexReaderBuilder()
-        .forFile(mapFile.toString(), ns, ns.getConf(), cs).withTableConfiguration(acuConf)
+        .forFile(dataFile.toString(), ns, ns.getConf(), cs).withTableConfiguration(acuConf)
         .withFileLenCache(fileLenCache).build();
 
     try {
@@ -302,7 +302,7 @@ public class BulkImport implements ImportDestinationArguments, ImportMappingOpti
           index.close();
         }
       } catch (IOException e) {
-        log.debug("Failed to close " + mapFile, e);
+        log.debug("Failed to close " + dataFile, e);
       }
     }
 
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/bulk/LoadMappingIterator.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/bulk/LoadMappingIterator.java
index d48d3cce6b..36b2d5b357 100644
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/bulk/LoadMappingIterator.java
+++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/bulk/LoadMappingIterator.java
@@ -46,9 +46,9 @@ public class LoadMappingIterator
   private static final Gson gson = ByteArrayToBase64TypeAdapter.createBase64Gson();
   private Map<String,String> renameMap;
 
-  LoadMappingIterator(TableId tableId, InputStream loadMapFile) throws IOException {
+  LoadMappingIterator(TableId tableId, InputStream loadMappingFile) throws IOException {
     this.tableId = tableId;
-    this.reader = new JsonReader(new BufferedReader(new InputStreamReader(loadMapFile, UTF_8)));
+    this.reader = new JsonReader(new BufferedReader(new InputStreamReader(loadMappingFile, UTF_8)));
     this.reader.beginArray();
   }
 
diff --git a/core/src/main/java/org/apache/accumulo/core/conf/Property.java b/core/src/main/java/org/apache/accumulo/core/conf/Property.java
index 7ddc6fc433..25786601c5 100644
--- a/core/src/main/java/org/apache/accumulo/core/conf/Property.java
+++ b/core/src/main/java/org/apache/accumulo/core/conf/Property.java
@@ -874,7 +874,7 @@ public enum Property {
       "The maximum amount of memory that will be used to cache results of a client query/scan. "
           + "Once this limit is reached, the buffered data is sent to the client.",
       "1.3.5"),
-  TABLE_FILE_TYPE("table.file.type", RFile.EXTENSION, PropertyType.STRING,
+  TABLE_FILE_TYPE("table.file.type", RFile.EXTENSION, PropertyType.FILENAME_EXT,
       "Change the type of file a table writes", "1.3.5"),
   TABLE_LOAD_BALANCER("table.balancer", "org.apache.accumulo.core.spi.balancer.SimpleLoadBalancer",
       PropertyType.STRING,
diff --git a/core/src/main/java/org/apache/accumulo/core/conf/PropertyType.java b/core/src/main/java/org/apache/accumulo/core/conf/PropertyType.java
index 49def8cf40..be84b019e0 100644
--- a/core/src/main/java/org/apache/accumulo/core/conf/PropertyType.java
+++ b/core/src/main/java/org/apache/accumulo/core/conf/PropertyType.java
@@ -29,6 +29,7 @@ import java.util.regex.Pattern;
 import java.util.stream.IntStream;
 import java.util.stream.Stream;
 
+import org.apache.accumulo.core.file.rfile.RFile;
 import org.apache.commons.lang3.Range;
 import org.apache.hadoop.fs.Path;
 
@@ -135,7 +136,11 @@ public enum PropertyType {
   BOOLEAN("boolean", in(false, null, "true", "false"),
       "Has a value of either 'true' or 'false' (case-insensitive)"),
 
-  URI("uri", x -> true, "A valid URI");
+  URI("uri", x -> true, "A valid URI"),
+
+  FILENAME_EXT("file name extension", in(true, RFile.EXTENSION),
+      "One of the currently supported filename extensions for storing table data files. "
+          + "Currently, only " + RFile.EXTENSION + " is supported.");
 
   private String shortname, format;
   // Field is transient because enums are Serializable, but Predicates aren't necessarily,
diff --git a/core/src/main/java/org/apache/accumulo/core/file/BloomFilterLayer.java b/core/src/main/java/org/apache/accumulo/core/file/BloomFilterLayer.java
index 39cd7d2d13..59af8025f5 100644
--- a/core/src/main/java/org/apache/accumulo/core/file/BloomFilterLayer.java
+++ b/core/src/main/java/org/apache/accumulo/core/file/BloomFilterLayer.java
@@ -217,7 +217,7 @@ public class BloomFilterLayer {
       final String context = ClassLoaderUtil.tableContext(acuconf);
 
       loadTask = () -> {
-        // no need to load the bloom filter if the map file is closed
+        // no need to load the bloom filter if the data file is closed
         if (closed) {
           return;
         }
diff --git a/core/src/main/java/org/apache/accumulo/core/file/DispatchingFileFactory.java b/core/src/main/java/org/apache/accumulo/core/file/DispatchingFileFactory.java
index fb16b85741..bae4c433af 100644
--- a/core/src/main/java/org/apache/accumulo/core/file/DispatchingFileFactory.java
+++ b/core/src/main/java/org/apache/accumulo/core/file/DispatchingFileFactory.java
@@ -20,9 +20,7 @@ package org.apache.accumulo.core.file;
 
 import java.io.IOException;
 
-import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.conf.Property;
-import org.apache.accumulo.core.file.map.MapFileOperations;
 import org.apache.accumulo.core.file.rfile.RFile;
 import org.apache.accumulo.core.file.rfile.RFileOperations;
 import org.apache.accumulo.core.summary.SummaryWriter;
@@ -36,9 +34,6 @@ class DispatchingFileFactory extends FileOperations {
     Path p = new Path(file);
     String name = p.getName();
 
-    if (name.startsWith(Constants.MAPFILE_EXTENSION + "_")) {
-      return new MapFileOperations();
-    }
     String[] sp = name.split("\\.");
 
     if (sp.length < 2) {
@@ -47,10 +42,7 @@ class DispatchingFileFactory extends FileOperations {
 
     String extension = sp[sp.length - 1];
 
-    if (extension.equals(Constants.MAPFILE_EXTENSION)
-        || extension.equals(Constants.MAPFILE_EXTENSION + "_tmp")) {
-      return new MapFileOperations();
-    } else if (extension.equals(RFile.EXTENSION) || extension.equals(RFile.EXTENSION + "_tmp")) {
+    if (extension.equals(RFile.EXTENSION) || extension.equals(RFile.EXTENSION + "_tmp")) {
       return new RFileOperations();
     } else {
       throw new IllegalArgumentException("File type " + extension + " not supported");
diff --git a/core/src/main/java/org/apache/accumulo/core/file/FileOperations.java b/core/src/main/java/org/apache/accumulo/core/file/FileOperations.java
index 46a4359cb0..db65923641 100644
--- a/core/src/main/java/org/apache/accumulo/core/file/FileOperations.java
+++ b/core/src/main/java/org/apache/accumulo/core/file/FileOperations.java
@@ -45,8 +45,7 @@ public abstract class FileOperations {
   private static final String HADOOP_JOBHISTORY_LOCATION = "_logs"; // dir related to
                                                                     // hadoop.job.history.user.location
 
-  private static final Set<String> validExtensions =
-      Set.of(Constants.MAPFILE_EXTENSION, RFile.EXTENSION);
+  private static final Set<String> validExtensions = Set.of(RFile.EXTENSION);
 
   // Sometimes we want to know what files accumulo bulk processing creates
   private static final Set<String> bulkWorkingFiles =
diff --git a/core/src/main/java/org/apache/accumulo/core/file/map/MapFileOperations.java b/core/src/main/java/org/apache/accumulo/core/file/map/MapFileOperations.java
deleted file mode 100644
index c7da8f4784..0000000000
--- a/core/src/main/java/org/apache/accumulo/core/file/map/MapFileOperations.java
+++ /dev/null
@@ -1,186 +0,0 @@
-/*
- * 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
- *
- *   https://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.file.map;
-
-import java.io.DataInputStream;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Map;
-import java.util.concurrent.atomic.AtomicBoolean;
-
-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.Value;
-import org.apache.accumulo.core.file.FileOperations;
-import org.apache.accumulo.core.file.FileSKVIterator;
-import org.apache.accumulo.core.file.FileSKVWriter;
-import org.apache.accumulo.core.file.blockfile.impl.CacheProvider;
-import org.apache.accumulo.core.iterators.IteratorEnvironment;
-import org.apache.accumulo.core.iterators.SortedKeyValueIterator;
-import org.apache.accumulo.core.iteratorsImpl.system.MapFileIterator;
-import org.apache.accumulo.core.iteratorsImpl.system.SequenceFileIterator;
-import org.apache.accumulo.core.sample.impl.SamplerConfigurationImpl;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.MapFile;
-
-public class MapFileOperations extends FileOperations {
-
-  public static class RangeIterator implements FileSKVIterator {
-
-    SortedKeyValueIterator<Key,Value> reader;
-    private Range range;
-    private boolean hasTop;
-
-    public RangeIterator(SortedKeyValueIterator<Key,Value> reader) {
-      this.reader = reader;
-    }
-
-    @Override
-    public void close() throws IOException {
-      ((FileSKVIterator) reader).close();
-    }
-
-    @Override
-    public Key getFirstKey() throws IOException {
-      return ((FileSKVIterator) reader).getFirstKey();
-    }
-
-    @Override
-    public Key getLastKey() throws IOException {
-      return ((FileSKVIterator) reader).getLastKey();
-    }
-
-    @Override
-    public DataInputStream getMetaStore(String name) throws IOException {
-      return ((FileSKVIterator) reader).getMetaStore(name);
-    }
-
-    @Override
-    public SortedKeyValueIterator<Key,Value> deepCopy(IteratorEnvironment env) {
-      return new RangeIterator(reader.deepCopy(env));
-    }
-
-    @Override
-    public Key getTopKey() {
-      if (!hasTop) {
-        throw new IllegalStateException();
-      }
-      return reader.getTopKey();
-    }
-
-    @Override
-    public Value getTopValue() {
-      if (!hasTop) {
-        throw new IllegalStateException();
-      }
-      return reader.getTopValue();
-    }
-
-    @Override
-    public boolean hasTop() {
-      return hasTop;
-    }
-
-    @Override
-    public void init(SortedKeyValueIterator<Key,Value> source, Map<String,String> options,
-        IteratorEnvironment env) {
-      throw new UnsupportedOperationException();
-    }
-
-    @Override
-    public void next() throws IOException {
-      if (!hasTop) {
-        throw new IllegalStateException();
-      }
-      reader.next();
-      hasTop = reader.hasTop() && !range.afterEndKey(reader.getTopKey());
-    }
-
-    @Override
-    public void seek(Range range, Collection<ByteSequence> columnFamilies, boolean inclusive)
-        throws IOException {
-      reader.seek(range, columnFamilies, inclusive);
-      this.range = range;
-
-      hasTop = reader.hasTop() && !range.afterEndKey(reader.getTopKey());
-
-      while (hasTop() && range.beforeStartKey(getTopKey())) {
-        next();
-      }
-    }
-
-    @Override
-    public void closeDeepCopies() throws IOException {
-      ((FileSKVIterator) reader).closeDeepCopies();
-    }
-
-    @Override
-    public void setInterruptFlag(AtomicBoolean flag) {
-      ((FileSKVIterator) reader).setInterruptFlag(flag);
-    }
-
-    @Override
-    public FileSKVIterator getSample(SamplerConfigurationImpl sampleConfig) {
-      return ((FileSKVIterator) reader).getSample(sampleConfig);
-    }
-
-    @Override
-    public void setCacheProvider(CacheProvider cacheProvider) {}
-  }
-
-  @Override
-  protected FileSKVIterator openReader(FileOptions options) throws IOException {
-    FileSKVIterator iter = new RangeIterator(new MapFileIterator(options.getFileSystem(),
-        options.getFilename(), options.getConfiguration()));
-    if (options.isSeekToBeginning()) {
-      iter.seek(new Range(new Key(), null), new ArrayList<>(), false);
-    }
-    return iter;
-  }
-
-  @Override
-  protected FileSKVWriter openWriter(FileOptions options) {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  protected FileSKVIterator openIndex(FileOptions options) throws IOException {
-    return new SequenceFileIterator(MapFileUtil.openIndex(options.getConfiguration(),
-        options.getFileSystem(), new Path(options.getFilename())), false);
-  }
-
-  @Override
-  protected long getFileSize(FileOptions options) throws IOException {
-    return options.getFileSystem()
-        .getFileStatus(new Path(options.getFilename() + "/" + MapFile.DATA_FILE_NAME)).getLen();
-  }
-
-  @Override
-  protected FileSKVIterator openScanReader(FileOptions options) throws IOException {
-    MapFileIterator mfIter = new MapFileIterator(options.getFileSystem(), options.getFilename(),
-        options.getConfiguration());
-
-    FileSKVIterator iter = new RangeIterator(mfIter);
-    iter.seek(options.getRange(), options.getColumnFamilies(), options.isRangeInclusive());
-
-    return iter;
-  }
-}
diff --git a/core/src/main/java/org/apache/accumulo/core/file/map/MapFileUtil.java b/core/src/main/java/org/apache/accumulo/core/file/map/MapFileUtil.java
deleted file mode 100644
index 7fd7276fd8..0000000000
--- a/core/src/main/java/org/apache/accumulo/core/file/map/MapFileUtil.java
+++ /dev/null
@@ -1,52 +0,0 @@
-/*
- * 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
- *
- *   https://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.file.map;
-
-import java.io.IOException;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.MapFile;
-import org.apache.hadoop.io.SequenceFile;
-
-public class MapFileUtil {
-  public static MapFile.Reader openMapFile(FileSystem fs, String dirName, Configuration conf)
-      throws IOException {
-    MapFile.Reader mfr = null;
-    try {
-      mfr = new MapFile.Reader(fs.makeQualified(new Path(dirName)), conf);
-      return mfr;
-    } catch (IOException e) {
-      throw e;
-    }
-  }
-
-  public static SequenceFile.Reader openIndex(Configuration conf, FileSystem fs, Path mapFile)
-      throws IOException {
-    Path indexPath = new Path(mapFile, MapFile.INDEX_FILE_NAME);
-    SequenceFile.Reader index = null;
-    try {
-      index = new SequenceFile.Reader(conf, SequenceFile.Reader.file(fs.makeQualified(indexPath)));
-      return index;
-    } catch (IOException e) {
-      throw e;
-    }
-  }
-}
diff --git a/core/src/main/java/org/apache/accumulo/core/iteratorsImpl/system/MapFileIterator.java b/core/src/main/java/org/apache/accumulo/core/iteratorsImpl/system/MapFileIterator.java
deleted file mode 100644
index a0eca19ba6..0000000000
--- a/core/src/main/java/org/apache/accumulo/core/iteratorsImpl/system/MapFileIterator.java
+++ /dev/null
@@ -1,176 +0,0 @@
-/*
- * 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
- *
- *   https://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.iteratorsImpl.system;
-
-import java.io.DataInputStream;
-import java.io.IOException;
-import java.io.UncheckedIOException;
-import java.util.Collection;
-import java.util.Map;
-import java.util.concurrent.atomic.AtomicBoolean;
-
-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.Value;
-import org.apache.accumulo.core.file.FileSKVIterator;
-import org.apache.accumulo.core.file.NoSuchMetaStoreException;
-import org.apache.accumulo.core.file.blockfile.impl.CacheProvider;
-import org.apache.accumulo.core.file.map.MapFileUtil;
-import org.apache.accumulo.core.iterators.IteratorEnvironment;
-import org.apache.accumulo.core.iterators.SortedKeyValueIterator;
-import org.apache.accumulo.core.sample.impl.SamplerConfigurationImpl;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.MapFile.Reader;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-public class MapFileIterator implements FileSKVIterator {
-  private static final Logger log = LoggerFactory.getLogger(MapFileIterator.class);
-
-  private final Reader reader;
-  private Value topValue;
-  private Key topKey;
-  private AtomicBoolean interruptFlag;
-  private int interruptCheckCount = 0;
-  private final FileSystem fs;
-  private final String dirName;
-  private final Configuration hadoopConf;
-
-  public MapFileIterator(FileSystem fs, String dir, Configuration conf) throws IOException {
-    this.reader = MapFileUtil.openMapFile(fs, dir, conf);
-    this.fs = fs;
-    this.dirName = dir;
-    this.hadoopConf = conf;
-  }
-
-  @Override
-  public void setInterruptFlag(AtomicBoolean flag) {
-    this.interruptFlag = flag;
-  }
-
-  @Override
-  public void init(SortedKeyValueIterator<Key,Value> source, Map<String,String> options,
-      IteratorEnvironment env) throws IOException {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public boolean hasTop() {
-    return topKey != null;
-  }
-
-  @Override
-  public void next() throws IOException {
-    if (interruptFlag != null && interruptCheckCount++ % 100 == 0 && interruptFlag.get()) {
-      throw new IterationInterruptedException();
-    }
-
-    reader.next(topKey, topValue);
-  }
-
-  @Override
-  public void seek(Range range, Collection<ByteSequence> columnFamilies, boolean inclusive)
-      throws IOException {
-    if (!columnFamilies.isEmpty() || inclusive) {
-      throw new IllegalArgumentException("I do not know how to filter column families");
-    }
-
-    if (range == null) {
-      throw new IllegalArgumentException("Cannot seek to null range");
-    }
-
-    if (interruptFlag != null && interruptFlag.get()) {
-      throw new IterationInterruptedException();
-    }
-
-    Key key = range.getStartKey();
-    if (key == null) {
-      key = new Key();
-    }
-
-    reader.seek(key);
-
-    while (hasTop() && range.beforeStartKey(getTopKey())) {
-      next();
-    }
-  }
-
-  @Override
-  public Key getTopKey() {
-    return topKey;
-  }
-
-  @Override
-  public Value getTopValue() {
-    return topValue;
-  }
-
-  @Override
-  public SortedKeyValueIterator<Key,Value> deepCopy(IteratorEnvironment env) {
-    try {
-      MapFileIterator other = new MapFileIterator(fs, dirName, hadoopConf);
-      other.setInterruptFlag(interruptFlag);
-      log.debug("deep copying MapFile: {} -> {}", this, other);
-      return other;
-    } catch (IOException e) {
-      log.error("failed to clone map file reader", e);
-      throw new UncheckedIOException(e);
-    }
-  }
-
-  @Override
-  public Key getFirstKey() throws IOException {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public Key getLastKey() throws IOException {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public DataInputStream getMetaStore(String name) throws IOException {
-    Path path = new Path(this.dirName, name);
-    if (!fs.exists(path)) {
-      throw new NoSuchMetaStoreException("name = " + name);
-    }
-    return fs.open(path);
-  }
-
-  @Override
-  public void closeDeepCopies() throws IOException {
-    // nothing to do, deep copies are externally managed/closed
-  }
-
-  @Override
-  public void close() throws IOException {
-    reader.close();
-  }
-
-  @Override
-  public FileSKVIterator getSample(SamplerConfigurationImpl sampleConfig) {
-    return null;
-  }
-
-  @Override
-  public void setCacheProvider(CacheProvider cacheProvider) {}
-}
diff --git a/core/src/main/thrift-gen-java/org/apache/accumulo/core/dataImpl/thrift/MapFileInfo.java b/core/src/main/thrift-gen-java/org/apache/accumulo/core/tabletingest/thrift/DataFileInfo.java
similarity index 86%
rename from core/src/main/thrift-gen-java/org/apache/accumulo/core/dataImpl/thrift/MapFileInfo.java
rename to core/src/main/thrift-gen-java/org/apache/accumulo/core/tabletingest/thrift/DataFileInfo.java
index 0d7de77a00..12276fd878 100644
--- a/core/src/main/thrift-gen-java/org/apache/accumulo/core/dataImpl/thrift/MapFileInfo.java
+++ b/core/src/main/thrift-gen-java/org/apache/accumulo/core/tabletingest/thrift/DataFileInfo.java
@@ -22,16 +22,16 @@
  * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
  *  @generated
  */
-package org.apache.accumulo.core.dataImpl.thrift;
+package org.apache.accumulo.core.tabletingest.thrift;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"})
-public class MapFileInfo implements org.apache.thrift.TBase<MapFileInfo, MapFileInfo._Fields>, java.io.Serializable, Cloneable, Comparable<MapFileInfo> {
-  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("MapFileInfo");
+public class DataFileInfo implements org.apache.thrift.TBase<DataFileInfo, DataFileInfo._Fields>, java.io.Serializable, Cloneable, Comparable<DataFileInfo> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("DataFileInfo");
 
   private static final org.apache.thrift.protocol.TField ESTIMATED_SIZE_FIELD_DESC = new org.apache.thrift.protocol.TField("estimatedSize", org.apache.thrift.protocol.TType.I64, (short)1);
 
-  private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new MapFileInfoStandardSchemeFactory();
-  private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new MapFileInfoTupleSchemeFactory();
+  private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new DataFileInfoStandardSchemeFactory();
+  private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new DataFileInfoTupleSchemeFactory();
 
   public long estimatedSize; // required
 
@@ -106,13 +106,13 @@ public class MapFileInfo implements org.apache.thrift.TBase<MapFileInfo, MapFile
     tmpMap.put(_Fields.ESTIMATED_SIZE, new org.apache.thrift.meta_data.FieldMetaData("estimatedSize", org.apache.thrift.TFieldRequirementType.DEFAULT, 
         new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64)));
     metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
-    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(MapFileInfo.class, metaDataMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(DataFileInfo.class, metaDataMap);
   }
 
-  public MapFileInfo() {
+  public DataFileInfo() {
   }
 
-  public MapFileInfo(
+  public DataFileInfo(
     long estimatedSize)
   {
     this();
@@ -123,14 +123,14 @@ public class MapFileInfo implements org.apache.thrift.TBase<MapFileInfo, MapFile
   /**
    * Performs a deep copy on <i>other</i>.
    */
-  public MapFileInfo(MapFileInfo other) {
+  public DataFileInfo(DataFileInfo other) {
     __isset_bitfield = other.__isset_bitfield;
     this.estimatedSize = other.estimatedSize;
   }
 
   @Override
-  public MapFileInfo deepCopy() {
-    return new MapFileInfo(this);
+  public DataFileInfo deepCopy() {
+    return new DataFileInfo(this);
   }
 
   @Override
@@ -143,7 +143,7 @@ public class MapFileInfo implements org.apache.thrift.TBase<MapFileInfo, MapFile
     return this.estimatedSize;
   }
 
-  public MapFileInfo setEstimatedSize(long estimatedSize) {
+  public DataFileInfo setEstimatedSize(long estimatedSize) {
     this.estimatedSize = estimatedSize;
     setEstimatedSizeIsSet(true);
     return this;
@@ -203,12 +203,12 @@ public class MapFileInfo implements org.apache.thrift.TBase<MapFileInfo, MapFile
 
   @Override
   public boolean equals(java.lang.Object that) {
-    if (that instanceof MapFileInfo)
-      return this.equals((MapFileInfo)that);
+    if (that instanceof DataFileInfo)
+      return this.equals((DataFileInfo)that);
     return false;
   }
 
-  public boolean equals(MapFileInfo that) {
+  public boolean equals(DataFileInfo that) {
     if (that == null)
       return false;
     if (this == that)
@@ -236,7 +236,7 @@ public class MapFileInfo implements org.apache.thrift.TBase<MapFileInfo, MapFile
   }
 
   @Override
-  public int compareTo(MapFileInfo other) {
+  public int compareTo(DataFileInfo other) {
     if (!getClass().equals(other.getClass())) {
       return getClass().getName().compareTo(other.getClass().getName());
     }
@@ -274,7 +274,7 @@ public class MapFileInfo implements org.apache.thrift.TBase<MapFileInfo, MapFile
 
   @Override
   public java.lang.String toString() {
-    java.lang.StringBuilder sb = new java.lang.StringBuilder("MapFileInfo(");
+    java.lang.StringBuilder sb = new java.lang.StringBuilder("DataFileInfo(");
     boolean first = true;
 
     sb.append("estimatedSize:");
@@ -307,17 +307,17 @@ public class MapFileInfo implements org.apache.thrift.TBase<MapFileInfo, MapFile
     }
   }
 
-  private static class MapFileInfoStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+  private static class DataFileInfoStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
     @Override
-    public MapFileInfoStandardScheme getScheme() {
-      return new MapFileInfoStandardScheme();
+    public DataFileInfoStandardScheme getScheme() {
+      return new DataFileInfoStandardScheme();
     }
   }
 
-  private static class MapFileInfoStandardScheme extends org.apache.thrift.scheme.StandardScheme<MapFileInfo> {
+  private static class DataFileInfoStandardScheme extends org.apache.thrift.scheme.StandardScheme<DataFileInfo> {
 
     @Override
-    public void read(org.apache.thrift.protocol.TProtocol iprot, MapFileInfo struct) throws org.apache.thrift.TException {
+    public void read(org.apache.thrift.protocol.TProtocol iprot, DataFileInfo struct) throws org.apache.thrift.TException {
       org.apache.thrift.protocol.TField schemeField;
       iprot.readStructBegin();
       while (true)
@@ -347,7 +347,7 @@ public class MapFileInfo implements org.apache.thrift.TBase<MapFileInfo, MapFile
     }
 
     @Override
-    public void write(org.apache.thrift.protocol.TProtocol oprot, MapFileInfo struct) throws org.apache.thrift.TException {
+    public void write(org.apache.thrift.protocol.TProtocol oprot, DataFileInfo struct) throws org.apache.thrift.TException {
       struct.validate();
 
       oprot.writeStructBegin(STRUCT_DESC);
@@ -360,17 +360,17 @@ public class MapFileInfo implements org.apache.thrift.TBase<MapFileInfo, MapFile
 
   }
 
-  private static class MapFileInfoTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+  private static class DataFileInfoTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
     @Override
-    public MapFileInfoTupleScheme getScheme() {
-      return new MapFileInfoTupleScheme();
+    public DataFileInfoTupleScheme getScheme() {
+      return new DataFileInfoTupleScheme();
     }
   }
 
-  private static class MapFileInfoTupleScheme extends org.apache.thrift.scheme.TupleScheme<MapFileInfo> {
+  private static class DataFileInfoTupleScheme extends org.apache.thrift.scheme.TupleScheme<DataFileInfo> {
 
     @Override
-    public void write(org.apache.thrift.protocol.TProtocol prot, MapFileInfo struct) throws org.apache.thrift.TException {
+    public void write(org.apache.thrift.protocol.TProtocol prot, DataFileInfo 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.isSetEstimatedSize()) {
@@ -383,7 +383,7 @@ public class MapFileInfo implements org.apache.thrift.TBase<MapFileInfo, MapFile
     }
 
     @Override
-    public void read(org.apache.thrift.protocol.TProtocol prot, MapFileInfo struct) throws org.apache.thrift.TException {
+    public void read(org.apache.thrift.protocol.TProtocol prot, DataFileInfo struct) throws org.apache.thrift.TException {
       org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
       java.util.BitSet incoming = iprot.readBitSet(1);
       if (incoming.get(0)) {
diff --git a/core/src/main/thrift-gen-java/org/apache/accumulo/core/tabletingest/thrift/TabletIngestClientService.java b/core/src/main/thrift-gen-java/org/apache/accumulo/core/tabletingest/thrift/TabletIngestClientService.java
index 590131a5ce..49fc226ec6 100644
--- a/core/src/main/thrift-gen-java/org/apache/accumulo/core/tabletingest/thrift/TabletIngestClientService.java
+++ b/core/src/main/thrift-gen-java/org/apache/accumulo/core/tabletingest/thrift/TabletIngestClientService.java
@@ -45,7 +45,7 @@ public class TabletIngestClientService {
 
     public void closeConditionalUpdate(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, long sessID) throws org.apache.thrift.TException;
 
-    public void loadFiles(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, long tid, java.lang.String dir, java.util.Map<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent,java.util.Map<java.lang.String,org.apache.accumulo.core.dataImpl.thrift.MapFileInfo>> files, boolean setTime) throws org.apache.thrift.TException;
+    public void loadFiles(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, long tid, java.lang.String dir, java.util.Map<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent,java.util.Map<java.lang.String,DataFileInfo>> files, boolean setTime) throws org.apache.thrift.TException;
 
   }
 
@@ -67,7 +67,7 @@ public class TabletIngestClientService {
 
     public void closeConditionalUpdate(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, long sessID, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
 
-    public void loadFiles(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, long tid, java.lang.String dir, java.util.Map<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent,java.util.Map<java.lang.String,org.apache.accumulo.core.dataImpl.thrift.MapFileInfo>> files, boolean setTime, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
+    public void loadFiles(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, long tid, java.lang.String dir, java.util.Map<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent,java.util.Map<java.lang.String,DataFileInfo>> files, boolean setTime, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
 
   }
 
@@ -299,12 +299,12 @@ public class TabletIngestClientService {
     }
 
     @Override
-    public void loadFiles(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, long tid, java.lang.String dir, java.util.Map<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent,java.util.Map<java.lang.String,org.apache.accumulo.core.dataImpl.thrift.MapFileInfo>> files, boolean setTime) throws org.apache.thrift.TException
+    public void loadFiles(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, long tid, java.lang.String dir, java.util.Map<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent,java.util.Map<java.lang.String,DataFileInfo>> files, boolean setTime) throws org.apache.thrift.TException
     {
       send_loadFiles(tinfo, credentials, tid, dir, files, setTime);
     }
 
-    public void send_loadFiles(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, long tid, java.lang.String dir, java.util.Map<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent,java.util.Map<java.lang.String,org.apache.accumulo.core.dataImpl.thrift.MapFileInfo>> files, boolean setTime) throws org.apache.thrift.TException
+    public void send_loadFiles(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, long tid, java.lang.String dir, java.util.Map<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent,java.util.Map<java.lang.String,DataFileInfo>> files, boolean setTime) throws org.apache.thrift.TException
     {
       loadFiles_args args = new loadFiles_args();
       args.setTinfo(tinfo);
@@ -678,7 +678,7 @@ public class TabletIngestClientService {
     }
 
     @Override
-    public void loadFiles(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, long tid, java.lang.String dir, java.util.Map<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent,java.util.Map<java.lang.String,org.apache.accumulo.core.dataImpl.thrift.MapFileInfo>> files, boolean setTime, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
+    public void loadFiles(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, long tid, java.lang.String dir, java.util.Map<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent,java.util.Map<java.lang.String,DataFileInfo>> 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, dir, files, setTime, resultHandler, this, ___protocolFactory, ___transport);
       this.___currentMethod = method_call;
@@ -690,9 +690,9 @@ public class TabletIngestClientService {
       private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials;
       private long tid;
       private java.lang.String dir;
-      private java.util.Map<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent,java.util.Map<java.lang.String,org.apache.accumulo.core.dataImpl.thrift.MapFileInfo>> files;
+      private java.util.Map<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent,java.util.Map<java.lang.String,DataFileInfo>> files;
       private boolean setTime;
-      public loadFiles_call(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, long tid, java.lang.String dir, java.util.Map<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent,java.util.Map<java.lang.String,org.apache.accumulo.core.dataImpl.thrift.MapFileInfo>> files, boolean setTime, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.p [...]
+      public loadFiles_call(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, long tid, java.lang.String dir, java.util.Map<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent,java.util.Map<java.lang.String,DataFileInfo>> files, boolean setTime, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory [...]
         super(client, protocolFactory, transport, resultHandler, true);
         this.tinfo = tinfo;
         this.credentials = credentials;
@@ -9951,7 +9951,7 @@ public class TabletIngestClientService {
     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 DIR_FIELD_DESC = new org.apache.thrift.protocol.TField("dir", org.apache.thrift.protocol.TType.STRING, (short)4);
-    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)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)7);
     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)6);
 
     private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new loadFiles_argsStandardSchemeFactory();
@@ -9961,7 +9961,7 @@ public class TabletIngestClientService {
     public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials; // required
     public long tid; // required
     public @org.apache.thrift.annotation.Nullable java.lang.String dir; // required
-    public @org.apache.thrift.annotation.Nullable java.util.Map<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent,java.util.Map<java.lang.String,org.apache.accumulo.core.dataImpl.thrift.MapFileInfo>> files; // required
+    public @org.apache.thrift.annotation.Nullable java.util.Map<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent,java.util.Map<java.lang.String,DataFileInfo>> files; // required
     public boolean setTime; // required
 
     /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
@@ -9970,7 +9970,7 @@ public class TabletIngestClientService {
       CREDENTIALS((short)2, "credentials"),
       TID((short)3, "tid"),
       DIR((short)4, "dir"),
-      FILES((short)5, "files"),
+      FILES((short)7, "files"),
       SET_TIME((short)6, "setTime");
 
       private static final java.util.Map<java.lang.String, _Fields> byName = new java.util.HashMap<java.lang.String, _Fields>();
@@ -9995,7 +9995,7 @@ public class TabletIngestClientService {
             return TID;
           case 4: // DIR
             return DIR;
-          case 5: // FILES
+          case 7: // FILES
             return FILES;
           case 6: // SET_TIME
             return SET_TIME;
@@ -10061,7 +10061,7 @@ public class TabletIngestClientService {
               new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent.class), 
               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.dataImpl.thrift.MapFileInfo.class)))));
+                  new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, DataFileInfo.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);
@@ -10076,7 +10076,7 @@ public class TabletIngestClientService {
       org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials,
       long tid,
       java.lang.String dir,
-      java.util.Map<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent,java.util.Map<java.lang.String,org.apache.accumulo.core.dataImpl.thrift.MapFileInfo>> files,
+      java.util.Map<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent,java.util.Map<java.lang.String,DataFileInfo>> files,
       boolean setTime)
     {
       this();
@@ -10106,23 +10106,23 @@ public class TabletIngestClientService {
         this.dir = other.dir;
       }
       if (other.isSetFiles()) {
-        java.util.Map<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent,java.util.Map<java.lang.String,org.apache.accumulo.core.dataImpl.thrift.MapFileInfo>> __this__files = new java.util.HashMap<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent,java.util.Map<java.lang.String,org.apache.accumulo.core.dataImpl.thrift.MapFileInfo>>(other.files.size());
-        for (java.util.Map.Entry<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent, java.util.Map<java.lang.String,org.apache.accumulo.core.dataImpl.thrift.MapFileInfo>> other_element : other.files.entrySet()) {
+        java.util.Map<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent,java.util.Map<java.lang.String,DataFileInfo>> __this__files = new java.util.HashMap<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent,java.util.Map<java.lang.String,DataFileInfo>>(other.files.size());
+        for (java.util.Map.Entry<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent, java.util.Map<java.lang.String,DataFileInfo>> other_element : other.files.entrySet()) {
 
           org.apache.accumulo.core.dataImpl.thrift.TKeyExtent other_element_key = other_element.getKey();
-          java.util.Map<java.lang.String,org.apache.accumulo.core.dataImpl.thrift.MapFileInfo> other_element_value = other_element.getValue();
+          java.util.Map<java.lang.String,DataFileInfo> other_element_value = other_element.getValue();
 
           org.apache.accumulo.core.dataImpl.thrift.TKeyExtent __this__files_copy_key = new org.apache.accumulo.core.dataImpl.thrift.TKeyExtent(other_element_key);
 
-          java.util.Map<java.lang.String,org.apache.accumulo.core.dataImpl.thrift.MapFileInfo> __this__files_copy_value = new java.util.HashMap<java.lang.String,org.apache.accumulo.core.dataImpl.thrift.MapFileInfo>(other_element_value.size());
-          for (java.util.Map.Entry<java.lang.String, org.apache.accumulo.core.dataImpl.thrift.MapFileInfo> other_element_value_element : other_element_value.entrySet()) {
+          java.util.Map<java.lang.String,DataFileInfo> __this__files_copy_value = new java.util.HashMap<java.lang.String,DataFileInfo>(other_element_value.size());
+          for (java.util.Map.Entry<java.lang.String, DataFileInfo> other_element_value_element : other_element_value.entrySet()) {
 
             java.lang.String other_element_value_element_key = other_element_value_element.getKey();
-            org.apache.accumulo.core.dataImpl.thrift.MapFileInfo other_element_value_element_value = other_element_value_element.getValue();
+            DataFileInfo other_element_value_element_value = other_element_value_element.getValue();
 
             java.lang.String __this__files_copy_value_copy_key = other_element_value_element_key;
 
-            org.apache.accumulo.core.dataImpl.thrift.MapFileInfo __this__files_copy_value_copy_value = new org.apache.accumulo.core.dataImpl.thrift.MapFileInfo(other_element_value_element_value);
+            DataFileInfo __this__files_copy_value_copy_value = new DataFileInfo(other_element_value_element_value);
 
             __this__files_copy_value.put(__this__files_copy_value_copy_key, __this__files_copy_value_copy_value);
           }
@@ -10253,19 +10253,19 @@ public class TabletIngestClientService {
       return (this.files == null) ? 0 : this.files.size();
     }
 
-    public void putToFiles(org.apache.accumulo.core.dataImpl.thrift.TKeyExtent key, java.util.Map<java.lang.String,org.apache.accumulo.core.dataImpl.thrift.MapFileInfo> val) {
+    public void putToFiles(org.apache.accumulo.core.dataImpl.thrift.TKeyExtent key, java.util.Map<java.lang.String,DataFileInfo> val) {
       if (this.files == null) {
-        this.files = new java.util.HashMap<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent,java.util.Map<java.lang.String,org.apache.accumulo.core.dataImpl.thrift.MapFileInfo>>();
+        this.files = new java.util.HashMap<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent,java.util.Map<java.lang.String,DataFileInfo>>();
       }
       this.files.put(key, val);
     }
 
     @org.apache.thrift.annotation.Nullable
-    public java.util.Map<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent,java.util.Map<java.lang.String,org.apache.accumulo.core.dataImpl.thrift.MapFileInfo>> getFiles() {
+    public java.util.Map<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent,java.util.Map<java.lang.String,DataFileInfo>> getFiles() {
       return this.files;
     }
 
-    public loadFiles_args setFiles(@org.apache.thrift.annotation.Nullable java.util.Map<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent,java.util.Map<java.lang.String,org.apache.accumulo.core.dataImpl.thrift.MapFileInfo>> files) {
+    public loadFiles_args setFiles(@org.apache.thrift.annotation.Nullable java.util.Map<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent,java.util.Map<java.lang.String,DataFileInfo>> files) {
       this.files = files;
       return this;
     }
@@ -10347,7 +10347,7 @@ public class TabletIngestClientService {
         if (value == null) {
           unsetFiles();
         } else {
-          setFiles((java.util.Map<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent,java.util.Map<java.lang.String,org.apache.accumulo.core.dataImpl.thrift.MapFileInfo>>)value);
+          setFiles((java.util.Map<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent,java.util.Map<java.lang.String,DataFileInfo>>)value);
         }
         break;
 
@@ -10727,26 +10727,26 @@ public class TabletIngestClientService {
                 org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
               }
               break;
-            case 5: // FILES
+            case 7: // FILES
               if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
                 {
                   org.apache.thrift.protocol.TMap _map58 = iprot.readMapBegin();
-                  struct.files = new java.util.HashMap<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent,java.util.Map<java.lang.String,org.apache.accumulo.core.dataImpl.thrift.MapFileInfo>>(2*_map58.size);
+                  struct.files = new java.util.HashMap<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent,java.util.Map<java.lang.String,DataFileInfo>>(2*_map58.size);
                   @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TKeyExtent _key59;
-                  @org.apache.thrift.annotation.Nullable java.util.Map<java.lang.String,org.apache.accumulo.core.dataImpl.thrift.MapFileInfo> _val60;
+                  @org.apache.thrift.annotation.Nullable java.util.Map<java.lang.String,DataFileInfo> _val60;
                   for (int _i61 = 0; _i61 < _map58.size; ++_i61)
                   {
                     _key59 = new org.apache.accumulo.core.dataImpl.thrift.TKeyExtent();
                     _key59.read(iprot);
                     {
                       org.apache.thrift.protocol.TMap _map62 = iprot.readMapBegin();
-                      _val60 = new java.util.HashMap<java.lang.String,org.apache.accumulo.core.dataImpl.thrift.MapFileInfo>(2*_map62.size);
+                      _val60 = new java.util.HashMap<java.lang.String,DataFileInfo>(2*_map62.size);
                       @org.apache.thrift.annotation.Nullable java.lang.String _key63;
-                      @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.MapFileInfo _val64;
+                      @org.apache.thrift.annotation.Nullable DataFileInfo _val64;
                       for (int _i65 = 0; _i65 < _map62.size; ++_i65)
                       {
                         _key63 = iprot.readString();
-                        _val64 = new org.apache.accumulo.core.dataImpl.thrift.MapFileInfo();
+                        _val64 = new DataFileInfo();
                         _val64.read(iprot);
                         _val60.put(_key63, _val64);
                       }
@@ -10803,16 +10803,19 @@ public class TabletIngestClientService {
           oprot.writeString(struct.dir);
           oprot.writeFieldEnd();
         }
+        oprot.writeFieldBegin(SET_TIME_FIELD_DESC);
+        oprot.writeBool(struct.setTime);
+        oprot.writeFieldEnd();
         if (struct.files != null) {
           oprot.writeFieldBegin(FILES_FIELD_DESC);
           {
             oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRUCT, org.apache.thrift.protocol.TType.MAP, struct.files.size()));
-            for (java.util.Map.Entry<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent, java.util.Map<java.lang.String,org.apache.accumulo.core.dataImpl.thrift.MapFileInfo>> _iter66 : struct.files.entrySet())
+            for (java.util.Map.Entry<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent, java.util.Map<java.lang.String,DataFileInfo>> _iter66 : struct.files.entrySet())
             {
               _iter66.getKey().write(oprot);
               {
                 oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT, _iter66.getValue().size()));
-                for (java.util.Map.Entry<java.lang.String, org.apache.accumulo.core.dataImpl.thrift.MapFileInfo> _iter67 : _iter66.getValue().entrySet())
+                for (java.util.Map.Entry<java.lang.String, DataFileInfo> _iter67 : _iter66.getValue().entrySet())
                 {
                   oprot.writeString(_iter67.getKey());
                   _iter67.getValue().write(oprot);
@@ -10824,9 +10827,6 @@ public class TabletIngestClientService {
           }
           oprot.writeFieldEnd();
         }
-        oprot.writeFieldBegin(SET_TIME_FIELD_DESC);
-        oprot.writeBool(struct.setTime);
-        oprot.writeFieldEnd();
         oprot.writeFieldStop();
         oprot.writeStructEnd();
       }
@@ -10880,12 +10880,12 @@ public class TabletIngestClientService {
         if (struct.isSetFiles()) {
           {
             oprot.writeI32(struct.files.size());
-            for (java.util.Map.Entry<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent, java.util.Map<java.lang.String,org.apache.accumulo.core.dataImpl.thrift.MapFileInfo>> _iter68 : struct.files.entrySet())
+            for (java.util.Map.Entry<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent, java.util.Map<java.lang.String,DataFileInfo>> _iter68 : struct.files.entrySet())
             {
               _iter68.getKey().write(oprot);
               {
                 oprot.writeI32(_iter68.getValue().size());
-                for (java.util.Map.Entry<java.lang.String, org.apache.accumulo.core.dataImpl.thrift.MapFileInfo> _iter69 : _iter68.getValue().entrySet())
+                for (java.util.Map.Entry<java.lang.String, DataFileInfo> _iter69 : _iter68.getValue().entrySet())
                 {
                   oprot.writeString(_iter69.getKey());
                   _iter69.getValue().write(oprot);
@@ -10924,22 +10924,22 @@ public class TabletIngestClientService {
         if (incoming.get(4)) {
           {
             org.apache.thrift.protocol.TMap _map70 = iprot.readMapBegin(org.apache.thrift.protocol.TType.STRUCT, org.apache.thrift.protocol.TType.MAP); 
-            struct.files = new java.util.HashMap<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent,java.util.Map<java.lang.String,org.apache.accumulo.core.dataImpl.thrift.MapFileInfo>>(2*_map70.size);
+            struct.files = new java.util.HashMap<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent,java.util.Map<java.lang.String,DataFileInfo>>(2*_map70.size);
             @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TKeyExtent _key71;
-            @org.apache.thrift.annotation.Nullable java.util.Map<java.lang.String,org.apache.accumulo.core.dataImpl.thrift.MapFileInfo> _val72;
+            @org.apache.thrift.annotation.Nullable java.util.Map<java.lang.String,DataFileInfo> _val72;
             for (int _i73 = 0; _i73 < _map70.size; ++_i73)
             {
               _key71 = new org.apache.accumulo.core.dataImpl.thrift.TKeyExtent();
               _key71.read(iprot);
               {
                 org.apache.thrift.protocol.TMap _map74 = iprot.readMapBegin(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT); 
-                _val72 = new java.util.HashMap<java.lang.String,org.apache.accumulo.core.dataImpl.thrift.MapFileInfo>(2*_map74.size);
+                _val72 = new java.util.HashMap<java.lang.String,DataFileInfo>(2*_map74.size);
                 @org.apache.thrift.annotation.Nullable java.lang.String _key75;
-                @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.MapFileInfo _val76;
+                @org.apache.thrift.annotation.Nullable DataFileInfo _val76;
                 for (int _i77 = 0; _i77 < _map74.size; ++_i77)
                 {
                   _key75 = iprot.readString();
-                  _val76 = new org.apache.accumulo.core.dataImpl.thrift.MapFileInfo();
+                  _val76 = new DataFileInfo();
                   _val76.read(iprot);
                   _val72.put(_key75, _val76);
                 }
diff --git a/core/src/main/thrift/data.thrift b/core/src/main/thrift/data.thrift
index ad98c31e62..d83c99fae1 100644
--- a/core/src/main/thrift/data.thrift
+++ b/core/src/main/thrift/data.thrift
@@ -125,10 +125,6 @@ struct TCMResult {
   2:TCMStatus status
 }
 
-struct MapFileInfo {
-  1:i64 estimatedSize
-}
-
 struct TCondition {
   1:binary cf
   2:binary cq
diff --git a/core/src/main/thrift/tabletingest.thrift b/core/src/main/thrift/tabletingest.thrift
index b5ccae7249..b57d70faf9 100644
--- a/core/src/main/thrift/tabletingest.thrift
+++ b/core/src/main/thrift/tabletingest.thrift
@@ -37,6 +37,10 @@ enum TDurability {
   NONE = 4
 }
 
+struct DataFileInfo {
+  1:i64 estimatedSize
+}
+
 service TabletIngestClientService {
 
   //the following calls support a batch update to multiple tablets on a tablet server
@@ -110,7 +114,7 @@ service TabletIngestClientService {
     2:security.TCredentials credentials
     3:i64 tid
     4:string dir
-    5:map<data.TKeyExtent, map<string, data.MapFileInfo>> files
+    7:map<data.TKeyExtent, map<string, DataFileInfo>> files
     6:bool setTime
   )
 
diff --git a/core/src/test/java/org/apache/accumulo/core/conf/PropertyTypeTest.java b/core/src/test/java/org/apache/accumulo/core/conf/PropertyTypeTest.java
index 514fd1e95c..41a651ebe6 100644
--- a/core/src/test/java/org/apache/accumulo/core/conf/PropertyTypeTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/conf/PropertyTypeTest.java
@@ -28,6 +28,7 @@ import java.util.stream.Collectors;
 import java.util.stream.Stream;
 
 import org.apache.accumulo.core.WithTestNames;
+import org.apache.accumulo.core.file.rfile.RFile;
 import org.junit.jupiter.api.BeforeEach;
 import org.junit.jupiter.api.Test;
 
@@ -209,4 +210,10 @@ public class PropertyTypeTest extends WithTestNames {
     valid(null, "", "hdfs://hostname", "file:///path/", "hdfs://example.com:port/path");
   }
 
+  @Test
+  public void testTypeFILENAME_EXT() {
+    valid(RFile.EXTENSION, "rf");
+    invalid(null, "RF", "map", "", "MAP", "rF", "Rf", " rf ");
+  }
+
 }
diff --git a/core/src/test/java/org/apache/accumulo/core/iterators/user/IndexedDocIteratorTest.java b/core/src/test/java/org/apache/accumulo/core/iterators/user/IndexedDocIteratorTest.java
index 9c6bdb6a54..b886179c4c 100644
--- a/core/src/test/java/org/apache/accumulo/core/iterators/user/IndexedDocIteratorTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/iterators/user/IndexedDocIteratorTest.java
@@ -149,7 +149,7 @@ public class IndexedDocIteratorTest {
   private SortedKeyValueIterator<Key,Value> createIteratorStack(float hitRatio, int numRows,
       int numDocsPerRow, Text[] columnFamilies, Text[] otherColumnFamilies, HashSet<Text> docs,
       Text[] negatedColumns) throws IOException {
-    // write a map file
+    // write a data file
     trf.openWriter(false);
 
     TreeMap<Key,Value> inMemoryMap = createSortedMap(hitRatio, numRows, numDocsPerRow,
diff --git a/server/base/src/main/java/org/apache/accumulo/server/compaction/FileCompactor.java b/server/base/src/main/java/org/apache/accumulo/server/compaction/FileCompactor.java
index 3b6b84a7d0..2d4b1b7fd3 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/compaction/FileCompactor.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/compaction/FileCompactor.java
@@ -330,39 +330,39 @@ public class FileCompactor implements Callable<CompactionStats> {
 
     List<SortedKeyValueIterator<Key,Value>> iters = new ArrayList<>(filesToCompact.size());
 
-    for (TabletFile mapFile : filesToCompact.keySet()) {
+    for (TabletFile dataFile : filesToCompact.keySet()) {
       try {
 
         FileOperations fileFactory = FileOperations.getInstance();
-        FileSystem fs = this.fs.getFileSystemByPath(mapFile.getPath());
+        FileSystem fs = this.fs.getFileSystemByPath(dataFile.getPath());
         FileSKVIterator reader;
 
         reader = fileFactory.newReaderBuilder()
-            .forFile(mapFile.getPathStr(), fs, fs.getConf(), cryptoService)
+            .forFile(dataFile.getPathStr(), fs, fs.getConf(), cryptoService)
             .withTableConfiguration(acuTableConf).withRateLimiter(env.getReadLimiter())
             .dropCachesBehind().build();
 
         readers.add(reader);
 
         InterruptibleIterator iter = new ProblemReportingIterator(context, extent.tableId(),
-            mapFile.getPathStr(), false, reader);
+            dataFile.getPathStr(), false, reader);
 
-        iter = filesToCompact.get(mapFile).wrapFileIterator(iter);
+        iter = filesToCompact.get(dataFile).wrapFileIterator(iter);
 
         iters.add(iter);
 
       } catch (Exception e) {
 
         ProblemReports.getInstance(context).report(
-            new ProblemReport(extent.tableId(), ProblemType.FILE_READ, mapFile.getPathStr(), e));
+            new ProblemReport(extent.tableId(), ProblemType.FILE_READ, dataFile.getPathStr(), e));
 
-        log.warn("Some problem opening map file {} {}", mapFile, e.getMessage(), e);
-        // failed to open some map file... close the ones that were opened
+        log.warn("Some problem opening data file {} {}", dataFile, e.getMessage(), e);
+        // failed to open some data file... close the ones that were opened
         for (FileSKVIterator reader : readers) {
           try {
             reader.close();
           } catch (Exception e2) {
-            log.warn("Failed to close map file", e2);
+            log.warn("Failed to close data file", e2);
           }
         }
 
@@ -371,7 +371,7 @@ public class FileCompactor implements Callable<CompactionStats> {
         if (e instanceof IOException) {
           throw (IOException) e;
         }
-        throw new IOException("Failed to open map data files", e);
+        throw new IOException("Failed to open data files", e);
       }
     }
 
@@ -477,7 +477,7 @@ public class FileCompactor implements Callable<CompactionStats> {
         try {
           reader.close();
         } catch (Exception e) {
-          log.warn("Failed to close map file", e);
+          log.warn("Failed to close data file", e);
         }
       }
       compactSpan.end();
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 3c189644df..3b7d331e29 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
@@ -111,10 +111,10 @@ public class FileUtil {
   }
 
   public static Collection<String> reduceFiles(ServerContext context, TableConfiguration tableConf,
-      Text prevEndRow, Text endRow, Collection<String> mapFiles, int maxFiles, Path tmpDir,
+      Text prevEndRow, Text endRow, Collection<String> dataFiles, int maxFiles, Path tmpDir,
       int pass) throws IOException {
 
-    ArrayList<String> paths = new ArrayList<>(mapFiles);
+    ArrayList<String> paths = new ArrayList<>(dataFiles);
 
     if (paths.size() <= maxFiles) {
       return paths;
@@ -135,12 +135,12 @@ public class FileUtil {
       start = end;
 
       // temporary tablet file does not conform to typical path verified in TabletFile
-      String newMapFile = String.format("%s/%04d.%s", newDir, count++, RFile.EXTENSION);
+      String newDataFile = String.format("%s/%04d.%s", newDir, count++, RFile.EXTENSION);
 
-      outFiles.add(newMapFile);
-      FileSystem ns = context.getVolumeManager().getFileSystemByPath(new Path(newMapFile));
+      outFiles.add(newDataFile);
+      FileSystem ns = context.getVolumeManager().getFileSystemByPath(new Path(newDataFile));
       FileSKVWriter writer = new RFileOperations().newWriterBuilder()
-          .forFile(newMapFile, ns, ns.getConf(), tableConf.getCryptoService())
+          .forFile(newDataFile, ns, ns.getConf(), tableConf.getCryptoService())
           .withTableConfiguration(tableConf).build();
       writer.startDefaultLocalityGroup();
       List<SortedKeyValueIterator<Key,Value>> iters = new ArrayList<>(inFiles.size());
@@ -207,25 +207,25 @@ public class FileUtil {
   }
 
   public static double estimatePercentageLTE(ServerContext context, TableConfiguration tableConf,
-      String tabletDir, Text prevEndRow, Text endRow, Collection<String> mapFiles, Text splitRow)
+      String tabletDir, Text prevEndRow, Text endRow, Collection<String> dataFiles, Text splitRow)
       throws IOException {
 
     Path tmpDir = null;
 
     int maxToOpen =
         context.getConfiguration().getCount(Property.TSERV_TABLET_SPLIT_FINDMIDPOINT_MAXOPEN);
-    ArrayList<FileSKVIterator> readers = new ArrayList<>(mapFiles.size());
+    ArrayList<FileSKVIterator> readers = new ArrayList<>(dataFiles.size());
 
     try {
-      if (mapFiles.size() > maxToOpen) {
+      if (dataFiles.size() > maxToOpen) {
         tmpDir = createTmpDir(context, tabletDir);
 
         log.debug("Too many indexes ({}) to open at once for {} {}, reducing in tmpDir = {}",
-            mapFiles.size(), endRow, prevEndRow, tmpDir);
+            dataFiles.size(), endRow, prevEndRow, tmpDir);
 
         long t1 = System.currentTimeMillis();
-        mapFiles =
-            reduceFiles(context, tableConf, prevEndRow, endRow, mapFiles, maxToOpen, tmpDir, 0);
+        dataFiles =
+            reduceFiles(context, tableConf, prevEndRow, endRow, dataFiles, maxToOpen, tmpDir, 0);
         long t2 = System.currentTimeMillis();
 
         log.debug("Finished reducing indexes for {} {} in {}", endRow, prevEndRow,
@@ -237,7 +237,7 @@ public class FileUtil {
       }
 
       long numKeys =
-          countIndexEntries(context, tableConf, prevEndRow, endRow, mapFiles, true, readers);
+          countIndexEntries(context, tableConf, prevEndRow, endRow, dataFiles, true, readers);
 
       if (numKeys == 0) {
         // not enough info in the index to answer the question, so instead of going to
@@ -263,7 +263,7 @@ public class FileUtil {
       if (numLte > numKeys) {
         // something went wrong
         throw new RuntimeException("numLte > numKeys " + numLte + " " + numKeys + " " + prevEndRow
-            + " " + endRow + " " + splitRow + " " + mapFiles);
+            + " " + endRow + " " + splitRow + " " + dataFiles);
       }
 
       // do not want to return 0% or 100%, so add 1 and 2 below
@@ -276,39 +276,39 @@ public class FileUtil {
 
   /**
    *
-   * @param mapFiles - list MapFiles to find the mid point key
+   * @param dataFiles - list of data files to find the mid point key
    *
-   *        ISSUES : This method used the index files to find the mid point. If the map files have
+   *        ISSUES : This method used the index files to find the mid point. If the data files have
    *        different index intervals this method will not return an accurate mid point. Also, it
    *        would be tricky to use this method in conjunction with an in memory map because the
    *        indexing interval is unknown.
    */
   public static SortedMap<Double,Key> findMidPoint(ServerContext context,
       TableConfiguration tableConf, String tabletDirectory, Text prevEndRow, Text endRow,
-      Collection<String> mapFiles, double minSplit, boolean useIndex) throws IOException {
+      Collection<String> dataFiles, double minSplit, boolean useIndex) throws IOException {
 
-    Collection<String> origMapFiles = mapFiles;
+    Collection<String> origDataFiles = dataFiles;
 
     Path tmpDir = null;
 
     int maxToOpen =
         context.getConfiguration().getCount(Property.TSERV_TABLET_SPLIT_FINDMIDPOINT_MAXOPEN);
-    ArrayList<FileSKVIterator> readers = new ArrayList<>(mapFiles.size());
+    ArrayList<FileSKVIterator> readers = new ArrayList<>(dataFiles.size());
 
     try {
-      if (mapFiles.size() > maxToOpen) {
+      if (dataFiles.size() > maxToOpen) {
         if (!useIndex) {
           throw new IOException(
-              "Cannot find mid point using data files, too many " + mapFiles.size());
+              "Cannot find mid point using data files, too many " + dataFiles.size());
         }
         tmpDir = createTmpDir(context, tabletDirectory);
 
         log.debug("Too many indexes ({}) to open at once for {} {}, reducing in tmpDir = {}",
-            mapFiles.size(), endRow, prevEndRow, tmpDir);
+            dataFiles.size(), endRow, prevEndRow, tmpDir);
 
         long t1 = System.currentTimeMillis();
-        mapFiles =
-            reduceFiles(context, tableConf, prevEndRow, endRow, mapFiles, maxToOpen, tmpDir, 0);
+        dataFiles =
+            reduceFiles(context, tableConf, prevEndRow, endRow, dataFiles, maxToOpen, tmpDir, 0);
         long t2 = System.currentTimeMillis();
 
         log.debug("Finished reducing indexes for {} {} in {}", endRow, prevEndRow,
@@ -321,7 +321,7 @@ public class FileUtil {
 
       long t1 = System.currentTimeMillis();
 
-      long numKeys = countIndexEntries(context, tableConf, prevEndRow, endRow, mapFiles,
+      long numKeys = countIndexEntries(context, tableConf, prevEndRow, endRow, dataFiles,
           tmpDir == null ? useIndex : false, readers);
 
       if (numKeys == 0) {
@@ -329,10 +329,10 @@ public class FileUtil {
           log.warn(
               "Failed to find mid point using indexes, falling back to"
                   + " data files which is slower. No entries between {} and {} for {}",
-              prevEndRow, endRow, mapFiles);
-          // need to pass original map files, not possibly reduced indexes
-          return findMidPoint(context, tableConf, tabletDirectory, prevEndRow, endRow, origMapFiles,
-              minSplit, false);
+              prevEndRow, endRow, dataFiles);
+          // need to pass original data files, not possibly reduced indexes
+          return findMidPoint(context, tableConf, tabletDirectory, prevEndRow, endRow,
+              origDataFiles, minSplit, false);
         }
         return Collections.emptySortedMap();
       }
@@ -389,7 +389,7 @@ public class FileUtil {
             (key.compareRow(prevEndRow) > 0 && (endRow == null || key.compareRow(endRow) < 1));
         if (!inRange) {
           throw new IOException("Found mid point is not in range " + key + " " + prevEndRow + " "
-              + endRow + " " + mapFiles);
+              + endRow + " " + dataFiles);
         }
       }
 
@@ -425,12 +425,12 @@ public class FileUtil {
   }
 
   private static long countIndexEntries(ServerContext context, TableConfiguration tableConf,
-      Text prevEndRow, Text endRow, Collection<String> mapFiles, boolean useIndex,
+      Text prevEndRow, Text endRow, Collection<String> dataFiles, boolean useIndex,
       ArrayList<FileSKVIterator> readers) throws IOException {
     long numKeys = 0;
 
     // count the total number of index entries
-    for (String file : mapFiles) {
+    for (String file : dataFiles) {
       FileSKVIterator reader = null;
       Path path = new Path(file);
       FileSystem ns = context.getVolumeManager().getFileSystemByPath(path);
@@ -482,34 +482,34 @@ public class FileUtil {
   }
 
   public static Map<TabletFile,FileInfo> tryToGetFirstAndLastRows(ServerContext context,
-      TableConfiguration tableConf, Set<TabletFile> mapfiles) {
+      TableConfiguration tableConf, Set<TabletFile> dataFiles) {
 
-    HashMap<TabletFile,FileInfo> mapFilesInfo = new HashMap<>();
+    HashMap<TabletFile,FileInfo> dataFilesInfo = new HashMap<>();
 
     long t1 = System.currentTimeMillis();
 
-    for (TabletFile mapfile : mapfiles) {
+    for (TabletFile dataFile : dataFiles) {
 
       FileSKVIterator reader = null;
-      FileSystem ns = context.getVolumeManager().getFileSystemByPath(mapfile.getPath());
+      FileSystem ns = context.getVolumeManager().getFileSystemByPath(dataFile.getPath());
       try {
         reader = FileOperations.getInstance().newReaderBuilder()
-            .forFile(mapfile.getPathStr(), ns, ns.getConf(), tableConf.getCryptoService())
+            .forFile(dataFile.getPathStr(), ns, ns.getConf(), tableConf.getCryptoService())
             .withTableConfiguration(tableConf).build();
 
         Key firstKey = reader.getFirstKey();
         if (firstKey != null) {
-          mapFilesInfo.put(mapfile, new FileInfo(firstKey, reader.getLastKey()));
+          dataFilesInfo.put(dataFile, new FileInfo(firstKey, reader.getLastKey()));
         }
 
       } catch (IOException ioe) {
-        log.warn("Failed to read map file to determine first and last key : " + mapfile, ioe);
+        log.warn("Failed to read data file to determine first and last key : " + dataFile, ioe);
       } finally {
         if (reader != null) {
           try {
             reader.close();
           } catch (IOException ioe) {
-            log.warn("failed to close " + mapfile, ioe);
+            log.warn("failed to close " + dataFile, ioe);
           }
         }
       }
@@ -518,18 +518,18 @@ public class FileUtil {
 
     long t2 = System.currentTimeMillis();
 
-    log.debug(String.format("Found first and last keys for %d map files in %6.2f secs",
-        mapfiles.size(), (t2 - t1) / 1000.0));
+    log.debug(String.format("Found first and last keys for %d data files in %6.2f secs",
+        dataFiles.size(), (t2 - t1) / 1000.0));
 
-    return mapFilesInfo;
+    return dataFilesInfo;
   }
 
   public static WritableComparable<Key> findLastKey(ServerContext context,
-      TableConfiguration tableConf, Collection<TabletFile> mapFiles) throws IOException {
+      TableConfiguration tableConf, Collection<TabletFile> dataFiles) throws IOException {
 
     Key lastKey = null;
 
-    for (TabletFile file : mapFiles) {
+    for (TabletFile file : dataFiles) {
       FileSystem ns = context.getVolumeManager().getFileSystemByPath(file.getPath());
       FileSKVIterator reader = FileOperations.getInstance().newReaderBuilder()
           .forFile(file.getPathStr(), ns, ns.getConf(), tableConf.getCryptoService())
diff --git a/server/base/src/main/java/org/apache/accumulo/server/util/RemoveEntriesForMissingFiles.java b/server/base/src/main/java/org/apache/accumulo/server/util/RemoveEntriesForMissingFiles.java
index bce5912463..2f6e0a1e82 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/util/RemoveEntriesForMissingFiles.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/util/RemoveEntriesForMissingFiles.java
@@ -55,7 +55,7 @@ import io.opentelemetry.api.trace.Span;
 import io.opentelemetry.context.Scope;
 
 /**
- * Remove file entries for map files that don't exist.
+ * Remove file entries for data files that don't exist.
  */
 public class RemoveEntriesForMissingFiles {
 
diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/bulkVer2/LoadFiles.java b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/bulkVer2/LoadFiles.java
index be56ae4108..619e2dc522 100644
--- a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/bulkVer2/LoadFiles.java
+++ b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/bulkVer2/LoadFiles.java
@@ -42,7 +42,6 @@ import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.data.Mutation;
 import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.dataImpl.KeyExtent;
-import org.apache.accumulo.core.dataImpl.thrift.MapFileInfo;
 import org.apache.accumulo.core.dataImpl.thrift.TKeyExtent;
 import org.apache.accumulo.core.fate.FateTxId;
 import org.apache.accumulo.core.fate.Repo;
@@ -57,6 +56,7 @@ import org.apache.accumulo.core.metadata.schema.TabletMetadata.Location;
 import org.apache.accumulo.core.metadata.schema.TabletsMetadata;
 import org.apache.accumulo.core.rpc.ThriftUtil;
 import org.apache.accumulo.core.rpc.clients.ThriftClientTypes;
+import org.apache.accumulo.core.tabletingest.thrift.DataFileInfo;
 import org.apache.accumulo.core.tabletingest.thrift.TabletIngestClientService.Client;
 import org.apache.accumulo.core.trace.TraceUtil;
 import org.apache.accumulo.core.util.MapCounter;
@@ -145,7 +145,7 @@ class LoadFiles extends ManagerRepo {
     // Each RPC to a tablet server needs to check in zookeeper to see if the transaction is still
     // active. The purpose of this map is to group load request by tablet servers inorder to do less
     // RPCs. Less RPCs will result in less calls to Zookeeper.
-    Map<HostAndPort,Map<TKeyExtent,Map<String,MapFileInfo>>> loadQueue;
+    Map<HostAndPort,Map<TKeyExtent,Map<String,DataFileInfo>>> loadQueue;
     private int queuedDataSize = 0;
 
     @Override
@@ -188,11 +188,11 @@ class LoadFiles extends ManagerRepo {
     }
 
     private void addToQueue(HostAndPort server, KeyExtent extent,
-        Map<String,MapFileInfo> thriftImports) {
+        Map<String,DataFileInfo> thriftImports) {
       if (!thriftImports.isEmpty()) {
         loadMsgs.increment(server, 1);
 
-        Map<String,MapFileInfo> prev = loadQueue.computeIfAbsent(server, k -> new HashMap<>())
+        Map<String,DataFileInfo> prev = loadQueue.computeIfAbsent(server, k -> new HashMap<>())
             .putIfAbsent(extent.toThrift(), thriftImports);
 
         Preconditions.checkState(prev == null, "Unexpectedly saw extent %s twice", extent);
@@ -221,14 +221,14 @@ class LoadFiles extends ManagerRepo {
 
         Set<TabletFile> loadedFiles = tablet.getLoaded().keySet();
 
-        Map<String,MapFileInfo> thriftImports = new HashMap<>();
+        Map<String,DataFileInfo> thriftImports = new HashMap<>();
 
         for (final Bulk.FileInfo fileInfo : files) {
           Path fullPath = new Path(bulkDir, fileInfo.getFileName());
           TabletFile bulkFile = new TabletFile(fullPath);
 
           if (!loadedFiles.contains(bulkFile)) {
-            thriftImports.put(fileInfo.getFileName(), new MapFileInfo(fileInfo.getEstFileSize()));
+            thriftImports.put(fileInfo.getFileName(), new DataFileInfo(fileInfo.getEstFileSize()));
           }
         }
 
diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/delete/CleanUp.java b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/delete/CleanUp.java
index 038db0bc14..a4bb5b95f7 100644
--- a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/delete/CleanUp.java
+++ b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/delete/CleanUp.java
@@ -171,7 +171,7 @@ class CleanUp extends ManagerRepo {
     }
 
     if (refCount == 0) {
-      // delete the map files
+      // delete the data files
       try {
         VolumeManager fs = manager.getVolumeManager();
         for (String dir : manager.getContext().getTablesDirs()) {
diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/tableImport/MapImportFileNames.java b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/tableImport/MapImportFileNames.java
index 9d23f06601..19c9635b99 100644
--- a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/tableImport/MapImportFileNames.java
+++ b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/tableImport/MapImportFileNames.java
@@ -25,7 +25,6 @@ import java.io.BufferedWriter;
 import java.io.IOException;
 import java.io.OutputStreamWriter;
 
-import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.clientImpl.AcceptableThriftTableOperationException;
 import org.apache.accumulo.core.clientImpl.thrift.TableOperation;
 import org.apache.accumulo.core.clientImpl.thrift.TableOperationExceptionType;
@@ -78,12 +77,13 @@ class MapImportFileNames extends ManagerRepo {
           if (sa.length > 1) {
             extension = sa[sa.length - 1];
 
+            // skip files with unknown extensions
             if (!FileOperations.getValidExtensions().contains(extension)) {
               continue;
             }
           } else {
-            // assume it is a map file
-            extension = Constants.MAPFILE_EXTENSION;
+            // skip files without an extension
+            continue;
           }
 
           String newName = "I" + namer.getNextName() + "." + extension;
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletClientHandler.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletClientHandler.java
index 9539fd6fa4..d11efa5607 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletClientHandler.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletClientHandler.java
@@ -60,7 +60,6 @@ import org.apache.accumulo.core.data.Mutation;
 import org.apache.accumulo.core.data.NamespaceId;
 import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.dataImpl.KeyExtent;
-import org.apache.accumulo.core.dataImpl.thrift.MapFileInfo;
 import org.apache.accumulo.core.dataImpl.thrift.TCMResult;
 import org.apache.accumulo.core.dataImpl.thrift.TCMStatus;
 import org.apache.accumulo.core.dataImpl.thrift.TConditionalMutation;
@@ -90,6 +89,7 @@ import org.apache.accumulo.core.summary.SummaryCollection;
 import org.apache.accumulo.core.tablet.thrift.TUnloadTabletGoal;
 import org.apache.accumulo.core.tablet.thrift.TabletManagementClientService;
 import org.apache.accumulo.core.tabletingest.thrift.ConstraintViolationException;
+import org.apache.accumulo.core.tabletingest.thrift.DataFileInfo;
 import org.apache.accumulo.core.tabletingest.thrift.TDurability;
 import org.apache.accumulo.core.tabletingest.thrift.TabletIngestClientService;
 import org.apache.accumulo.core.tabletserver.thrift.ActiveCompaction;
@@ -165,7 +165,7 @@ public class TabletClientHandler implements TabletServerClientService.Iface,
 
   @Override
   public void loadFiles(TInfo tinfo, TCredentials credentials, long tid, String dir,
-      Map<TKeyExtent,Map<String,MapFileInfo>> tabletImports, boolean setTime)
+      Map<TKeyExtent,Map<String,DataFileInfo>> tabletImports, boolean setTime)
       throws ThriftSecurityException {
     if (!security.canPerformSystemActions(credentials)) {
       throw new ThriftSecurityException(credentials.getPrincipal(),
@@ -174,9 +174,9 @@ public class TabletClientHandler implements TabletServerClientService.Iface,
 
     watcher.runQuietly(Constants.BULK_ARBITRATOR_TYPE, tid, () -> {
       tabletImports.forEach((tke, fileMap) -> {
-        Map<TabletFile,MapFileInfo> newFileMap = new HashMap<>();
+        Map<TabletFile,DataFileInfo> newFileMap = new HashMap<>();
 
-        for (Entry<String,MapFileInfo> mapping : fileMap.entrySet()) {
+        for (Entry<String,DataFileInfo> mapping : fileMap.entrySet()) {
           Path path = new Path(dir, mapping.getKey());
           FileSystem ns = context.getVolumeManager().getFileSystemByPath(path);
           path = ns.makeQualified(path);
@@ -190,7 +190,7 @@ public class TabletClientHandler implements TabletServerClientService.Iface,
         if (importTablet != null) {
           try {
             server.updateBulkImportState(files, BulkImportState.PROCESSING);
-            importTablet.importMapFiles(tid, newFileMap, setTime);
+            importTablet.importDataFiles(tid, newFileMap, setTime);
           } catch (IOException ioe) {
             log.debug("files {} not imported to {}: {}", fileMap.keySet(),
                 KeyExtent.fromThrift(tke), ioe.getMessage());
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 dcdaa4c266..829e4dd5c0 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
@@ -1087,9 +1087,9 @@ public class TabletServer extends AbstractServer implements TabletHostingServer
   }
 
   public void minorCompactionStarted(CommitSession tablet, long lastUpdateSequence,
-      String newMapfileLocation) throws IOException {
+      String newDataFileLocation) throws IOException {
     Durability durability = getMincEventDurability(tablet.getExtent());
-    logger.minorCompactionStarted(tablet, lastUpdateSequence, newMapfileLocation, durability);
+    logger.minorCompactionStarted(tablet, lastUpdateSequence, newDataFileLocation, durability);
   }
 
   public void recover(VolumeManager fs, KeyExtent extent, List<LogEntry> logEntries,
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServerResourceManager.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServerResourceManager.java
index e9cdbf4b17..2903e55640 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServerResourceManager.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServerResourceManager.java
@@ -657,9 +657,9 @@ public class TabletServerResourceManager {
       return tableConf;
     }
 
-    // BEGIN methods that Tablets call to manage their set of open map files
+    // BEGIN methods that Tablets call to manage their set of open data files
 
-    public void importedMapFiles() {
+    public void importedDataFiles() {
       lastReportedCommitTime = System.currentTimeMillis();
     }
 
@@ -672,7 +672,7 @@ public class TabletServerResourceManager {
           new ScanCacheProvider(tableConf, scanDispatch, _iCache, _dCache));
     }
 
-    // END methods that Tablets call to manage their set of open map files
+    // END methods that Tablets call to manage their set of open data files
 
     // BEGIN methods that Tablets call to manage memory
 
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/CompactableImpl.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/CompactableImpl.java
index e7c638a5a9..b77594f3f5 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/CompactableImpl.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/CompactableImpl.java
@@ -1280,7 +1280,7 @@ public class CompactableImpl implements Compactable {
       TabletLogger.compacting(getExtent(), job, cInfo.localCompactionCfg);
       tablet.incrementStatusMajor();
       var check = new CompactionCheck(service, kind, cInfo.checkCompactionId);
-      TabletFile tmpFileName = tablet.getNextMapFilenameForMajc(cInfo.propagateDeletes);
+      TabletFile tmpFileName = tablet.getNextDataFilenameForMajc(cInfo.propagateDeletes);
       var compactEnv = new MajCEnv(kind, check, readLimiter, writeLimiter, cInfo.propagateDeletes);
 
       SortedMap<StoredTabletFile,DataFileValue> allFiles = tablet.getDatafiles();
@@ -1323,7 +1323,7 @@ public class CompactableImpl implements Compactable {
       Map<String,String> overrides =
           CompactableUtils.getOverrides(job.getKind(), tablet, cInfo.localHelper, job.getFiles());
 
-      TabletFile compactTmpName = tablet.getNextMapFilenameForMajc(cInfo.propagateDeletes);
+      TabletFile compactTmpName = tablet.getNextDataFilenameForMajc(cInfo.propagateDeletes);
 
       ExternalCompactionInfo ecInfo = new ExternalCompactionInfo();
 
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/DatafileManager.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/DatafileManager.java
index 313fb0f3bb..eb9f2f8d54 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/DatafileManager.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/DatafileManager.java
@@ -209,7 +209,7 @@ class DatafileManager {
     return inUse;
   }
 
-  public Collection<StoredTabletFile> importMapFiles(long tid, Map<TabletFile,DataFileValue> paths,
+  public Collection<StoredTabletFile> importDataFiles(long tid, Map<TabletFile,DataFileValue> paths,
       boolean setTime) throws IOException {
 
     String bulkDir = null;
@@ -273,7 +273,7 @@ class DatafileManager {
           datafileSizes.put(tpath.getKey(), tpath.getValue());
         }
 
-        tablet.getTabletResources().importedMapFiles();
+        tablet.getTabletResources().importedDataFiles();
 
         tablet.computeNumEntries();
       }
@@ -308,7 +308,7 @@ class DatafileManager {
           vm.deleteRecursively(tmpDatafile.getPath());
         } else {
           if (!attemptedRename && vm.exists(newDatafile.getPath())) {
-            log.warn("Target map file already exist {}", newDatafile);
+            log.warn("Target data file already exist {}", newDatafile);
             throw new RuntimeException("File unexpectedly exists " + newDatafile.getPath());
           }
           // the following checks for spurious rename failures that succeeded but gave an IoE
@@ -406,8 +406,8 @@ class DatafileManager {
     TabletFile newDatafile = CompactableUtils.computeCompactionFileDest(tmpDatafile);
 
     if (vm.exists(newDatafile.getPath())) {
-      log.error("Target map file already exist " + newDatafile, new Exception());
-      throw new IllegalStateException("Target map file already exist " + newDatafile);
+      log.error("Target data file already exist " + newDatafile, new Exception());
+      throw new IllegalStateException("Target data file already exist " + newDatafile);
     }
 
     if (dfv.getNumEntries() == 0) {
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/MinorCompactionTask.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/MinorCompactionTask.java
index 9399df6e37..e2aa06c997 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/MinorCompactionTask.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/MinorCompactionTask.java
@@ -77,7 +77,7 @@ class MinorCompactionTask implements Runnable {
           while (true) {
             try {
               if (newFile == null) {
-                newFile = tablet.getNextMapFilename(FilePrefix.MINOR_COMPACTION);
+                newFile = tablet.getNextDataFilename(FilePrefix.MINOR_COMPACTION);
                 tmpFile = new TabletFile(new Path(newFile.getPathStr() + "_tmp"));
               }
               /*
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/MinorCompactor.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/MinorCompactor.java
index 0092da7712..d3aebece72 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/MinorCompactor.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/MinorCompactor.java
@@ -82,7 +82,7 @@ public class MinorCompactor extends FileCompactor {
     final String outputFileName = getOutputFile();
     log.trace("Begin minor compaction {} {}", outputFileName, getExtent());
 
-    // output to new MapFile with a temporary name
+    // output to new data file with a temporary name
     int sleepTime = 100;
     double growthFactor = 4;
     int maxSleepTime = 1000 * 60 * 3; // 3 minutes
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/ScanDataSource.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/ScanDataSource.java
index 93b0e8a7f2..af4ac3b35b 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/ScanDataSource.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/ScanDataSource.java
@@ -164,15 +164,15 @@ class ScanDataSource implements DataSource {
       files = reservation.getSecond();
     }
 
-    Collection<InterruptibleIterator> mapfiles =
+    Collection<InterruptibleIterator> datafiles =
         fileManager.openFiles(files, scanParams.isIsolated(), samplerConfig);
 
-    List.of(mapfiles, memIters).forEach(c -> c.forEach(ii -> ii.setInterruptFlag(interruptFlag)));
+    List.of(datafiles, memIters).forEach(c -> c.forEach(ii -> ii.setInterruptFlag(interruptFlag)));
 
     List<SortedKeyValueIterator<Key,Value>> iters =
-        new ArrayList<>(mapfiles.size() + memIters.size());
+        new ArrayList<>(datafiles.size() + memIters.size());
 
-    iters.addAll(mapfiles);
+    iters.addAll(datafiles);
     iters.addAll(memIters);
 
     MultiIterator multiIter = new MultiIterator(iters, tablet.getExtent());
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Scanner.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Scanner.java
index 107d36f24a..5092984d92 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Scanner.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Scanner.java
@@ -149,7 +149,7 @@ public class Scanner {
       throw re;
     } finally {
       // code in finally block because always want
-      // to return mapfiles, even when exception is thrown
+      // to return datafiles, even when exception is thrown
       if (dataSource != null && !scanParams.isIsolated()) {
         dataSource.close(false);
       } else if (dataSource != null) {
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 de7a865b5b..dc4403927b 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
@@ -63,7 +63,6 @@ import org.apache.accumulo.core.data.Mutation;
 import org.apache.accumulo.core.data.Range;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.dataImpl.KeyExtent;
-import org.apache.accumulo.core.dataImpl.thrift.MapFileInfo;
 import org.apache.accumulo.core.file.FileOperations;
 import org.apache.accumulo.core.file.FilePrefix;
 import org.apache.accumulo.core.iterators.SortedKeyValueIterator;
@@ -86,6 +85,7 @@ import org.apache.accumulo.core.sample.impl.SamplerConfigurationImpl;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.spi.fs.VolumeChooserEnvironment;
 import org.apache.accumulo.core.spi.scan.ScanDispatch;
+import org.apache.accumulo.core.tabletingest.thrift.DataFileInfo;
 import org.apache.accumulo.core.tabletserver.log.LogEntry;
 import org.apache.accumulo.core.tabletserver.thrift.TabletStats;
 import org.apache.accumulo.core.trace.TraceUtil;
@@ -214,7 +214,7 @@ public class Tablet extends TabletBase {
   private final Rate scannedRate = new Rate(0.95);
 
   private long lastMinorCompactionFinishTime = 0;
-  private long lastMapFileImportTime = 0;
+  private long lastDataFileImportTime = 0;
 
   private volatile long numEntries = 0;
   private volatile long numEntriesInMemory = 0;
@@ -252,14 +252,14 @@ public class Tablet extends TabletBase {
     return dirUri;
   }
 
-  TabletFile getNextMapFilename(FilePrefix prefix) throws IOException {
+  TabletFile getNextDataFilename(FilePrefix prefix) throws IOException {
     String extension = FileOperations.getNewFileExtension(tableConfiguration);
     return new TabletFile(new Path(chooseTabletDir() + "/" + prefix.toPrefix()
         + context.getUniqueNameAllocator().getNextName() + "." + extension));
   }
 
-  TabletFile getNextMapFilenameForMajc(boolean propagateDeletes) throws IOException {
-    String tmpFileName = getNextMapFilename(
+  TabletFile getNextDataFilenameForMajc(boolean propagateDeletes) throws IOException {
+    String tmpFileName = getNextDataFilename(
         !propagateDeletes ? FilePrefix.MAJOR_COMPACTION_ALL_FILES : FilePrefix.MAJOR_COMPACTION)
         .getMetaInsert() + "_tmp";
     return new TabletFile(new Path(tmpFileName));
@@ -444,7 +444,7 @@ public class Tablet extends TabletBase {
       throw ioe;
     } finally {
       // code in finally block because always want
-      // to return mapfiles, even when exception is thrown
+      // to return data files, even when exception is thrown
       dataSource.close(false);
     }
   }
@@ -871,7 +871,7 @@ public class Tablet extends TabletBase {
   }
 
   /**
-   * Closes the mapfiles associated with a Tablet. If saveState is true, a minor compaction is
+   * Closes the data files associated with a Tablet. If saveState is true, a minor compaction is
    * performed.
    */
   @Override
@@ -1011,7 +1011,7 @@ public class Tablet extends TabletBase {
 
     getTabletMemory().close();
 
-    // close map files
+    // close data files
     getTabletResources().close();
 
     if (completeClose) {
@@ -1290,8 +1290,8 @@ public class Tablet extends TabletBase {
   private boolean isFindSplitsSuppressed() {
     if (supressFindSplits) {
       if (timeOfLastMinCWhenFindSplitsWasSupressed != lastMinorCompactionFinishTime
-          || timeOfLastImportWhenFindSplitsWasSupressed != lastMapFileImportTime) {
-        // a minor compaction or map file import has occurred... check again
+          || timeOfLastImportWhenFindSplitsWasSupressed != lastDataFileImportTime) {
+        // a minor compaction or data file import has occurred... check again
         supressFindSplits = false;
       } else {
         // nothing changed, do not split
@@ -1308,7 +1308,7 @@ public class Tablet extends TabletBase {
   private void suppressFindSplits() {
     supressFindSplits = true;
     timeOfLastMinCWhenFindSplitsWasSupressed = lastMinorCompactionFinishTime;
-    timeOfLastImportWhenFindSplitsWasSupressed = lastMapFileImportTime;
+    timeOfLastImportWhenFindSplitsWasSupressed = lastDataFileImportTime;
   }
 
   private static int longestCommonLength(Text text, Text beforeMid) {
@@ -1510,10 +1510,10 @@ public class Tablet extends TabletBase {
     }
 
     // obtain this info outside of synch block since it will involve opening
-    // the map files... it is ok if the set of map files changes, because
-    // this info is used for optimization... it is ok if map files are missing
+    // the data files... it is ok if the set of data files changes, because
+    // this info is used for optimization... it is ok if data files are missing
     // from the set... can still query and insert into the tablet while this
-    // map file operation is happening
+    // data file operation is happening
     Map<TabletFile,FileUtil.FileInfo> firstAndLastRows = FileUtil.tryToGetFirstAndLastRows(context,
         tableConfiguration, getDatafileManager().getFiles());
 
@@ -1659,12 +1659,12 @@ public class Tablet extends TabletBase {
     return splitCreationTime;
   }
 
-  public void importMapFiles(long tid, Map<TabletFile,MapFileInfo> fileMap, boolean setTime)
+  public void importDataFiles(long tid, Map<TabletFile,DataFileInfo> fileMap, boolean setTime)
       throws IOException {
     Map<TabletFile,DataFileValue> entries = new HashMap<>(fileMap.size());
     List<String> files = new ArrayList<>();
 
-    for (Entry<TabletFile,MapFileInfo> entry : fileMap.entrySet()) {
+    for (Entry<TabletFile,DataFileInfo> entry : fileMap.entrySet()) {
       entries.put(entry.getKey(), new DataFileValue(entry.getValue().estimatedSize, 0L));
       files.add(entry.getKey().getPathStr());
     }
@@ -1714,8 +1714,8 @@ public class Tablet extends TabletBase {
     try {
       tabletServer.updateBulkImportState(files, BulkImportState.LOADING);
 
-      var storedTabletFile = getDatafileManager().importMapFiles(tid, entries, setTime);
-      lastMapFileImportTime = System.currentTimeMillis();
+      var storedTabletFile = getDatafileManager().importDataFiles(tid, entries, setTime);
+      lastDataFileImportTime = System.currentTimeMillis();
 
       if (isSplitPossible()) {
         getTabletServer().executeSplit(this);
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/TabletBase.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/TabletBase.java
index 1f54e66eac..1d0fb6829e 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/TabletBase.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/TabletBase.java
@@ -216,7 +216,7 @@ public abstract class TabletBase {
       throw ioe;
     } finally {
       // code in finally block because always want
-      // to return mapfiles, even when exception is thrown
+      // to return data files, even when exception is thrown
       dataSource.close(false);
 
       synchronized (this) {
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/BulkFailureIT.java b/test/src/main/java/org/apache/accumulo/test/functional/BulkFailureIT.java
index e2c21382d4..fe8bd12b1b 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/BulkFailureIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/BulkFailureIT.java
@@ -59,7 +59,6 @@ import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.dataImpl.KeyExtent;
-import org.apache.accumulo.core.dataImpl.thrift.MapFileInfo;
 import org.apache.accumulo.core.file.FileOperations;
 import org.apache.accumulo.core.metadata.MetadataTable;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.BulkFileColumnFamily;
@@ -68,6 +67,7 @@ import org.apache.accumulo.core.rpc.ThriftUtil;
 import org.apache.accumulo.core.rpc.clients.ThriftClientTypes;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.security.TablePermission;
+import org.apache.accumulo.core.tabletingest.thrift.DataFileInfo;
 import org.apache.accumulo.core.tabletingest.thrift.TabletIngestClientService;
 import org.apache.accumulo.core.trace.TraceUtil;
 import org.apache.accumulo.core.util.threads.ThreadPools;
@@ -79,7 +79,6 @@ import org.apache.accumulo.server.zookeeper.TransactionWatcher.ZooArbitrator;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.MapFile;
 import org.apache.hadoop.io.Text;
 import org.apache.thrift.TServiceClient;
 import org.apache.thrift.transport.TTransportException;
@@ -147,7 +146,7 @@ public class BulkFailureIT extends AccumuloClusterHarness {
         "/" + bulkDir.getParent().getName() + "/" + bulkDir.getName(), tid);
 
     Path dirPath = new Path(dir);
-    FileStatus[] mapFiles = fs.listStatus(dirPath);
+    FileStatus[] dataFiles = fs.listStatus(dirPath);
 
     final UniqueNameAllocator namer = manager.getUniqueNameAllocator();
 
@@ -157,7 +156,7 @@ public class BulkFailureIT extends AccumuloClusterHarness {
         ThreadPools.getServerThreadPools().createFixedThreadPool(numThreads, "bulk rename", false);
     List<Future<Exception>> results = new ArrayList<>();
 
-    for (FileStatus file : mapFiles) {
+    for (FileStatus file : dataFiles) {
       final FileStatus fileStatus = file;
       results.add(workers.submit(() -> {
         try {
@@ -171,32 +170,8 @@ public class BulkFailureIT extends AccumuloClusterHarness {
               return null;
             }
           } else {
-            // assume it is a map file
-            extension = Constants.MAPFILE_EXTENSION;
-          }
-
-          if (extension.equals(Constants.MAPFILE_EXTENSION)) {
-            if (!fileStatus.isDirectory()) {
-              LOG.warn("{} is not a map file, ignoring", fileStatus.getPath());
-              return null;
-            }
-
-            if (fileStatus.getPath().getName().equals("_logs")) {
-              LOG.info("{} is probably a log directory from a map/reduce task, skipping",
-                  fileStatus.getPath());
-              return null;
-            }
-            try {
-              FileStatus dataStatus =
-                  fs.getFileStatus(new Path(fileStatus.getPath(), MapFile.DATA_FILE_NAME));
-              if (dataStatus.isDirectory()) {
-                LOG.warn("{} is not a map file, ignoring", fileStatus.getPath());
-                return null;
-              }
-            } catch (FileNotFoundException fnfe) {
-              LOG.warn("{} is not a map file, ignoring", fileStatus.getPath());
-              return null;
-            }
+            LOG.warn("{} does not have any extension, ignoring", fileStatus.getPath());
+            return null;
           }
 
           String newName = "I" + namer.getNextName() + "." + extension;
@@ -385,8 +360,8 @@ public class BulkFailureIT extends AccumuloClusterHarness {
     TabletIngestClientService.Iface client = getClient(context, extent);
     try {
 
-      Map<String,MapFileInfo> val = Map.of(path.getName(), new MapFileInfo(size));
-      Map<KeyExtent,Map<String,MapFileInfo>> files = Map.of(extent, val);
+      Map<String,DataFileInfo> val = Map.of(path.getName(), new DataFileInfo(size));
+      Map<KeyExtent,Map<String,DataFileInfo>> files = Map.of(extent, val);
 
       client.loadFiles(TraceUtil.traceInfo(), context.rpcCreds(), txid, path.getParent().toString(),
           files.entrySet().stream().collect(
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/BulkSplitOptimizationIT.java b/test/src/main/java/org/apache/accumulo/test/functional/BulkSplitOptimizationIT.java
index 200405b675..5263c888d1 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/BulkSplitOptimizationIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/BulkSplitOptimizationIT.java
@@ -40,7 +40,7 @@ import org.junit.jupiter.api.Test;
 
 /**
  * This test verifies that when a lot of files are bulk imported into a table with one tablet and
- * then splits that not all map files go to the children tablets.
+ * then splits that not all data files go to the children tablets.
  */
 public class BulkSplitOptimizationIT extends AccumuloClusterHarness {
 
@@ -121,7 +121,7 @@ public class BulkSplitOptimizationIT extends AccumuloClusterHarness {
       params.cols = 1;
       VerifyIngest.verifyIngest(c, params);
 
-      // ensure each tablet does not have all map files, should be ~2.5 files per tablet
+      // ensure each tablet does not have all data files, should be ~2.5 files per tablet
       FunctionalTestUtils.checkRFiles(c, tableName, 50, 100, 1, 4);
     }
   }
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/FunctionalTestUtils.java b/test/src/main/java/org/apache/accumulo/test/functional/FunctionalTestUtils.java
index da5118e88a..e5d34e446c 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/FunctionalTestUtils.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/FunctionalTestUtils.java
@@ -134,7 +134,7 @@ public class FunctionalTestUtils {
       for (Entry<Text,Integer> entry : es) {
         if (entry.getValue() > maxRFiles || entry.getValue() < minRFiles) {
           throw new Exception(
-              "tablet " + entry.getKey() + " has " + entry.getValue() + " map files");
+              "tablet " + entry.getKey() + " has " + entry.getValue() + " data files");
         }
       }
     }
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/MaxOpenIT.java b/test/src/main/java/org/apache/accumulo/test/functional/MaxOpenIT.java
index 9e60bc6589..8e1a2f5c1e 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/MaxOpenIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/MaxOpenIT.java
@@ -96,7 +96,7 @@ public class MaxOpenIT extends AccumuloClusterHarness {
           .withSplits(TestIngest.getSplitPoints(0, NUM_TO_INGEST, NUM_TABLETS));
       c.tableOperations().create(tableName, ntc);
 
-      // the following loop should create three tablets in each map file
+      // the following loop should create three tablets in each data file
       for (int i = 0; i < 3; i++) {
         IngestParams params = new IngestParams(getClientProps(), tableName, NUM_TO_INGEST);
         params.timestamp = i;
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/SplitRecoveryIT.java b/test/src/main/java/org/apache/accumulo/test/functional/SplitRecoveryIT.java
index 3022c2614c..fb0a99d260 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/SplitRecoveryIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/SplitRecoveryIT.java
@@ -155,7 +155,7 @@ public class SplitRecoveryIT extends ConfigurableMacBase {
 
     Text midRow = new Text(mr);
 
-    SortedMap<StoredTabletFile,DataFileValue> splitMapFiles = null;
+    SortedMap<StoredTabletFile,DataFileValue> splitDataFiles = null;
 
     for (int i = 0; i < extents.length; i++) {
       KeyExtent extent = extents[i];
@@ -164,17 +164,17 @@ public class SplitRecoveryIT extends ConfigurableMacBase {
       String tdir =
           context.getTablesDirs().iterator().next() + "/" + extent.tableId() + "/" + dirName;
       MetadataTableUtil.addTablet(extent, dirName, context, TimeType.LOGICAL, zl);
-      SortedMap<TabletFile,DataFileValue> mapFiles = new TreeMap<>();
-      mapFiles.put(new TabletFile(new Path(tdir + "/" + RFile.EXTENSION + "_000_000")),
+      SortedMap<TabletFile,DataFileValue> dataFiles = new TreeMap<>();
+      dataFiles.put(new TabletFile(new Path(tdir + "/" + RFile.EXTENSION + "_000_000")),
           new DataFileValue(1000017 + i, 10000 + i));
 
       int tid = 0;
       TransactionWatcher.ZooArbitrator.start(context, Constants.BULK_ARBITRATOR_TYPE, tid);
       SortedMap<StoredTabletFile,DataFileValue> storedFiles =
-          new TreeMap<>(MetadataTableUtil.updateTabletDataFile(tid, extent, mapFiles,
+          new TreeMap<>(MetadataTableUtil.updateTabletDataFile(tid, extent, dataFiles,
               new MetadataTime(0, TimeType.LOGICAL), context, zl));
       if (i == extentToSplit) {
-        splitMapFiles = storedFiles;
+        splitDataFiles = storedFiles;
       }
     }
 
@@ -183,7 +183,7 @@ public class SplitRecoveryIT extends ConfigurableMacBase {
     KeyExtent high = new KeyExtent(extent.tableId(), extent.endRow(), midRow);
     KeyExtent low = new KeyExtent(extent.tableId(), midRow, extent.prevEndRow());
 
-    splitPartiallyAndRecover(context, extent, high, low, .4, splitMapFiles, midRow,
+    splitPartiallyAndRecover(context, extent, high, low, .4, splitDataFiles, midRow,
         "localhost:1234", failPoint, zl);
   }
 
@@ -198,14 +198,14 @@ public class SplitRecoveryIT extends ConfigurableMacBase {
   }
 
   private void splitPartiallyAndRecover(ServerContext context, KeyExtent extent, KeyExtent high,
-      KeyExtent low, double splitRatio, SortedMap<StoredTabletFile,DataFileValue> mapFiles,
+      KeyExtent low, double splitRatio, SortedMap<StoredTabletFile,DataFileValue> dataFiles,
       Text midRow, String location, int steps, ServiceLock zl) throws Exception {
 
     SortedMap<StoredTabletFile,DataFileValue> lowDatafileSizes = new TreeMap<>();
     SortedMap<StoredTabletFile,DataFileValue> highDatafileSizes = new TreeMap<>();
     List<StoredTabletFile> highDatafilesToRemove = new ArrayList<>();
 
-    MetadataTableUtil.splitDatafiles(midRow, splitRatio, new HashMap<>(), mapFiles,
+    MetadataTableUtil.splitDatafiles(midRow, splitRatio, new HashMap<>(), dataFiles,
         lowDatafileSizes, highDatafileSizes, highDatafilesToRemove);
 
     MetadataTableUtil.splitTablet(high, extent.prevEndRow(), splitRatio, context, zl, Set.of());
@@ -250,12 +250,12 @@ public class SplitRecoveryIT extends ConfigurableMacBase {
       }
     } else {
       assertEquals(extent, fixedExtent);
-      ensureTabletHasNoUnexpectedMetadataEntries(context, extent, mapFiles);
+      ensureTabletHasNoUnexpectedMetadataEntries(context, extent, dataFiles);
     }
   }
 
   private void ensureTabletHasNoUnexpectedMetadataEntries(ServerContext context, KeyExtent extent,
-      SortedMap<StoredTabletFile,DataFileValue> expectedMapFiles) throws Exception {
+      SortedMap<StoredTabletFile,DataFileValue> expectedDataFiles) throws Exception {
     try (Scanner scanner = new ScannerImpl(context, MetadataTable.ID, Authorizations.EMPTY)) {
       scanner.setRange(extent.toMetaRange());
 
@@ -310,9 +310,9 @@ public class SplitRecoveryIT extends ConfigurableMacBase {
 
       assertTrue(sawPer);
 
-      SortedMap<StoredTabletFile,DataFileValue> fixedMapFiles =
+      SortedMap<StoredTabletFile,DataFileValue> fixedDataFiles =
           MetadataTableUtil.getFileAndLogEntries(context, extent).getSecond();
-      verifySame(expectedMapFiles, fixedMapFiles);
+      verifySame(expectedDataFiles, fixedDataFiles);
     }
   }
 
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 698f2d5b96..93cc242f94 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
@@ -40,7 +40,6 @@ import org.apache.accumulo.core.dataImpl.KeyExtent;
 import org.apache.accumulo.core.dataImpl.thrift.InitialMultiScan;
 import org.apache.accumulo.core.dataImpl.thrift.InitialScan;
 import org.apache.accumulo.core.dataImpl.thrift.IterInfo;
-import org.apache.accumulo.core.dataImpl.thrift.MapFileInfo;
 import org.apache.accumulo.core.dataImpl.thrift.MultiScanResult;
 import org.apache.accumulo.core.dataImpl.thrift.ScanResult;
 import org.apache.accumulo.core.dataImpl.thrift.TCMResult;
@@ -63,6 +62,7 @@ import org.apache.accumulo.core.rpc.clients.ThriftClientTypes;
 import org.apache.accumulo.core.securityImpl.thrift.TCredentials;
 import org.apache.accumulo.core.tablet.thrift.TUnloadTabletGoal;
 import org.apache.accumulo.core.tablet.thrift.TabletManagementClientService;
+import org.apache.accumulo.core.tabletingest.thrift.DataFileInfo;
 import org.apache.accumulo.core.tabletingest.thrift.TDurability;
 import org.apache.accumulo.core.tabletingest.thrift.TabletIngestClientService;
 import org.apache.accumulo.core.tabletscan.thrift.ActiveScan;
@@ -118,7 +118,7 @@ public class NullTserver {
 
     @Override
     public void loadFiles(TInfo tinfo, TCredentials credentials, long tid, String dir,
-        Map<TKeyExtent,Map<String,MapFileInfo>> fileMap, boolean setTime) {}
+        Map<TKeyExtent,Map<String,DataFileInfo>> fileMap, boolean setTime) {}
 
     @Override
     public void closeMultiScan(TInfo tinfo, long scanID) {}
diff --git a/test/src/main/java/org/apache/accumulo/test/performance/scan/CollectTabletStats.java b/test/src/main/java/org/apache/accumulo/test/performance/scan/CollectTabletStats.java
index 69ac1553aa..d22508299d 100644
--- a/test/src/main/java/org/apache/accumulo/test/performance/scan/CollectTabletStats.java
+++ b/test/src/main/java/org/apache/accumulo/test/performance/scan/CollectTabletStats.java
@@ -76,7 +76,6 @@ import org.apache.accumulo.server.util.MetadataTableUtil;
 import org.apache.hadoop.fs.BlockLocation;
 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;
@@ -405,8 +404,8 @@ public class CollectTabletStats {
       FileStatus status = fs.getFileStatus(file.getPath());
 
       if (status.isDirectory()) {
-        // assume it is a map file
-        status = fs.getFileStatus(new Path(file + "/data"));
+        log.warn("Saw unexpected directory at {} while getting block locations", file);
+        continue;
       }
       FileSystem ns = fs.getFileSystemByPath(file.getPath());
       BlockLocation[] locs = ns.getFileBlockLocations(status, 0, status.getLen());
@@ -427,16 +426,16 @@ public class CollectTabletStats {
   }
 
   private static SortedKeyValueIterator<Key,Value> createScanIterator(KeyExtent ke,
-      Collection<SortedKeyValueIterator<Key,Value>> mapfiles, Authorizations authorizations,
+      Collection<SortedKeyValueIterator<Key,Value>> dataFiles, Authorizations authorizations,
       byte[] defaultLabels, HashSet<Column> columnSet, List<IterInfo> ssiList,
       Map<String,Map<String,String>> ssio, boolean useTableIterators, TableConfiguration conf)
       throws IOException {
 
     SortedMapIterator smi = new SortedMapIterator(new TreeMap<>());
 
-    List<SortedKeyValueIterator<Key,Value>> iters = new ArrayList<>(mapfiles.size() + 1);
+    List<SortedKeyValueIterator<Key,Value>> iters = new ArrayList<>(dataFiles.size() + 1);
 
-    iters.addAll(mapfiles);
+    iters.addAll(dataFiles);
     iters.add(smi);
 
     MultiIterator multiIter = new MultiIterator(iters, ke);