Clean up relative path code (#1479)

* Drop methods in VolumeManager dealing with relative paths
* Create TabletFileUtil to do path validation using Path object
diff --git a/core/src/main/java/org/apache/accumulo/core/metadata/TabletFileUtil.java b/core/src/main/java/org/apache/accumulo/core/metadata/TabletFileUtil.java
new file mode 100644
index 0000000..ff202a0
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/metadata/TabletFileUtil.java
@@ -0,0 +1,40 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.accumulo.core.metadata;
+
+import org.apache.hadoop.fs.Path;
+
+/**
+ * Utility class for validation of metadata tablet files.
+ */
+public class TabletFileUtil {
+
+  /**
+   * Validate if string is a valid path. Return normalized string or throw exception if not valid.
+   * This was added to facilitate more use of TabletFile over String but this puts the validation in
+   * one location in the case where TabletFile can't be used.
+   */
+  public static String validate(String path) {
+    Path p = new Path(path);
+    if (p.toUri().getScheme() == null) {
+      throw new IllegalArgumentException("Invalid path provided, no scheme in " + path);
+    }
+    return p.toString();
+  }
+}
diff --git a/core/src/main/java/org/apache/accumulo/core/metadata/schema/TabletFile.java b/core/src/main/java/org/apache/accumulo/core/metadata/schema/TabletFile.java
index 36667d4..85df8b6 100644
--- a/core/src/main/java/org/apache/accumulo/core/metadata/schema/TabletFile.java
+++ b/core/src/main/java/org/apache/accumulo/core/metadata/schema/TabletFile.java
@@ -102,12 +102,12 @@
     return normalizedPath;
   }
 
-  public Text meta() {
-    return new Text(metadataEntry);
+  public Path getPath() {
+    return new Path(normalizedPath);
   }
 
-  public Path path() {
-    return metaPath;
+  public Text meta() {
+    return new Text(metadataEntry);
   }
 
   @Override
diff --git a/server/base/src/main/java/org/apache/accumulo/server/fs/FileRef.java b/server/base/src/main/java/org/apache/accumulo/server/fs/FileRef.java
index f7cef76..1f81fd4 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/fs/FileRef.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/fs/FileRef.java
@@ -18,9 +18,8 @@
  */
 package org.apache.accumulo.server.fs;
 
-import org.apache.accumulo.core.data.Key;
-import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.metadata.schema.Ample;
+import org.apache.accumulo.core.metadata.schema.TabletFile;
 import org.apache.accumulo.server.fs.VolumeManager.FileType;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.Text;
@@ -35,12 +34,8 @@
   private Path fullReference; // something like hdfs://nn:9001/accumulo/tables/2/d-00000/A00001.rf
   private Path suffix;
 
-  public FileRef(VolumeManager fs, Key key) {
-    this(key.getColumnQualifier().toString(), fs.getFullPath(key));
-  }
-
-  public FileRef(VolumeManager fs, String metaReference, TableId tableId) {
-    this(metaReference, fs.getFullPath(tableId, metaReference));
+  public FileRef(TabletFile tabletFile) {
+    this(tabletFile.getMetadataEntry(), tabletFile.getPath());
   }
 
   public FileRef(String metaReference, Path fullReference) {
diff --git a/server/base/src/main/java/org/apache/accumulo/server/fs/VolumeManager.java b/server/base/src/main/java/org/apache/accumulo/server/fs/VolumeManager.java
index fc269e5..407b8ba 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/fs/VolumeManager.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/fs/VolumeManager.java
@@ -24,8 +24,6 @@
 import java.util.Collection;
 
 import org.apache.accumulo.core.conf.AccumuloConfiguration;
-import org.apache.accumulo.core.data.Key;
-import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.volume.Volume;
 import org.apache.accumulo.core.volume.VolumeConfiguration;
 import org.apache.accumulo.server.ServerConstants;
@@ -162,14 +160,6 @@
   // forward to the appropriate FileSystem object
   FileStatus[] globStatus(Path path) throws IOException;
 
-  // Convert a file or directory metadata reference into a path
-  Path getFullPath(Key key);
-
-  Path getFullPath(TableId tableId, String path);
-
-  // Given a filename, figure out the qualified path given multiple namespaces
-  Path getFullPath(FileType fileType, String fileName);
-
   // forward to the appropriate FileSystem object
   ContentSummary getContentSummary(Path dir) throws IOException;
 
diff --git a/server/base/src/main/java/org/apache/accumulo/server/fs/VolumeManagerImpl.java b/server/base/src/main/java/org/apache/accumulo/server/fs/VolumeManagerImpl.java
index dbe75a4..c320c9b 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/fs/VolumeManagerImpl.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/fs/VolumeManagerImpl.java
@@ -34,10 +34,6 @@
 import org.apache.accumulo.core.conf.AccumuloConfiguration;
 import org.apache.accumulo.core.conf.DefaultConfiguration;
 import org.apache.accumulo.core.conf.Property;
-import org.apache.accumulo.core.data.Key;
-import org.apache.accumulo.core.data.TableId;
-import org.apache.accumulo.core.dataImpl.KeyExtent;
-import org.apache.accumulo.core.file.rfile.RFile;
 import org.apache.accumulo.core.volume.NonConfiguredVolume;
 import org.apache.accumulo.core.volume.Volume;
 import org.apache.accumulo.core.volume.VolumeConfiguration;
@@ -389,14 +385,6 @@
   }
 
   @Override
-  public Path getFullPath(Key key) {
-    // TODO sanity check col fam
-    String relPath = key.getColumnQualifierData().toString();
-    TableId tableId = KeyExtent.tableOfMetadataRow(key.getRow());
-    return getFullPath(tableId, relPath);
-  }
-
-  @Override
   public Path matchingFileSystem(Path source, String[] options) {
     try {
       if (ViewFSUtils.isViewFS(source, hadoopConf)) {
@@ -420,66 +408,6 @@
   }
 
   @Override
-  public Path getFullPath(TableId tableId, String path) {
-    if (path.contains(":"))
-      return new Path(path);
-
-    if (path.startsWith("../"))
-      path = path.substring(2);
-    else if (path.startsWith("/"))
-      path = "/" + tableId.canonical() + path;
-    else
-      throw new IllegalArgumentException("Unexpected path prefix " + path);
-
-    return getFullPath(FileType.TABLE, path);
-  }
-
-  private static final String RFILE_SUFFIX = "." + RFile.EXTENSION;
-
-  @Override
-  public Path getFullPath(FileType fileType, String path) {
-    int colon = path.indexOf(':');
-    if (colon > -1) {
-      // Check if this is really an absolute path or if this is a 1.4 style relative path for a WAL
-      if (fileType == FileType.WAL && path.charAt(colon + 1) != '/') {
-        path = path.substring(path.indexOf('/'));
-      } else {
-        return new Path(path);
-      }
-    }
-
-    if (path.startsWith("/"))
-      path = path.substring(1);
-
-    // ACCUMULO-2974 To ensure that a proper absolute path is created, the caller needs to include
-    // the table ID
-    // in the relative path. Fail when this doesn't appear to happen.
-    if (fileType == FileType.TABLE) {
-      // Trailing slash doesn't create an additional element
-      String[] pathComponents = path.split(Path.SEPARATOR);
-
-      // Is an rfile
-      if (path.endsWith(RFILE_SUFFIX)) {
-        if (pathComponents.length < 3) {
-          throw new IllegalArgumentException("Fewer components in file path than expected");
-        }
-      } else {
-        // is a directory
-        if (pathComponents.length < 2) {
-          throw new IllegalArgumentException("Fewer components in directory path than expected");
-        }
-      }
-    }
-
-    // normalize the path
-    Path fullPath = new Path(defaultVolume.getBasePath(), fileType.getDirectory());
-    fullPath = new Path(fullPath, path);
-
-    FileSystem fs = getVolumeByPath(fullPath).getFileSystem();
-    return fs.makeQualified(fullPath);
-  }
-
-  @Override
   public ContentSummary getContentSummary(Path dir) throws IOException {
     return getVolumeByPath(dir).getFileSystem().getContentSummary(dir);
   }
diff --git a/server/base/src/main/java/org/apache/accumulo/server/metadata/ServerAmpleImpl.java b/server/base/src/main/java/org/apache/accumulo/server/metadata/ServerAmpleImpl.java
index 1cec8a3..d8b0e55 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/metadata/ServerAmpleImpl.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/metadata/ServerAmpleImpl.java
@@ -40,13 +40,13 @@
 import org.apache.accumulo.core.dataImpl.KeyExtent;
 import org.apache.accumulo.core.metadata.MetadataTable;
 import org.apache.accumulo.core.metadata.RootTable;
+import org.apache.accumulo.core.metadata.TabletFileUtil;
 import org.apache.accumulo.core.metadata.schema.Ample;
 import org.apache.accumulo.core.metadata.schema.AmpleImpl;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.DeletesSection;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.fate.zookeeper.ZooUtil;
 import org.apache.accumulo.server.ServerContext;
-import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.Text;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -117,7 +117,7 @@
 
     try (BatchWriter writer = createWriter(tableId)) {
       for (Ample.FileMeta file : candidates) {
-        writer.addMutation(createDeleteMutation(context, tableId, file.path().toString()));
+        writer.addMutation(createDeleteMutation(file.path().toString()));
       }
     } catch (MutationsRejectedException e) {
       throw new RuntimeException(e);
@@ -192,10 +192,9 @@
     }
   }
 
-  public static Mutation createDeleteMutation(ServerContext context, TableId tableId,
-      String pathToRemove) {
-    Path path = context.getVolumeManager().getFullPath(tableId, pathToRemove);
-    Mutation delFlag = new Mutation(new Text(DeletesSection.encodeRow(path.toString())));
+  public static Mutation createDeleteMutation(String pathToRemove) {
+    String path = TabletFileUtil.validate(pathToRemove);
+    Mutation delFlag = new Mutation(new Text(DeletesSection.encodeRow(path)));
     delFlag.put(EMPTY_TEXT, EMPTY_TEXT, DeletesSection.SkewedKeyValue.NAME);
     return delFlag;
   }
diff --git a/server/base/src/main/java/org/apache/accumulo/server/util/LocalityCheck.java b/server/base/src/main/java/org/apache/accumulo/server/util/LocalityCheck.java
index dc99a80..985152f 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/util/LocalityCheck.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/util/LocalityCheck.java
@@ -29,6 +29,7 @@
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.metadata.MetadataTable;
+import org.apache.accumulo.core.metadata.TabletFileUtil;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.DataFileColumnFamily;
@@ -69,8 +70,7 @@
             addBlocks(fs, host, files, totalBlocks, localBlocks);
             files.clear();
           } else if (key.compareColumnFamily(DataFileColumnFamily.NAME) == 0) {
-
-            files.add(fs.getFullPath(key).toString());
+            files.add(TabletFileUtil.validate(key.getColumnQualifierData().toString()));
           }
         }
         System.out.println(" Server         %local  total blocks");
diff --git a/server/base/src/main/java/org/apache/accumulo/server/util/MasterMetadataUtil.java b/server/base/src/main/java/org/apache/accumulo/server/util/MasterMetadataUtil.java
index 63a5fe8..9220054 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/util/MasterMetadataUtil.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/util/MasterMetadataUtil.java
@@ -41,6 +41,7 @@
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.dataImpl.KeyExtent;
 import org.apache.accumulo.core.metadata.MetadataTable;
+import org.apache.accumulo.core.metadata.TabletFileUtil;
 import org.apache.accumulo.core.metadata.schema.Ample.TabletMutator;
 import org.apache.accumulo.core.metadata.schema.DataFileValue;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection;
@@ -149,7 +150,9 @@
 
           for (Entry<Key,Value> entry : scanner3) {
             if (entry.getKey().compareColumnFamily(DataFileColumnFamily.NAME) == 0) {
-              origDatafileSizes.put(new FileRef(context.getVolumeManager(), entry.getKey()),
+              String tabletFilePath =
+                  TabletFileUtil.validate(entry.getKey().getColumnQualifierData().toString());
+              origDatafileSizes.put(new FileRef(tabletFilePath),
                   new DataFileValue(entry.getValue().get()));
             }
           }
diff --git a/server/base/src/main/java/org/apache/accumulo/server/util/MetadataTableUtil.java b/server/base/src/main/java/org/apache/accumulo/server/util/MetadataTableUtil.java
index 2613107..b37ef66 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/util/MetadataTableUtil.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/util/MetadataTableUtil.java
@@ -64,6 +64,7 @@
 import org.apache.accumulo.core.dataImpl.KeyExtent;
 import org.apache.accumulo.core.metadata.MetadataTable;
 import org.apache.accumulo.core.metadata.RootTable;
+import org.apache.accumulo.core.metadata.TabletFileUtil;
 import org.apache.accumulo.core.metadata.schema.Ample.TabletMutator;
 import org.apache.accumulo.core.metadata.schema.DataFileValue;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema;
@@ -87,7 +88,6 @@
 import org.apache.accumulo.fate.zookeeper.ZooLock;
 import org.apache.accumulo.server.ServerContext;
 import org.apache.accumulo.server.fs.FileRef;
-import org.apache.accumulo.server.fs.VolumeManager;
 import org.apache.accumulo.server.gc.GcVolumeUtil;
 import org.apache.accumulo.server.metadata.ServerAmpleImpl;
 import org.apache.hadoop.io.Text;
@@ -278,15 +278,12 @@
     // TODO could use batch writer,would need to handle failure and retry like update does -
     // ACCUMULO-1294
     for (FileRef pathToRemove : datafilesToDelete) {
-      update(context,
-          ServerAmpleImpl.createDeleteMutation(context, tableId, pathToRemove.path().toString()),
-          extent);
+      update(context, ServerAmpleImpl.createDeleteMutation(pathToRemove.path().toString()), extent);
     }
   }
 
   public static void addDeleteEntry(ServerContext context, TableId tableId, String path) {
-    update(context, ServerAmpleImpl.createDeleteMutation(context, tableId, path),
-        new KeyExtent(tableId, null, null));
+    update(context, ServerAmpleImpl.createDeleteMutation(path), new KeyExtent(tableId, null, null));
   }
 
   public static void removeScanFiles(KeyExtent extent, Set<FileRef> scanFiles,
@@ -367,15 +364,15 @@
           Key key = cell.getKey();
 
           if (key.getColumnFamily().equals(DataFileColumnFamily.NAME)) {
-            FileRef ref = new FileRef(context.getVolumeManager(), key);
-            bw.addMutation(
-                ServerAmpleImpl.createDeleteMutation(context, tableId, ref.meta().toString()));
+            FileRef ref =
+                new FileRef(TabletFileUtil.validate(key.getColumnQualifierData().toString()));
+            bw.addMutation(ServerAmpleImpl.createDeleteMutation(ref.meta().toString()));
           }
 
           if (TabletsSection.ServerColumnFamily.DIRECTORY_COLUMN.hasColumns(key)) {
             String uri =
                 GcVolumeUtil.getDeleteTabletOnAllVolumesUri(tableId, cell.getValue().toString());
-            bw.addMutation(ServerAmpleImpl.createDeleteMutation(context, tableId, uri));
+            bw.addMutation(ServerAmpleImpl.createDeleteMutation(uri));
           }
         }
 
@@ -412,8 +409,6 @@
     ArrayList<LogEntry> result = new ArrayList<>();
     TreeMap<FileRef,DataFileValue> sizes = new TreeMap<>();
 
-    VolumeManager fs = context.getVolumeManager();
-
     TabletMetadata tablet = context.getAmple().readTablet(extent, FILES, LOGS, PREV_ROW, DIR);
 
     if (!tablet.getExtent().equals(extent))
@@ -421,10 +416,7 @@
 
     result.addAll(tablet.getLogs());
 
-    tablet.getFilesMap().forEach((k, v) -> {
-      sizes.put(new FileRef(k.getMetadataEntry(),
-          fs.getFullPath(tablet.getTableId(), k.getMetadataEntry())), v);
-    });
+    tablet.getFilesMap().forEach((tf, v) -> sizes.put(new FileRef(tf.getMetadataEntry()), v));
 
     return new Pair<>(result, sizes);
   }
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 86eeb37..0b82d82 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
@@ -40,6 +40,7 @@
 import org.apache.accumulo.core.dataImpl.KeyExtent;
 import org.apache.accumulo.core.metadata.MetadataTable;
 import org.apache.accumulo.core.metadata.RootTable;
+import org.apache.accumulo.core.metadata.TabletFileUtil;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.DataFileColumnFamily;
 import org.apache.accumulo.core.security.Authorizations;
@@ -145,7 +146,7 @@
 
       count++;
       Key key = entry.getKey();
-      Path map = fs.getFullPath(key);
+      Path map = new Path(TabletFileUtil.validate(key.getColumnQualifierData().toString()));
 
       synchronized (processing) {
         while (processing.size() >= 64 || processing.contains(map))
diff --git a/server/base/src/test/java/org/apache/accumulo/server/fs/VolumeManagerImplTest.java b/server/base/src/test/java/org/apache/accumulo/server/fs/VolumeManagerImplTest.java
index 48613da..32d51856 100644
--- a/server/base/src/test/java/org/apache/accumulo/server/fs/VolumeManagerImplTest.java
+++ b/server/base/src/test/java/org/apache/accumulo/server/fs/VolumeManagerImplTest.java
@@ -18,7 +18,6 @@
  */
 package org.apache.accumulo.server.fs;
 
-import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 
 import java.util.Arrays;
@@ -27,9 +26,7 @@
 import org.apache.accumulo.core.conf.ConfigurationCopy;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.data.TableId;
-import org.apache.accumulo.server.fs.VolumeManager.FileType;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.Path;
 import org.junit.Before;
 import org.junit.Rule;
 import org.junit.Test;
@@ -49,30 +46,6 @@
   }
 
   @Test
-  public void defaultTabletDirWithoutTableId() {
-    thrown.expect(IllegalArgumentException.class);
-    fs.getFullPath(FileType.TABLE, "/default_tablet/");
-  }
-
-  @Test
-  public void tabletDirWithoutTableId() {
-    thrown.expect(IllegalArgumentException.class);
-    fs.getFullPath(FileType.TABLE, "/t-0000001/");
-  }
-
-  @Test
-  public void defaultTabletFileWithoutTableId() {
-    thrown.expect(IllegalArgumentException.class);
-    fs.getFullPath(FileType.TABLE, "/default_tablet/C0000001.rf");
-  }
-
-  @Test
-  public void tabletFileWithoutTableId() {
-    thrown.expect(IllegalArgumentException.class);
-    fs.getFullPath(FileType.TABLE, "/t-0000001/C0000001.rf");
-  }
-
-  @Test
   public void invalidChooserConfigured() throws Exception {
     List<String> volumes = Arrays.asList("file://one/", "file://two/", "file://three/");
     ConfigurationCopy conf = new ConfigurationCopy();
@@ -85,32 +58,6 @@
   }
 
   @Test
-  public void tabletDirWithTableId() throws Exception {
-    String basePath = fs.getDefaultVolume().getBasePath();
-    String scheme = fs.getDefaultVolume().getFileSystem().getUri().toURL().getProtocol();
-    Path expectedBase = new Path(scheme + ":" + basePath, FileType.TABLE.getDirectory());
-    List<String> pathsToTest =
-        Arrays.asList("1/default_tablet", "1/default_tablet/", "1/t-0000001");
-    for (String pathToTest : pathsToTest) {
-      Path fullPath = fs.getFullPath(FileType.TABLE, pathToTest);
-      assertEquals(new Path(expectedBase, pathToTest), fullPath);
-    }
-  }
-
-  @Test
-  public void tabletFileWithTableId() throws Exception {
-    String basePath = fs.getDefaultVolume().getBasePath();
-    String scheme = fs.getDefaultVolume().getFileSystem().getUri().toURL().getProtocol();
-    Path expectedBase = new Path(scheme + ":" + basePath, FileType.TABLE.getDirectory());
-    List<String> pathsToTest =
-        Arrays.asList("1/default_tablet/C0000001.rf", "1/t-0000001/C0000001.rf");
-    for (String pathToTest : pathsToTest) {
-      Path fullPath = fs.getFullPath(FileType.TABLE, pathToTest);
-      assertEquals(new Path(expectedBase, pathToTest), fullPath);
-    }
-  }
-
-  @Test
   public void noViewFS() throws Exception {
     ConfigurationCopy conf = new ConfigurationCopy();
     conf.set(Property.INSTANCE_VOLUMES, "viewfs://dummy");
diff --git a/server/gc/src/main/java/org/apache/accumulo/gc/SimpleGarbageCollector.java b/server/gc/src/main/java/org/apache/accumulo/gc/SimpleGarbageCollector.java
index f87cbf6..2c1cfbd 100644
--- a/server/gc/src/main/java/org/apache/accumulo/gc/SimpleGarbageCollector.java
+++ b/server/gc/src/main/java/org/apache/accumulo/gc/SimpleGarbageCollector.java
@@ -55,6 +55,7 @@
 import org.apache.accumulo.core.master.state.tables.TableState;
 import org.apache.accumulo.core.metadata.MetadataTable;
 import org.apache.accumulo.core.metadata.RootTable;
+import org.apache.accumulo.core.metadata.TabletFileUtil;
 import org.apache.accumulo.core.metadata.schema.Ample;
 import org.apache.accumulo.core.metadata.schema.Ample.DataLevel;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema;
@@ -319,9 +320,9 @@
               // of deleting something that should not be deleted. Must not change value of delete
               // variable because thats whats stored in metadata table.
               log.debug("Volume replaced {} -> {}", delete, switchedDelete);
-              fullPath = fs.getFullPath(FileType.TABLE, switchedDelete);
+              fullPath = new Path(TabletFileUtil.validate(switchedDelete));
             } else {
-              fullPath = fs.getFullPath(FileType.TABLE, delete);
+              fullPath = new Path(TabletFileUtil.validate(delete));
             }
 
             for (Path pathToDel : GcVolumeUtil.expandAllVolumesUri(fs, fullPath)) {
@@ -715,7 +716,7 @@
     while (cdIter.hasNext()) {
       Entry<String,String> entry = cdIter.next();
       String relPath = entry.getKey();
-      Path absPath = fs.getFullPath(FileType.TABLE, entry.getValue());
+      Path absPath = new Path(entry.getValue());
 
       if (isDir(relPath)) {
         lastDirRel = relPath;
diff --git a/server/gc/src/test/java/org/apache/accumulo/gc/SimpleGarbageCollectorTest.java b/server/gc/src/test/java/org/apache/accumulo/gc/SimpleGarbageCollectorTest.java
index d02b77e..0885d35 100644
--- a/server/gc/src/test/java/org/apache/accumulo/gc/SimpleGarbageCollectorTest.java
+++ b/server/gc/src/test/java/org/apache/accumulo/gc/SimpleGarbageCollectorTest.java
@@ -21,7 +21,6 @@
 import static org.apache.accumulo.gc.SimpleGarbageCollector.CANDIDATE_MEMORY_PERCENTAGE;
 import static org.easymock.EasyMock.anyObject;
 import static org.easymock.EasyMock.createMock;
-import static org.easymock.EasyMock.eq;
 import static org.easymock.EasyMock.expect;
 import static org.easymock.EasyMock.expectLastCall;
 import static org.easymock.EasyMock.getCurrentArguments;
@@ -50,7 +49,6 @@
 import org.apache.accumulo.core.volume.Volume;
 import org.apache.accumulo.server.ServerContext;
 import org.apache.accumulo.server.fs.VolumeManager;
-import org.apache.accumulo.server.fs.VolumeManager.FileType;
 import org.apache.accumulo.server.gc.GcVolumeUtil;
 import org.apache.accumulo.server.security.SystemCredentials;
 import org.apache.hadoop.fs.Path;
@@ -185,8 +183,6 @@
 
     VolumeManager volMgr2 = createMock(VolumeManager.class);
     expect(volMgr2.getVolumes()).andReturn(vols).anyTimes();
-    expect(volMgr2.getFullPath(eq(FileType.TABLE), anyObject()))
-        .andAnswer(() -> new Path(getCurrentArguments()[1].toString())).anyTimes();
 
     replay(vol1, vol2, volMgr2);
 
diff --git a/server/master/src/main/java/org/apache/accumulo/master/TabletGroupWatcher.java b/server/master/src/main/java/org/apache/accumulo/master/TabletGroupWatcher.java
index f791692..43004ae 100644
--- a/server/master/src/main/java/org/apache/accumulo/master/TabletGroupWatcher.java
+++ b/server/master/src/main/java/org/apache/accumulo/master/TabletGroupWatcher.java
@@ -57,6 +57,7 @@
 import org.apache.accumulo.core.master.thrift.TabletServerStatus;
 import org.apache.accumulo.core.metadata.MetadataTable;
 import org.apache.accumulo.core.metadata.RootTable;
+import org.apache.accumulo.core.metadata.TabletFileUtil;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.ChoppedColumnFamily;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.CurrentLocationColumnFamily;
@@ -614,7 +615,8 @@
       for (Entry<Key,Value> entry : scanner) {
         Key key = entry.getKey();
         if (key.compareColumnFamily(DataFileColumnFamily.NAME) == 0) {
-          datafiles.add(new FileRef(this.master.fs, key));
+          datafiles
+              .add(new FileRef(TabletFileUtil.validate(key.getColumnQualifierData().toString())));
           if (datafiles.size() > 1000) {
             MetadataTableUtil.addDeleteEntries(extent, datafiles, master.getContext());
             datafiles.clear();
@@ -714,8 +716,7 @@
         } else if (TabletsSection.ServerColumnFamily.DIRECTORY_COLUMN.hasColumns(key)) {
           String uri =
               GcVolumeUtil.getDeleteTabletOnAllVolumesUri(range.getTableId(), value.toString());
-          bw.addMutation(
-              ServerAmpleImpl.createDeleteMutation(master.getContext(), range.getTableId(), uri));
+          bw.addMutation(ServerAmpleImpl.createDeleteMutation(uri));
         }
       }
 
diff --git a/server/master/src/main/java/org/apache/accumulo/master/recovery/RecoveryManager.java b/server/master/src/main/java/org/apache/accumulo/master/recovery/RecoveryManager.java
index 2853d68..03c07bd 100644
--- a/server/master/src/main/java/org/apache/accumulo/master/recovery/RecoveryManager.java
+++ b/server/master/src/main/java/org/apache/accumulo/master/recovery/RecoveryManager.java
@@ -168,7 +168,7 @@
 
         String[] parts = walog.split("/");
         String sortId = parts[parts.length - 1];
-        String filename = master.getFileSystem().getFullPath(FileType.WAL, walog).toString();
+        String filename = new Path(walog).toString();
         String dest = RecoveryPath.getRecoveryPath(new Path(filename)).toString();
 
         boolean sortQueued;
diff --git a/server/master/src/main/java/org/apache/accumulo/master/tableOps/bulkVer1/CopyFailed.java b/server/master/src/main/java/org/apache/accumulo/master/tableOps/bulkVer1/CopyFailed.java
index 26c1f08..e84ffa0 100644
--- a/server/master/src/main/java/org/apache/accumulo/master/tableOps/bulkVer1/CopyFailed.java
+++ b/server/master/src/main/java/org/apache/accumulo/master/tableOps/bulkVer1/CopyFailed.java
@@ -37,6 +37,7 @@
 import org.apache.accumulo.core.dataImpl.KeyExtent;
 import org.apache.accumulo.core.master.thrift.BulkImportState;
 import org.apache.accumulo.core.metadata.MetadataTable;
+import org.apache.accumulo.core.metadata.TabletFileUtil;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.BulkFileColumnFamily;
 import org.apache.accumulo.core.security.Authorizations;
@@ -126,7 +127,8 @@
 
       for (Entry<Key,Value> entry : mscanner) {
         if (BulkFileColumnFamily.getBulkLoadTid(entry.getValue()) == tid) {
-          FileRef loadedFile = new FileRef(fs, entry.getKey());
+          FileRef loadedFile = new FileRef(
+              TabletFileUtil.validate(entry.getKey().getColumnQualifierData().toString()));
           String absPath = failures.remove(loadedFile);
           if (absPath != null) {
             loadedFailures.put(loadedFile, absPath);
diff --git a/server/master/src/main/java/org/apache/accumulo/master/tableOps/tableExport/WriteExportFiles.java b/server/master/src/main/java/org/apache/accumulo/master/tableOps/tableExport/WriteExportFiles.java
index 7c6797e..21f5397 100644
--- a/server/master/src/main/java/org/apache/accumulo/master/tableOps/tableExport/WriteExportFiles.java
+++ b/server/master/src/main/java/org/apache/accumulo/master/tableOps/tableExport/WriteExportFiles.java
@@ -50,6 +50,7 @@
 import org.apache.accumulo.core.dataImpl.KeyExtent;
 import org.apache.accumulo.core.master.state.tables.TableState;
 import org.apache.accumulo.core.metadata.MetadataTable;
+import org.apache.accumulo.core.metadata.TabletFileUtil;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.DataFileColumnFamily;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.LogColumnFamily;
@@ -231,7 +232,7 @@
       entry.getValue().write(dataOut);
 
       if (entry.getKey().getColumnFamily().equals(DataFileColumnFamily.NAME)) {
-        String path = fs.getFullPath(entry.getKey()).toString();
+        String path = TabletFileUtil.validate(entry.getKey().getColumnQualifierData().toString());
         String[] tokens = path.split("/");
         if (tokens.length < 1) {
           throw new RuntimeException("Illegal path " + path);
diff --git a/server/master/src/main/java/org/apache/accumulo/master/upgrade/Upgrader9to10.java b/server/master/src/main/java/org/apache/accumulo/master/upgrade/Upgrader9to10.java
index 68c7db8..329a160 100644
--- a/server/master/src/main/java/org/apache/accumulo/master/upgrade/Upgrader9to10.java
+++ b/server/master/src/main/java/org/apache/accumulo/master/upgrade/Upgrader9to10.java
@@ -414,8 +414,7 @@
 
           String updatedDel = switchToAllVolumes(olddelete);
 
-          writer
-              .addMutation(ServerAmpleImpl.createDeleteMutation(ctx, level.tableId(), updatedDel));
+          writer.addMutation(ServerAmpleImpl.createDeleteMutation(updatedDel));
         }
         writer.flush();
         // if nothing thrown then we're good so mark all deleted
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 cf823f7..01696c6 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
@@ -186,7 +186,6 @@
 import org.apache.accumulo.server.fs.VolumeChooserEnvironment;
 import org.apache.accumulo.server.fs.VolumeChooserEnvironmentImpl;
 import org.apache.accumulo.server.fs.VolumeManager;
-import org.apache.accumulo.server.fs.VolumeManager.FileType;
 import org.apache.accumulo.server.log.SortedLogState;
 import org.apache.accumulo.server.log.WalStateManager;
 import org.apache.accumulo.server.log.WalStateManager.WalMarkerException;
@@ -3217,7 +3216,7 @@
     Collections.sort(sorted, (e1, e2) -> (int) (e1.timestamp - e2.timestamp));
     for (LogEntry entry : sorted) {
       Path recovery = null;
-      Path finished = RecoveryPath.getRecoveryPath(fs.getFullPath(FileType.WAL, entry.filename));
+      Path finished = RecoveryPath.getRecoveryPath(new Path(entry.filename));
       finished = SortedLogState.getFinishedMarkerPath(finished);
       TabletServer.log.debug("Looking for " + finished);
       if (fs.exists(finished)) {
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/TabletData.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/TabletData.java
index 198d5b2..a6077ca 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/TabletData.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/TabletData.java
@@ -58,19 +58,17 @@
     this.flushID = meta.getFlushId().orElse(-1);
     this.directoryName = meta.getDirName();
     this.logEntries.addAll(meta.getLogs());
-    meta.getScans().forEach(tabletFile -> scanFiles
-        .add(new FileRef(fs, tabletFile.getMetadataEntry(), meta.getTableId())));
+    meta.getScans().forEach(tabletFile -> scanFiles.add(new FileRef(tabletFile)));
 
     if (meta.getLast() != null)
       this.lastLocation = new TServerInstance(meta.getLast());
 
     meta.getFilesMap().forEach((tabletFile, dfv) -> {
-      dataFiles.put(new FileRef(fs, tabletFile.getMetadataEntry(), meta.getTableId()), dfv);
+      dataFiles.put(new FileRef(tabletFile), dfv);
     });
 
     meta.getLoaded().forEach((path, txid) -> {
-      bulkImported.computeIfAbsent(txid, k -> new ArrayList<FileRef>())
-          .add(new FileRef(fs, path, meta.getTableId()));
+      bulkImported.computeIfAbsent(txid, k -> new ArrayList<FileRef>()).add(new FileRef(path));
     });
   }
 
diff --git a/server/tserver/src/test/java/org/apache/accumulo/tserver/TabletServerSyncCheckTest.java b/server/tserver/src/test/java/org/apache/accumulo/tserver/TabletServerSyncCheckTest.java
index d50205c..ba1d1e4 100644
--- a/server/tserver/src/test/java/org/apache/accumulo/tserver/TabletServerSyncCheckTest.java
+++ b/server/tserver/src/test/java/org/apache/accumulo/tserver/TabletServerSyncCheckTest.java
@@ -22,8 +22,6 @@
 import java.util.Map;
 
 import org.apache.accumulo.core.conf.ConfigurationCopy;
-import org.apache.accumulo.core.data.Key;
-import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.volume.Volume;
 import org.apache.accumulo.core.volume.VolumeImpl;
 import org.apache.accumulo.server.fs.VolumeChooserEnvironment;
@@ -180,21 +178,6 @@
     }
 
     @Override
-    public Path getFullPath(Key key) {
-      return null;
-    }
-
-    @Override
-    public Path getFullPath(TableId tableId, String path) {
-      return null;
-    }
-
-    @Override
-    public Path getFullPath(FileType fileType, String fileName) {
-      return null;
-    }
-
-    @Override
     public ContentSummary getContentSummary(Path dir) {
       return null;
     }
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/GarbageCollectorIT.java b/test/src/main/java/org/apache/accumulo/test/functional/GarbageCollectorIT.java
index 8a475fc..3df5b4e 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/GarbageCollectorIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/GarbageCollectorIT.java
@@ -311,8 +311,8 @@
       for (int i = 0; i < 100000; ++i) {
         String longpath = "aaaaaaaaaabbbbbbbbbbccccccccccddddddddddeeeeeeeeee"
             + "ffffffffffgggggggggghhhhhhhhhhiiiiiiiiiijjjjjjjjjj";
-        Mutation delFlag = ServerAmpleImpl.createDeleteMutation(getServerContext(),
-            MetadataTable.ID, String.format("/%020d/%s", i, longpath));
+        Mutation delFlag =
+            ServerAmpleImpl.createDeleteMutation(String.format("/%020d/%s", i, longpath));
         bw.addMutation(delFlag);
       }
     }
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 5566bf7..f043f6c 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
@@ -179,9 +179,8 @@
       KeyExtent extent) {
     Map<Long,List<FileRef>> bulkFiles = new HashMap<>();
 
-    context.getAmple().readTablet(extent).getLoaded()
-        .forEach((path, txid) -> bulkFiles.computeIfAbsent(txid, k -> new ArrayList<FileRef>())
-            .add(new FileRef(context.getVolumeManager(), path, extent.getTableId())));
+    context.getAmple().readTablet(extent).getLoaded().forEach((path, txid) -> bulkFiles
+        .computeIfAbsent(txid, k -> new ArrayList<FileRef>()).add(new FileRef(path)));
 
     return bulkFiles;
   }