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 f9897862dd Stronger type checking and other WAL improvements (#4082)
f9897862dd is described below

commit f9897862dd4e6ff4892239ff5ebeb8ed6e34bc68
Author: Christopher Tubbs <ctubb...@apache.org>
AuthorDate: Tue Dec 19 01:24:23 2023 -0500

    Stronger type checking and other WAL improvements (#4082)
    
    * Use LogEntry type in more places where just the logEntry's path as
      a String was being used
    * Keep the UUID and HostAndPort in LogEntry that are created for
      validation, rather than their String equivalents
    * Remove no longer needed deleteWal(String) and create a
      deleteFromMutation method on LogEntry to keep the schema
      serialization/deserialization in LogEntry (may want to move that to
      LogColumnFamily class in the MetadataSchema later, but for now, at
      least it's in one place)
    * Use private constructor in LogEntry, do validation in a fromPath
      static method before calling the constructor with the strongly typed
      results
    * Update javadocs to clarify the path format
    * Fix the validation of the host+port in the path (it's a plus sign
      not a colon, because colons weren't valid characters in HDFS, so we
      used a plus instead)
    * Simplify VolumeUtil switchVolumes logic that was previously
      complicated because an old LogEntry format could contain multiple
      paths per entry and that's no longer the case
    * Clean up DfsLogger a little bit to remove some old stuff that wasn't
      needed: remove ServerResources used for testing when a mock
      ServerContext would suffice, pass LogEntry instead of Strings for the
      path and its metadata format (the meta reference being passed
      wasn't actually what was in the table anyway, it was coming from
      LogEntry, which was computing it, and Ample wasn't using this meta
      for deletions/cleanup anyway - may be an existing bug; if so, that
      will be a follow-on), stop using toString() to compute serialization
      and remove the redundant use of the logger name itself to try to
      compute the serialization in that toString() method
    * Simplify names to use "path" instead of "filePath" or "filename"
    * Remove unneeded toString calls
    * Avoid use of Java NIO Path type in LogEntryTest, which is confusing
      since the paths actually come from HDFS Path
    * Add check to verify `+` exists for port delimiter
---
 .../apache/accumulo/core/logging/TabletLogger.java |   3 +-
 .../accumulo/core/metadata/schema/Ample.java       |   4 +-
 .../accumulo/core/tabletserver/log/LogEntry.java   | 138 +++++++++++++-------
 .../core/metadata/schema/TabletMetadataTest.java   |   7 +-
 .../core/tabletserver/log/LogEntryTest.java        | 139 ++++++++++-----------
 .../org/apache/accumulo/server/fs/VolumeUtil.java  |  21 +---
 .../server/manager/state/MetaDataStateStore.java   |   2 +-
 .../server/manager/state/MetaDataTableScanner.java |   2 +-
 .../server/manager/state/ZooTabletStateStore.java  |   6 +-
 .../server/metadata/TabletMutatorBase.java         |  10 +-
 .../accumulo/server/util/ListVolumesUsed.java      |   2 +-
 .../accumulo/server/util/ManagerMetadataUtil.java  |   3 +-
 .../apache/accumulo/server/fs/VolumeUtilTest.java  |  12 +-
 .../accumulo/tserver/TabletClientHandler.java      |   8 +-
 .../org/apache/accumulo/tserver/TabletServer.java  |  17 +--
 .../org/apache/accumulo/tserver/log/DfsLogger.java |  91 +++++---------
 .../accumulo/tserver/log/TabletServerLogger.java   |  33 +++--
 .../accumulo/tserver/tablet/DatafileManager.java   |   3 +-
 .../org/apache/accumulo/tserver/tablet/Tablet.java |  35 +++---
 .../accumulo/tserver/WalRemovalOrderTest.java      |  43 ++++---
 .../test/MissingWalHeaderCompletesRecoveryIT.java  |   4 +-
 .../apache/accumulo/test/manager/MergeStateIT.java |   5 +-
 22 files changed, 281 insertions(+), 307 deletions(-)

diff --git 
a/core/src/main/java/org/apache/accumulo/core/logging/TabletLogger.java 
b/core/src/main/java/org/apache/accumulo/core/logging/TabletLogger.java
index e632971cf5..44832db7f1 100644
--- a/core/src/main/java/org/apache/accumulo/core/logging/TabletLogger.java
+++ b/core/src/main/java/org/apache/accumulo/core/logging/TabletLogger.java
@@ -23,6 +23,7 @@ import static java.util.stream.Collectors.toList;
 import java.util.Collection;
 import java.util.List;
 import java.util.Optional;
+import java.util.UUID;
 import java.util.concurrent.TimeUnit;
 
 import org.apache.accumulo.core.client.admin.CompactionConfig;
@@ -161,7 +162,7 @@ public class TabletLogger {
 
   public static void recovering(KeyExtent extent, List<LogEntry> logEntries) {
     if (recoveryLog.isDebugEnabled()) {
-      List<String> logIds = 
logEntries.stream().map(LogEntry::getUniqueID).collect(toList());
+      List<UUID> logIds = 
logEntries.stream().map(LogEntry::getUniqueID).collect(toList());
       recoveryLog.debug("For {} recovering data from walogs: {}", extent, 
logIds);
     }
   }
diff --git 
a/core/src/main/java/org/apache/accumulo/core/metadata/schema/Ample.java 
b/core/src/main/java/org/apache/accumulo/core/metadata/schema/Ample.java
index f6014df81b..d551b342af 100644
--- a/core/src/main/java/org/apache/accumulo/core/metadata/schema/Ample.java
+++ b/core/src/main/java/org/apache/accumulo/core/metadata/schema/Ample.java
@@ -292,9 +292,7 @@ public interface Ample {
 
     TabletMutator putWal(LogEntry logEntry);
 
-    TabletMutator deleteWal(String wal);
-
-    TabletMutator deleteWal(LogEntry logEntry);
+    TabletMutator deleteWal(LogEntry wal);
 
     TabletMutator putTime(MetadataTime time);
 
diff --git 
a/core/src/main/java/org/apache/accumulo/core/tabletserver/log/LogEntry.java 
b/core/src/main/java/org/apache/accumulo/core/tabletserver/log/LogEntry.java
index 73a9ecf979..dea6460eaf 100644
--- a/core/src/main/java/org/apache/accumulo/core/tabletserver/log/LogEntry.java
+++ b/core/src/main/java/org/apache/accumulo/core/tabletserver/log/LogEntry.java
@@ -27,69 +27,111 @@ import org.apache.accumulo.core.data.Mutation;
 import org.apache.accumulo.core.data.Value;
 import 
org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.LogColumnFamily;
 import org.apache.hadoop.io.Text;
+import org.checkerframework.checker.nullness.qual.NonNull;
 
+import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
 import com.google.common.net.HostAndPort;
 
-public class LogEntry {
+public final class LogEntry {
 
-  private final String filePath;
+  private final String path;
+  private final HostAndPort tserver;
+  private final UUID uniqueId;
 
-  public LogEntry(String filePath) {
-    validateFilePath(filePath);
-    this.filePath = filePath;
-  }
-
-  public String getFilePath() {
-    return this.filePath;
+  private LogEntry(String path, HostAndPort tserver, UUID uniqueId) {
+    this.path = path;
+    this.tserver = tserver;
+    this.uniqueId = uniqueId;
   }
 
   /**
-   * Validates the expected format of the file path. We expect the path to 
contain a tserver
-   * (host:port) followed by a UUID as the file name. For example,
-   * localhost:1234/927ba659-d109-4bce-b0a5-bcbbcb9942a2 is a valid file path.
+   * Creates a new LogEntry object after validating the expected format of the 
path. We expect the
+   * path to contain a tserver (host+port) followed by a UUID as the file name 
as the last two
+   * components.<br>
+   * For example, 
file:///some/dir/path/localhost+1234/927ba659-d109-4bce-b0a5-bcbbcb9942a2 is a
+   * valid path.
    *
-   * @param filePath path to validate
-   * @throws IllegalArgumentException if the filePath is invalid
+   * @param path path to validate
+   * @return an object representation of this log entry
+   * @throws IllegalArgumentException if the path is invalid
    */
-  private static void validateFilePath(String filePath) {
-    String[] parts = filePath.split("/");
+  public static LogEntry fromPath(String path) {
+    String[] parts = path.split("/");
 
     if (parts.length < 2) {
       throw new IllegalArgumentException(
-          "Invalid filePath format. The path should at least contain 
tserver/UUID.");
+          "Invalid path format. The path should end with tserver/UUID.");
     }
 
     String tserverPart = parts[parts.length - 2];
     String uuidPart = parts[parts.length - 1];
 
+    String badTServerMsg =
+        "Invalid tserver in path. Expected: host+port. Found '" + tserverPart 
+ "'";
+    if (tserverPart.contains(":") || !tserverPart.contains("+")) {
+      throw new IllegalArgumentException(badTServerMsg);
+    }
+    HostAndPort tserver;
     try {
-      HostAndPort.fromString(tserverPart);
+      tserver = HostAndPort.fromString(tserverPart.replace("+", ":"));
     } catch (IllegalArgumentException e) {
-      throw new IllegalArgumentException(
-          "Invalid tserver format in filePath. Expected format: host:port. 
Found '" + tserverPart
-              + "'");
+      throw new IllegalArgumentException(badTServerMsg);
     }
 
+    String badUuidMsg = "Expected valid UUID. Found '" + uuidPart + "'";
+    UUID uuid;
     try {
-      UUID.fromString(uuidPart);
+      uuid = UUID.fromString(uuidPart);
     } catch (IllegalArgumentException e) {
-      throw new IllegalArgumentException("Expected valid UUID. Found '" + 
uuidPart + "'");
+      throw new IllegalArgumentException(badUuidMsg);
+    }
+    if (!uuid.toString().equals(uuidPart)) {
+      throw new IllegalArgumentException(badUuidMsg);
     }
+
+    return new LogEntry(path, tserver, uuid);
   }
 
   /**
-   * Add LogEntry information to the provided mutation.
+   * Construct a new LogEntry object after deserializing it from a metadata 
entry.
    *
-   * @param mutation the mutation to update
+   * @param entry the metadata entry
+   * @return a new LogEntry object constructed from the path stored in the 
column qualifier
+   * @throws IllegalArgumentException if the path stored in the metadata entry 
is invalid or if the
+   *         serialized format of the entry is unrecognized
    */
-  public void addToMutation(Mutation mutation) {
-    
mutation.at().family(LogColumnFamily.NAME).qualifier(getColumnQualifier()).put(new
 Value());
+  public static LogEntry fromMetaWalEntry(Entry<Key,Value> entry) {
+    Text fam = entry.getKey().getColumnFamily();
+    Preconditions.checkArgument(LogColumnFamily.NAME.equals(fam),
+        "The provided metadata entry's column family is %s instead of %s", fam,
+        LogColumnFamily.NAME);
+    String qualifier = entry.getKey().getColumnQualifier().toString();
+    String[] parts = qualifier.split("/", 2);
+    Preconditions.checkArgument(parts.length == 2 && parts[0].equals("-"),
+        "Malformed write-ahead log %s", qualifier);
+    return fromPath(parts[1]);
+  }
+
+  @NonNull
+  @VisibleForTesting
+  HostAndPort getTServer() {
+    return tserver;
+  }
+
+  @NonNull
+  public String getPath() {
+    return path;
+  }
+
+  @NonNull
+  public UUID getUniqueID() {
+    return uniqueId;
   }
 
   @Override
   public String toString() {
-    return filePath;
+    return path;
   }
 
   @Override
@@ -97,33 +139,41 @@ public class LogEntry {
     if (this == other) {
       return true;
     }
-    if (!(other instanceof LogEntry)) {
-      return false;
+    if (other instanceof LogEntry) {
+      return path.equals(((LogEntry) other).path);
     }
-    LogEntry logEntry = (LogEntry) other;
-    return this.filePath.equals(logEntry.filePath);
+    return false;
   }
 
   @Override
   public int hashCode() {
-    return Objects.hash(filePath);
+    return Objects.hash(path);
   }
 
-  public static LogEntry fromMetaWalEntry(Entry<Key,Value> entry) {
-    String qualifier = entry.getKey().getColumnQualifier().toString();
-    String[] parts = qualifier.split("/", 2);
-    Preconditions.checkArgument(parts.length == 2 && parts[0].equals("-"),
-        "Malformed write-ahead log %s", qualifier);
-    return new LogEntry(parts[1]);
+  /**
+   * Get the Text that should be used as the column qualifier to store this as 
a metadata entry.
+   */
+  @VisibleForTesting
+  Text getColumnQualifier() {
+    return new Text("-/" + getPath());
   }
 
-  public String getUniqueID() {
-    String[] parts = filePath.split("/");
-    return parts[parts.length - 1];
+  /**
+   * Put a delete marker in the provided mutation for this LogEntry.
+   *
+   * @param mutation the mutation to update
+   */
+  public void deleteFromMutation(Mutation mutation) {
+    mutation.putDelete(LogColumnFamily.NAME, getColumnQualifier());
   }
 
-  public Text getColumnQualifier() {
-    return new Text("-/" + filePath);
+  /**
+   * Put this LogEntry into the provided mutation.
+   *
+   * @param mutation the mutation to update
+   */
+  public void addToMutation(Mutation mutation) {
+    mutation.put(LogColumnFamily.NAME, getColumnQualifier(), new Value());
   }
 
 }
diff --git 
a/core/src/test/java/org/apache/accumulo/core/metadata/schema/TabletMetadataTest.java
 
b/core/src/test/java/org/apache/accumulo/core/metadata/schema/TabletMetadataTest.java
index a4cb1b583c..5287d689ba 100644
--- 
a/core/src/test/java/org/apache/accumulo/core/metadata/schema/TabletMetadataTest.java
+++ 
b/core/src/test/java/org/apache/accumulo/core/metadata/schema/TabletMetadataTest.java
@@ -102,9 +102,9 @@ public class TabletMetadataTest {
 
     
mutation.at().family(LastLocationColumnFamily.NAME).qualifier("s000").put("server2:8555");
 
-    LogEntry le1 = new LogEntry("localhost:8020/" + UUID.randomUUID());
+    LogEntry le1 = LogEntry.fromPath("localhost+8020/" + UUID.randomUUID());
     le1.addToMutation(mutation);
-    LogEntry le2 = new LogEntry("localhost:8020/" + UUID.randomUUID());
+    LogEntry le2 = LogEntry.fromPath("localhost+8020/" + UUID.randomUUID());
     le2.addToMutation(mutation);
 
     StoredTabletFile sf1 = StoredTabletFile.of(new 
Path("hdfs://nn1/acc/tables/1/t-0001/sf1.rf"));
@@ -137,8 +137,7 @@ public class TabletMetadataTest {
     assertEquals(HostAndPort.fromParts("server2", 8555), 
tm.getLast().getHostAndPort());
     assertEquals("s000", tm.getLast().getSession());
     assertEquals(LocationType.LAST, tm.getLast().getType());
-    assertEquals(Set.of(le1.getColumnQualifier(), le2.getColumnQualifier()),
-        
tm.getLogs().stream().map(LogEntry::getColumnQualifier).collect(toSet()));
+    assertEquals(Set.of(le1, le2), tm.getLogs().stream().collect(toSet()));
     assertEquals(extent.prevEndRow(), tm.getPrevEndRow());
     assertEquals(extent.tableId(), tm.getTableId());
     assertTrue(tm.sawPrevEndRow());
diff --git 
a/core/src/test/java/org/apache/accumulo/core/tabletserver/log/LogEntryTest.java
 
b/core/src/test/java/org/apache/accumulo/core/tabletserver/log/LogEntryTest.java
index 3adec0aa0c..5273baae45 100644
--- 
a/core/src/test/java/org/apache/accumulo/core/tabletserver/log/LogEntryTest.java
+++ 
b/core/src/test/java/org/apache/accumulo/core/tabletserver/log/LogEntryTest.java
@@ -24,57 +24,59 @@ import static 
org.junit.jupiter.api.Assertions.assertNotSame;
 import static org.junit.jupiter.api.Assertions.assertThrows;
 import static org.junit.jupiter.api.Assertions.assertTrue;
 
-import java.nio.file.Path;
-import java.util.AbstractMap;
-import java.util.List;
+import java.util.AbstractMap.SimpleImmutableEntry;
 import java.util.UUID;
 import java.util.stream.Stream;
 
 import org.apache.accumulo.core.data.Key;
-import org.apache.accumulo.core.data.Value;
-import org.apache.accumulo.core.metadata.schema.MetadataSchema;
+import 
org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.LogColumnFamily;
 import org.apache.hadoop.io.Text;
-import org.junit.jupiter.api.Nested;
 import org.junit.jupiter.api.Test;
 
 import com.google.common.net.HostAndPort;
 
 public class LogEntryTest {
 
-  private final HostAndPort validHost = HostAndPort.fromParts("default", 8080);
+  private final String validHost = "localhost+9997";
   private final UUID validUUID = UUID.randomUUID();
-  private final String validFilename = validHost + "/" + validUUID;
+  private final String validPath = "viewfs:/a/accumulo/wal/" + validHost + "/" 
+ validUUID;
 
   @Test
-  public void test() throws Exception {
-    assertEquals(new Text("log"), 
MetadataSchema.TabletsSection.LogColumnFamily.NAME);
-
-    // test from constructor
-    LogEntry one = new LogEntry(validFilename);
-    assertEquals(validFilename, one.toString());
-    assertEquals(validFilename, one.getFilePath());
-    assertEquals(new Text("-/" + validFilename), one.getColumnQualifier());
-    assertEquals(validUUID.toString(), one.getUniqueID());
-
-    // test from metadata entry
-    LogEntry two = LogEntry.fromMetaWalEntry(new 
AbstractMap.SimpleImmutableEntry<>(
-        new Key(new Text("1<"), new Text("log"), one.getColumnQualifier()), 
new Value("unused")));
-    assertNotSame(one, two);
-    assertEquals(one.toString(), two.toString());
-    assertEquals(one.getFilePath(), two.getFilePath());
-    assertEquals(one.getColumnQualifier(), two.getColumnQualifier());
-    assertEquals(one.getUniqueID(), two.getUniqueID());
-    assertEquals(one, two);
+  public void testColumnFamily() {
+    assertEquals(new Text("log"), LogColumnFamily.NAME);
+  }
+
+  @Test
+  public void testFromPath() {
+    var logEntry = LogEntry.fromPath(validPath);
+    verifyLogEntry(logEntry);
+  }
+
+  @Test
+  public void testFromMetadata() {
+    var logEntry = LogEntry.fromMetaWalEntry(new SimpleImmutableEntry<>(
+        new Key("1<", LogColumnFamily.STR_NAME, "-/" + validPath), null));
+    verifyLogEntry(logEntry);
+  }
+
+  // helper for testing build from constructor or from metadata
+  private void verifyLogEntry(LogEntry logEntry) {
+    assertEquals(validPath, logEntry.toString());
+    assertEquals(validPath, logEntry.getPath());
+    assertEquals(HostAndPort.fromString(validHost.replace('+', ':')), 
logEntry.getTServer());
+    assertEquals(new Text("-/" + validPath), logEntry.getColumnQualifier());
+    assertEquals(validUUID, logEntry.getUniqueID());
   }
 
   @Test
   public void testEquals() {
-    LogEntry one = new LogEntry(validFilename);
-    LogEntry two = new LogEntry(validFilename);
+    LogEntry one = LogEntry.fromPath(validPath);
+    LogEntry two = LogEntry.fromPath(validPath);
 
     assertNotSame(one, two);
     assertEquals(one.toString(), two.toString());
-    assertEquals(one.getFilePath(), two.getFilePath());
+    assertEquals(one.getPath(), two.getPath());
+    assertEquals(one.getTServer(), two.getTServer());
     assertEquals(one.getColumnQualifier(), two.getColumnQualifier());
     assertEquals(one.getUniqueID(), two.getUniqueID());
     assertEquals(one, two);
@@ -83,50 +85,39 @@ public class LogEntryTest {
     assertEquals(two, two);
   }
 
-  @Nested
-  class ValidateFilePath {
-
-    @Test
-    public void testValidPaths() {
-      Path validPath = Path.of(validHost.toString(), validUUID.toString());
-      Path validPath2 = Path.of("dir1", validPath.toString());
-      Path validPath3 = Path.of("dir2", validPath2.toString());
-
-      Stream.of(validPath, validPath2, validPath3).map(Path::toString)
-          .forEach(validFilePath -> assertDoesNotThrow(() -> new 
LogEntry(validFilePath)));
-    }
-
-    @Test
-    public void testBadPathLength() {
-      List<String> badFilePaths = List.of("foo", "", validHost.toString());
-
-      for (String badFilePath : badFilePaths) {
-        IllegalArgumentException iae =
-            assertThrows(IllegalArgumentException.class, () -> new 
LogEntry(badFilePath));
-        assertTrue(iae.getMessage().contains("The path should at least contain 
tserver/UUID."));
-      }
-    }
-
-    @Test
-    public void testInvalidHostPort() {
-      final String badHostAndPort = "default:badPort";
-      final Path badFilepathHostPort = Path.of(badHostAndPort, 
validUUID.toString());
-
-      IllegalArgumentException iae = 
assertThrows(IllegalArgumentException.class,
-          () -> new LogEntry(badFilepathHostPort.toString()));
-      assertTrue(
-          iae.getMessage().contains("Expected format: host:port. Found '" + 
badHostAndPort + "'"));
-    }
-
-    @Test
-    public void testInvalidUUID() {
-      final String badUUID = "badUUID";
-      String filePathWithBadUUID = Path.of(validHost.toString(), 
badUUID).toString();
-
-      IllegalArgumentException iae =
-          assertThrows(IllegalArgumentException.class, () -> new 
LogEntry(filePathWithBadUUID));
-      assertTrue(iae.getMessage().contains("Expected valid UUID. Found '" + 
badUUID + "'"));
-    }
+  @Test
+  public void testValidPaths() {
+    var validPath1 = validHost + "/" + validUUID;
+    var validPath2 = "dir1/" + validPath1;
+    var validPath3 = "dir2/" + validPath2;
+
+    Stream.of(validPath1, validPath2, validPath3)
+        .forEach(s -> assertDoesNotThrow(() -> LogEntry.fromPath(s)));
+  }
+
+  @Test
+  public void testBadPathLength() {
+    Stream.of("foo", "", validHost).forEach(badPath -> {
+      var e = assertThrows(IllegalArgumentException.class, () -> 
LogEntry.fromPath(badPath));
+      assertTrue(e.getMessage().contains("The path should end with 
tserver/UUID."));
+    });
+  }
+
+  @Test
+  public void testInvalidHostPort() {
+    Stream.of("default:9997", "default+badPort").forEach(badHostAndPort -> {
+      var badPath = badHostAndPort + "/" + validUUID;
+      var e = assertThrows(IllegalArgumentException.class, () -> 
LogEntry.fromPath(badPath));
+      assertTrue(e.getMessage().contains("Expected: host+port. Found '" + 
badHostAndPort + "'"));
+    });
+  }
+
+  @Test
+  public void testInvalidUUID() {
+    var badUUID = "badUUID";
+    var pathWithBadUUID = validHost + "/" + badUUID;
+    var e = assertThrows(IllegalArgumentException.class, () -> 
LogEntry.fromPath(pathWithBadUUID));
+    assertTrue(e.getMessage().contains("Expected valid UUID. Found '" + 
badUUID + "'"));
   }
 
 }
diff --git 
a/server/base/src/main/java/org/apache/accumulo/server/fs/VolumeUtil.java 
b/server/base/src/main/java/org/apache/accumulo/server/fs/VolumeUtil.java
index 4021840869..19841ac7a3 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/fs/VolumeUtil.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/fs/VolumeUtil.java
@@ -88,25 +88,14 @@ public class VolumeUtil {
   }
 
   protected static LogEntry switchVolumes(LogEntry le, List<Pair<Path,Path>> 
replacements) {
-    Path switchedPath = switchVolume(new Path(le.getFilePath()), FileType.WAL, 
replacements);
-    String switchedString;
-    int numSwitched = 0;
-    if (switchedPath != null) {
-      switchedString = switchedPath.toString();
-      numSwitched++;
-    } else {
-      switchedString = le.getFilePath();
-    }
-
-    if (numSwitched == 0) {
+    Path switchedPath = switchVolume(new Path(le.getPath()), FileType.WAL, 
replacements);
+    if (switchedPath == null) {
       log.trace("Did not switch {}", le);
       return null;
     }
 
-    LogEntry newLogEntry = new LogEntry(switchedString);
-
+    LogEntry newLogEntry = LogEntry.fromPath(switchedPath.toString());
     log.trace("Switched {} to {}", le, newLogEntry);
-
     return newLogEntry;
   }
 
@@ -155,8 +144,8 @@ public class VolumeUtil {
         logsToRemove.add(logEntry);
         logsToAdd.add(switchedLogEntry);
         ret.logEntries.add(switchedLogEntry);
-        log.debug("Replacing volume {} : {} -> {}", extent, 
logEntry.getFilePath(),
-            switchedLogEntry.getFilePath());
+        log.debug("Replacing volume {} : {} -> {}", extent, logEntry.getPath(),
+            switchedLogEntry.getPath());
       } else {
         ret.logEntries.add(logEntry);
       }
diff --git 
a/server/base/src/main/java/org/apache/accumulo/server/manager/state/MetaDataStateStore.java
 
b/server/base/src/main/java/org/apache/accumulo/server/manager/state/MetaDataStateStore.java
index 7352eff601..8c921ab110 100644
--- 
a/server/base/src/main/java/org/apache/accumulo/server/manager/state/MetaDataStateStore.java
+++ 
b/server/base/src/main/java/org/apache/accumulo/server/manager/state/MetaDataStateStore.java
@@ -123,7 +123,7 @@ class MetaDataStateStore implements TabletStateStore {
             List<Path> logs = 
logsForDeadServers.get(tls.current.getServerInstance());
             if (logs != null) {
               for (Path log : logs) {
-                LogEntry entry = new LogEntry(log.toString());
+                LogEntry entry = LogEntry.fromPath(log.toString());
                 tabletMutator.putWal(entry);
               }
             }
diff --git 
a/server/base/src/main/java/org/apache/accumulo/server/manager/state/MetaDataTableScanner.java
 
b/server/base/src/main/java/org/apache/accumulo/server/manager/state/MetaDataTableScanner.java
index 900387fa06..88a7af6fe8 100644
--- 
a/server/base/src/main/java/org/apache/accumulo/server/manager/state/MetaDataTableScanner.java
+++ 
b/server/base/src/main/java/org/apache/accumulo/server/manager/state/MetaDataTableScanner.java
@@ -176,7 +176,7 @@ public class MetaDataTableScanner implements 
ClosableIterator<TabletLocationStat
         }
         current = location;
       } else if (cf.compareTo(LogColumnFamily.NAME) == 0) {
-        
walogs.add(Collections.singleton(LogEntry.fromMetaWalEntry(entry).getFilePath()));
+        
walogs.add(Collections.singleton(LogEntry.fromMetaWalEntry(entry).getPath()));
       } else if (cf.compareTo(LastLocationColumnFamily.NAME) == 0) {
         if (lastTimestamp < entry.getKey().getTimestamp()) {
           last = Location.last(new TServerInstance(entry.getValue(), cq));
diff --git 
a/server/base/src/main/java/org/apache/accumulo/server/manager/state/ZooTabletStateStore.java
 
b/server/base/src/main/java/org/apache/accumulo/server/manager/state/ZooTabletStateStore.java
index c6bc4bef39..44ddbc4cb4 100644
--- 
a/server/base/src/main/java/org/apache/accumulo/server/manager/state/ZooTabletStateStore.java
+++ 
b/server/base/src/main/java/org/apache/accumulo/server/manager/state/ZooTabletStateStore.java
@@ -98,8 +98,8 @@ class ZooTabletStateStore implements TabletStateStore {
 
           List<Collection<String>> logs = new ArrayList<>();
           rootMeta.getLogs().forEach(logEntry -> {
-            logs.add(Collections.singleton(logEntry.getFilePath()));
-            log.debug("root tablet log {}", logEntry.getFilePath());
+            logs.add(Collections.singleton(logEntry.getPath()));
+            log.debug("root tablet log {}", logEntry.getPath());
           });
 
           return new TabletLocationState(RootTable.EXTENT, futureSession, 
currentSession,
@@ -176,7 +176,7 @@ class ZooTabletStateStore implements TabletStateStore {
       List<Path> logs = logsForDeadServers.get(futureOrCurrent);
       if (logs != null) {
         for (Path entry : logs) {
-          LogEntry logEntry = new LogEntry(entry.toString());
+          LogEntry logEntry = LogEntry.fromPath(entry.toString());
           tabletMutator.putWal(logEntry);
         }
       }
diff --git 
a/server/base/src/main/java/org/apache/accumulo/server/metadata/TabletMutatorBase.java
 
b/server/base/src/main/java/org/apache/accumulo/server/metadata/TabletMutatorBase.java
index 4f7f9c230c..036c9ac342 100644
--- 
a/server/base/src/main/java/org/apache/accumulo/server/metadata/TabletMutatorBase.java
+++ 
b/server/base/src/main/java/org/apache/accumulo/server/metadata/TabletMutatorBase.java
@@ -38,7 +38,6 @@ import 
org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.Da
 import 
org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.ExternalCompactionColumnFamily;
 import 
org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.FutureLocationColumnFamily;
 import 
org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.LastLocationColumnFamily;
-import 
org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.LogColumnFamily;
 import 
org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.ScanFileColumnFamily;
 import 
org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.ServerColumnFamily;
 import 
org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.SuspendLocationColumn;
@@ -182,14 +181,7 @@ public abstract class TabletMutatorBase implements 
Ample.TabletMutator {
   @Override
   public Ample.TabletMutator deleteWal(LogEntry logEntry) {
     Preconditions.checkState(updatesEnabled, "Cannot make updates after 
calling mutate.");
-    mutation.putDelete(LogColumnFamily.NAME, logEntry.getColumnQualifier());
-    return this;
-  }
-
-  @Override
-  public Ample.TabletMutator deleteWal(String wal) {
-    Preconditions.checkState(updatesEnabled, "Cannot make updates after 
calling mutate.");
-    mutation.putDelete(LogColumnFamily.STR_NAME, wal);
+    logEntry.deleteFromMutation(mutation);
     return this;
   }
 
diff --git 
a/server/base/src/main/java/org/apache/accumulo/server/util/ListVolumesUsed.java
 
b/server/base/src/main/java/org/apache/accumulo/server/util/ListVolumesUsed.java
index 2055d793d4..30b3d1d0ed 100644
--- 
a/server/base/src/main/java/org/apache/accumulo/server/util/ListVolumesUsed.java
+++ 
b/server/base/src/main/java/org/apache/accumulo/server/util/ListVolumesUsed.java
@@ -55,7 +55,7 @@ public class ListVolumesUsed {
   }
 
   private static void getLogURIs(TreeSet<String> volumes, LogEntry logEntry) {
-    volumes.add(getLogURI(logEntry.getFilePath()));
+    volumes.add(getLogURI(logEntry.getPath()));
   }
 
   private static void listTable(Ample.DataLevel level, ServerContext context) 
throws Exception {
diff --git 
a/server/base/src/main/java/org/apache/accumulo/server/util/ManagerMetadataUtil.java
 
b/server/base/src/main/java/org/apache/accumulo/server/util/ManagerMetadataUtil.java
index 7e6f78c163..da974e4198 100644
--- 
a/server/base/src/main/java/org/apache/accumulo/server/util/ManagerMetadataUtil.java
+++ 
b/server/base/src/main/java/org/apache/accumulo/server/util/ManagerMetadataUtil.java
@@ -55,6 +55,7 @@ import org.apache.accumulo.core.metadata.schema.MetadataTime;
 import org.apache.accumulo.core.metadata.schema.TabletMetadata;
 import org.apache.accumulo.core.metadata.schema.TabletMetadata.Location;
 import org.apache.accumulo.core.security.Authorizations;
+import org.apache.accumulo.core.tabletserver.log.LogEntry;
 import org.apache.accumulo.server.ServerContext;
 import org.apache.hadoop.io.Text;
 import org.slf4j.Logger;
@@ -219,7 +220,7 @@ public class ManagerMetadataUtil {
    */
   public static Optional<StoredTabletFile> updateTabletDataFile(ServerContext 
context,
       KeyExtent extent, ReferencedTabletFile newDatafile, DataFileValue dfv, 
MetadataTime time,
-      TServerInstance tServerInstance, ServiceLock zooLock, Set<String> 
unusedWalLogs,
+      TServerInstance tServerInstance, ServiceLock zooLock, Set<LogEntry> 
unusedWalLogs,
       Location lastLocation, long flushId) {
 
     TabletMutator tablet = context.getAmple().mutateTablet(extent);
diff --git 
a/server/base/src/test/java/org/apache/accumulo/server/fs/VolumeUtilTest.java 
b/server/base/src/test/java/org/apache/accumulo/server/fs/VolumeUtilTest.java
index b6ac99845d..ce112129de 100644
--- 
a/server/base/src/test/java/org/apache/accumulo/server/fs/VolumeUtilTest.java
+++ 
b/server/base/src/test/java/org/apache/accumulo/server/fs/VolumeUtilTest.java
@@ -179,18 +179,18 @@ public class VolumeUtilTest {
 
     String walUUID = UUID.randomUUID().toString();
     String fileName = "hdfs://nn1/accumulo/wal/localhost+9997/" + walUUID;
-    LogEntry le = new LogEntry(fileName);
+    LogEntry le = LogEntry.fromPath(fileName);
     LogEntry fixedVolume = VolumeUtil.switchVolumes(le, replacements);
-    assertEquals("viewfs:/a/accumulo/wal/localhost+9997/" + walUUID, 
fixedVolume.getFilePath());
+    assertEquals("viewfs:/a/accumulo/wal/localhost+9997/" + walUUID, 
fixedVolume.getPath());
 
     fileName = "hdfs://nn1:9000/accumulo/wal/localhost+9997/" + walUUID;
-    le = new LogEntry(fileName);
+    le = LogEntry.fromPath(fileName);
     fixedVolume = VolumeUtil.switchVolumes(le, replacements);
-    assertEquals("viewfs:/a/accumulo/wal/localhost+9997/" + walUUID, 
fixedVolume.getFilePath());
+    assertEquals("viewfs:/a/accumulo/wal/localhost+9997/" + walUUID, 
fixedVolume.getPath());
 
     fileName = "hdfs://nn2/accumulo/wal/localhost+9997/" + walUUID;
-    le = new LogEntry(fileName);
+    le = LogEntry.fromPath(fileName);
     fixedVolume = VolumeUtil.switchVolumes(le, replacements);
-    assertEquals("viewfs:/b/accumulo/wal/localhost+9997/" + walUUID, 
fixedVolume.getFilePath());
+    assertEquals("viewfs:/b/accumulo/wal/localhost+9997/" + walUUID, 
fixedVolume.getPath());
   }
 }
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 4256064e8e..eb283b6b46 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
@@ -91,6 +91,7 @@ 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.tabletserver.log.LogEntry;
 import org.apache.accumulo.core.tabletserver.thrift.ActiveCompaction;
 import org.apache.accumulo.core.tabletserver.thrift.NoSuchScanIDException;
 import org.apache.accumulo.core.tabletserver.thrift.NotServingTabletException;
@@ -1281,12 +1282,9 @@ public class TabletClientHandler implements 
TabletServerClientService.Iface,
 
   @Override
   public List<String> getActiveLogs(TInfo tinfo, TCredentials credentials) {
-    String log = server.logger.getLogFile();
     // Might be null if there is no active logger
-    if (log == null) {
-      return Collections.emptyList();
-    }
-    return Collections.singletonList(log);
+    LogEntry le = server.logger.getLogEntry();
+    return le == null ? Collections.emptyList() : 
Collections.singletonList(le.getPath());
   }
 
   @Override
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 5df7c75169..4ab7587eb8 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
@@ -1103,7 +1103,7 @@ public class TabletServer extends AbstractServer 
implements TabletHostingServer
     List<Path> recoveryDirs = new ArrayList<>();
     for (LogEntry entry : logEntries) {
       Path recovery = null;
-      Path finished = RecoveryPath.getRecoveryPath(new 
Path(entry.getFilePath()));
+      Path finished = RecoveryPath.getRecoveryPath(new Path(entry.getPath()));
       finished = SortedLogState.getFinishedMarkerPath(finished);
       TabletServer.log.debug("Looking for " + finished);
       if (fs.exists(finished)) {
@@ -1131,21 +1131,6 @@ public class TabletServer extends AbstractServer 
implements TabletHostingServer
     return getContext().getTableConfiguration(extent.tableId());
   }
 
-  public DfsLogger.ServerResources getServerConfig() {
-    return new DfsLogger.ServerResources() {
-
-      @Override
-      public VolumeManager getVolumeManager() {
-        return TabletServer.this.getVolumeManager();
-      }
-
-      @Override
-      public AccumuloConfiguration getConfiguration() {
-        return TabletServer.this.getConfiguration();
-      }
-    };
-  }
-
   public SortedMap<KeyExtent,Tablet> getOnlineTablets() {
     return onlineTablets.snapshot();
   }
diff --git 
a/server/tserver/src/main/java/org/apache/accumulo/tserver/log/DfsLogger.java 
b/server/tserver/src/main/java/org/apache/accumulo/tserver/log/DfsLogger.java
index 75420aba6f..6c9ff3cc0a 100644
--- 
a/server/tserver/src/main/java/org/apache/accumulo/tserver/log/DfsLogger.java
+++ 
b/server/tserver/src/main/java/org/apache/accumulo/tserver/log/DfsLogger.java
@@ -57,6 +57,7 @@ import org.apache.accumulo.core.spi.crypto.CryptoService;
 import org.apache.accumulo.core.spi.crypto.FileDecrypter;
 import org.apache.accumulo.core.spi.crypto.FileEncrypter;
 import org.apache.accumulo.core.spi.crypto.NoCryptoService;
+import org.apache.accumulo.core.tabletserver.log.LogEntry;
 import org.apache.accumulo.core.util.Pair;
 import org.apache.accumulo.core.util.threads.Threads;
 import org.apache.accumulo.server.ServerContext;
@@ -81,7 +82,7 @@ import com.google.common.base.Preconditions;
  * Wrap a connection to a logger.
  *
  */
-public class DfsLogger implements Comparable<DfsLogger> {
+public final class DfsLogger implements Comparable<DfsLogger> {
   // older version supported for upgrade
   public static final String LOG_FILE_HEADER_V3 = "--- Log File Header (v3) 
---";
 
@@ -116,12 +117,6 @@ public class DfsLogger implements Comparable<DfsLogger> {
     }
   }
 
-  public interface ServerResources {
-    AccumuloConfiguration getConfiguration();
-
-    VolumeManager getVolumeManager();
-  }
-
   private final LinkedBlockingQueue<DfsLogger.LogWork> workQueue = new 
LinkedBlockingQueue<>();
 
   private final Object closeLock = new Object();
@@ -281,46 +276,33 @@ public class DfsLogger implements Comparable<DfsLogger> {
 
   @Override
   public boolean equals(Object obj) {
-    // filename is unique
     if (obj == null) {
       return false;
     }
     if (obj instanceof DfsLogger) {
-      return getFileName().equals(((DfsLogger) obj).getFileName());
+      return logEntry.equals(((DfsLogger) obj).logEntry);
     }
     return false;
   }
 
   @Override
   public int hashCode() {
-    // filename is unique
-    return getFileName().hashCode();
+    return logEntry.hashCode();
   }
 
   private final ServerContext context;
-  private final ServerResources conf;
   private FSDataOutputStream logFile;
   private DataOutputStream encryptingLogFile = null;
-  private String logPath;
+  private LogEntry logEntry;
   private Thread syncThread;
 
-  /* Track what's actually in +r/!0 for this logger ref */
-  private String metaReference;
   private AtomicLong syncCounter;
   private AtomicLong flushCounter;
   private final long slowFlushMillis;
   private long writes = 0;
 
-  private DfsLogger(ServerContext context, ServerResources conf) {
-    this.context = context;
-    this.conf = conf;
-    this.slowFlushMillis =
-        
conf.getConfiguration().getTimeInMillis(Property.TSERV_SLOW_FLUSH_MILLIS);
-  }
-
-  public DfsLogger(ServerContext context, ServerResources conf, AtomicLong 
syncCounter,
-      AtomicLong flushCounter) {
-    this(context, conf);
+  public DfsLogger(ServerContext context, AtomicLong syncCounter, AtomicLong 
flushCounter) {
+    this(context, null);
     this.syncCounter = syncCounter;
     this.flushCounter = flushCounter;
   }
@@ -328,12 +310,13 @@ public class DfsLogger implements Comparable<DfsLogger> {
   /**
    * Reference a pre-existing log file.
    *
-   * @param meta the cq for the "log" entry in +r/!0
+   * @param logEntry the "log" entry in +r/!0
    */
-  public DfsLogger(ServerContext context, ServerResources conf, String 
filename, String meta) {
-    this(context, conf);
-    this.logPath = filename;
-    metaReference = meta;
+  public DfsLogger(ServerContext context, LogEntry logEntry) {
+    this.context = context;
+    this.slowFlushMillis =
+        
context.getConfiguration().getTimeInMillis(Property.TSERV_SLOW_FLUSH_MILLIS);
+    this.logEntry = logEntry;
   }
 
   /**
@@ -401,23 +384,24 @@ public class DfsLogger implements Comparable<DfsLogger> {
     String logger = Joiner.on("+").join(address.split(":"));
 
     log.debug("DfsLogger.open() begin");
-    VolumeManager fs = conf.getVolumeManager();
+    VolumeManager fs = context.getVolumeManager();
 
     var chooserEnv = new VolumeChooserEnvironmentImpl(
         org.apache.accumulo.core.spi.fs.VolumeChooserEnvironment.Scope.LOGGER, 
context);
-    logPath = fs.choose(chooserEnv, context.getBaseUris()) + Path.SEPARATOR + 
Constants.WAL_DIR
-        + Path.SEPARATOR + logger + Path.SEPARATOR + filename;
+    String logPath = fs.choose(chooserEnv, context.getBaseUris()) + 
Path.SEPARATOR
+        + Constants.WAL_DIR + Path.SEPARATOR + logger + Path.SEPARATOR + 
filename;
+    this.logEntry = LogEntry.fromPath(logPath);
 
-    metaReference = toString();
     LoggerOperation op = null;
+    var serverConf = context.getConfiguration();
     try {
       Path logfilePath = new Path(logPath);
-      short replication = (short) 
conf.getConfiguration().getCount(Property.TSERV_WAL_REPLICATION);
+      short replication = (short) 
serverConf.getCount(Property.TSERV_WAL_REPLICATION);
       if (replication == 0) {
         replication = fs.getDefaultReplication(logfilePath);
       }
-      long blockSize = getWalBlockSize(conf.getConfiguration());
-      if (conf.getConfiguration().getBoolean(Property.TSERV_WAL_SYNC)) {
+      long blockSize = getWalBlockSize(serverConf);
+      if (serverConf.getBoolean(Property.TSERV_WAL_SYNC)) {
         logFile = fs.createSyncable(logfilePath, 0, replication, blockSize);
       } else {
         logFile = fs.create(logfilePath, true, 0, replication, blockSize);
@@ -439,8 +423,8 @@ public class DfsLogger implements Comparable<DfsLogger> {
 
       // Initialize the log file with a header and its encryption
       CryptoEnvironment env = new CryptoEnvironmentImpl(Scope.WAL);
-      CryptoService cryptoService = context.getCryptoFactory().getService(env,
-          conf.getConfiguration().getAllCryptoProperties());
+      CryptoService cryptoService =
+          context.getCryptoFactory().getService(env, 
serverConf.getAllCryptoProperties());
       logFile.write(LOG_FILE_HEADER_V4.getBytes(UTF_8));
 
       log.debug("Using {} for encrypting WAL {}", 
cryptoService.getClass().getSimpleName(),
@@ -491,29 +475,15 @@ public class DfsLogger implements Comparable<DfsLogger> {
 
   @Override
   public String toString() {
-    String fileName = getFileName();
-    if (fileName.contains(":")) {
-      return getLogger() + "/" + getFileName();
-    }
-    return fileName;
-  }
-
-  /**
-   * get the cq needed to reference this logger's entry in +r/!0
-   */
-  public String getMeta() {
-    if (metaReference == null) {
-      throw new IllegalStateException("logger doesn't have meta reference. " + 
this);
-    }
-    return metaReference;
+    return logEntry.toString();
   }
 
-  public String getFileName() {
-    return logPath;
+  public LogEntry getLogEntry() {
+    return logEntry;
   }
 
   public Path getPath() {
-    return new Path(logPath);
+    return new Path(logEntry.getPath());
   }
 
   public void close() throws IOException {
@@ -670,14 +640,9 @@ public class DfsLogger implements Comparable<DfsLogger> {
     return logKeyData(key, durability);
   }
 
-  private String getLogger() {
-    String[] parts = logPath.split("/");
-    return Joiner.on(":").join(parts[parts.length - 2].split("[+]"));
-  }
-
   @Override
   public int compareTo(DfsLogger o) {
-    return getFileName().compareTo(o.getFileName());
+    return logEntry.getPath().compareTo(o.logEntry.getPath());
   }
 
   /*
diff --git 
a/server/tserver/src/main/java/org/apache/accumulo/tserver/log/TabletServerLogger.java
 
b/server/tserver/src/main/java/org/apache/accumulo/tserver/log/TabletServerLogger.java
index fbb6db7667..3103d01044 100644
--- 
a/server/tserver/src/main/java/org/apache/accumulo/tserver/log/TabletServerLogger.java
+++ 
b/server/tserver/src/main/java/org/apache/accumulo/tserver/log/TabletServerLogger.java
@@ -38,6 +38,7 @@ import java.util.concurrent.locks.ReentrantReadWriteLock;
 import org.apache.accumulo.core.client.Durability;
 import org.apache.accumulo.core.data.Mutation;
 import org.apache.accumulo.core.dataImpl.KeyExtent;
+import org.apache.accumulo.core.tabletserver.log.LogEntry;
 import org.apache.accumulo.core.util.Halt;
 import org.apache.accumulo.core.util.Retry;
 import org.apache.accumulo.core.util.Retry.RetryFactory;
@@ -47,9 +48,9 @@ import org.apache.accumulo.server.fs.VolumeManager;
 import org.apache.accumulo.tserver.TabletMutations;
 import org.apache.accumulo.tserver.TabletServer;
 import org.apache.accumulo.tserver.log.DfsLogger.LoggerOperation;
-import org.apache.accumulo.tserver.log.DfsLogger.ServerResources;
 import org.apache.accumulo.tserver.tablet.CommitSession;
 import org.apache.hadoop.fs.Path;
+import org.checkerframework.checker.nullness.qual.Nullable;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -181,17 +182,15 @@ public class TabletServerLogger {
   }
 
   /**
-   * Get the current WAL file
+   * Get the current log entry
    *
-   * @return The name of the current log, or null if there is no current log.
+   * @return the current log entry, or null if there is no current log
    */
-  public String getLogFile() {
+  @Nullable
+  public LogEntry getLogEntry() {
     logIdLock.readLock().lock();
     try {
-      if (currentLog == null) {
-        return null;
-      }
-      return currentLog.getFileName();
+      return currentLog == null ? null : currentLog.getLogEntry();
     } finally {
       logIdLock.readLock().unlock();
     }
@@ -215,7 +214,7 @@ public class TabletServerLogger {
       if (next instanceof DfsLogger) {
         currentLog = (DfsLogger) next;
         logId.incrementAndGet();
-        log.info("Using next log {}", currentLog.getFileName());
+        log.info("Using next log {}", currentLog.getLogEntry());
 
         // When we successfully create a WAL, make sure to reset the Retry.
         if (createRetry != null) {
@@ -261,14 +260,13 @@ public class TabletServerLogger {
     nextLogMaker =
         ThreadPools.getServerThreadPools().createFixedThreadPool(1, "WALog 
creator", true);
     nextLogMaker.execute(() -> {
-      final ServerResources conf = tserver.getServerConfig();
-      final VolumeManager fs = conf.getVolumeManager();
+      final VolumeManager fs = tserver.getVolumeManager();
       while (!nextLogMaker.isShutdown()) {
         log.debug("Creating next WAL");
         DfsLogger alog = null;
 
         try {
-          alog = new DfsLogger(tserver.getContext(), conf, syncCounter, 
flushCounter);
+          alog = new DfsLogger(tserver.getContext(), syncCounter, 
flushCounter);
           alog.open(tserver.getClientAddressString());
         } catch (Exception t) {
           log.error("Failed to open WAL", t);
@@ -299,13 +297,12 @@ public class TabletServerLogger {
           continue;
         }
 
-        String fileName = alog.getFileName();
-        log.debug("Created next WAL {}", fileName);
+        log.debug("Created next WAL {}", alog.getLogEntry());
 
         try {
           tserver.addNewLogMarker(alog);
         } catch (Exception t) {
-          log.error("Failed to add new WAL marker for " + fileName, t);
+          log.error("Failed to add new WAL marker for " + alog.getLogEntry(), 
t);
 
           try {
             // Intentionally not deleting walog because it may have been 
advertised in ZK. See
@@ -323,7 +320,7 @@ public class TabletServerLogger {
           try {
             tserver.walogClosed(alog);
           } catch (Exception e) {
-            log.error("Failed to close WAL that failed to open: " + fileName, 
e);
+            log.error("Failed to close WAL that failed to open: " + 
alog.getLogEntry(), e);
           }
 
           try {
@@ -337,7 +334,7 @@ public class TabletServerLogger {
 
         try {
           while (!nextLog.offer(alog, 12, TimeUnit.HOURS)) {
-            log.info("Our WAL was not used for 12 hours: {}", fileName);
+            log.info("Our WAL was not used for 12 hours: {}", 
alog.getLogEntry());
           }
         } catch (InterruptedException e) {
           // ignore - server is shutting down
@@ -357,7 +354,7 @@ public class TabletServerLogger {
         } catch (DfsLogger.LogClosedException ex) {
           // ignore
         } catch (Exception ex) {
-          log.error("Unable to cleanly close log " + currentLog.getFileName() 
+ ": " + ex, ex);
+          log.error("Unable to cleanly close log " + currentLog.getLogEntry() 
+ ": " + ex, ex);
         } finally {
           this.tserver.walogClosed(currentLog);
           currentLog = null;
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 b1bb64c100..1c534de4e4 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
@@ -43,6 +43,7 @@ import org.apache.accumulo.core.metadata.StoredTabletFile;
 import org.apache.accumulo.core.metadata.schema.DataFileValue;
 import org.apache.accumulo.core.metadata.schema.ExternalCompactionId;
 import org.apache.accumulo.core.metadata.schema.TabletMetadata.Location;
+import org.apache.accumulo.core.tabletserver.log.LogEntry;
 import org.apache.accumulo.core.trace.TraceUtil;
 import org.apache.accumulo.core.util.MapCounter;
 import org.apache.accumulo.core.util.Pair;
@@ -355,7 +356,7 @@ class DatafileManager {
         // The following call pairs with tablet.finishClearingUnusedLogs() 
later in this block. If
         // moving where the following method is called, examine it and 
finishClearingUnusedLogs()
         // before moving.
-        Set<String> unusedWalLogs = tablet.beginClearingUnusedLogs();
+        Set<LogEntry> unusedWalLogs = tablet.beginClearingUnusedLogs();
 
         // the order of writing to metadata and walog is important in the face 
of machine/process
         // failures need to write to metadata before writing to walog, when 
things are done in the
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 f530058876..9d6043ef3e 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
@@ -369,8 +369,7 @@ public class Tablet extends TabletBase {
       // make some closed references that represent the recovered logs
       currentLogs = new HashSet<>();
       for (LogEntry logEntry : logEntries) {
-        currentLogs.add(new DfsLogger(tabletServer.getContext(), 
tabletServer.getServerConfig(),
-            logEntry.getFilePath(), logEntry.getColumnQualifier().toString()));
+        currentLogs.add(new DfsLogger(tabletServer.getContext(), logEntry));
       }
 
       rebuildReferencedLogs();
@@ -1870,7 +1869,7 @@ public class Tablet extends TabletBase {
         List<DfsLogger> oldClosed = closedLogs.subList(0, closedLogs.size() - 
maxLogs);
         for (DfsLogger closedLog : oldClosed) {
           if (currentLogs.contains(closedLog)) {
-            reason = "referenced at least one old write ahead log " + 
closedLog.getFileName();
+            reason = "referenced at least one old write ahead log " + 
closedLog.getLogEntry();
             break;
           }
         }
@@ -1888,12 +1887,12 @@ public class Tablet extends TabletBase {
     return logLock;
   }
 
-  Set<String> beginClearingUnusedLogs() {
+  Set<LogEntry> beginClearingUnusedLogs() {
     Preconditions.checkState(logLock.isHeldByCurrentThread());
-    Set<String> unusedLogs = new HashSet<>();
+    Set<LogEntry> unusedLogs = new HashSet<>();
 
-    ArrayList<String> otherLogsCopy = new ArrayList<>();
-    ArrayList<String> currentLogsCopy = new ArrayList<>();
+    ArrayList<LogEntry> otherLogsCopy = new ArrayList<>();
+    ArrayList<LogEntry> currentLogsCopy = new ArrayList<>();
 
     synchronized (this) {
       if (removingLogs) {
@@ -1902,13 +1901,13 @@ public class Tablet extends TabletBase {
       }
 
       for (DfsLogger logger : otherLogs) {
-        otherLogsCopy.add(logger.toString());
-        unusedLogs.add(logger.getMeta());
+        otherLogsCopy.add(logger.getLogEntry());
+        unusedLogs.add(logger.getLogEntry());
       }
 
       for (DfsLogger logger : currentLogs) {
-        currentLogsCopy.add(logger.toString());
-        unusedLogs.remove(logger.getMeta());
+        currentLogsCopy.add(logger.getLogEntry());
+        unusedLogs.remove(logger.getLogEntry());
       }
 
       if (!unusedLogs.isEmpty()) {
@@ -1917,16 +1916,16 @@ public class Tablet extends TabletBase {
     }
 
     // do debug logging outside tablet lock
-    for (String logger : otherLogsCopy) {
-      log.trace("Logs for memory compacted: {} {}", getExtent(), logger);
+    for (LogEntry logEntry : otherLogsCopy) {
+      log.trace("Logs for memory compacted: {} {}", getExtent(), logEntry);
     }
 
-    for (String logger : currentLogsCopy) {
-      log.trace("Logs for current memory: {} {}", getExtent(), logger);
+    for (LogEntry logEntry : currentLogsCopy) {
+      log.trace("Logs for current memory: {} {}", getExtent(), logEntry);
     }
 
-    for (String logger : unusedLogs) {
-      log.trace("Logs to be destroyed: {} {}", getExtent(), logger);
+    for (LogEntry logEntry : unusedLogs) {
+      log.trace("Logs to be destroyed: {} {}", getExtent(), logEntry);
     }
 
     return unusedLogs;
@@ -2092,7 +2091,7 @@ public class Tablet extends TabletBase {
    * Update tablet file data from flush. Returns a StoredTabletFile if there 
are data entries.
    */
   public Optional<StoredTabletFile> updateTabletDataFile(long maxCommittedTime,
-      ReferencedTabletFile newDatafile, DataFileValue dfv, Set<String> 
unusedWalLogs,
+      ReferencedTabletFile newDatafile, DataFileValue dfv, Set<LogEntry> 
unusedWalLogs,
       long flushId) {
     synchronized (timeLock) {
       if (maxCommittedTime > persistedTime) {
diff --git 
a/server/tserver/src/test/java/org/apache/accumulo/tserver/WalRemovalOrderTest.java
 
b/server/tserver/src/test/java/org/apache/accumulo/tserver/WalRemovalOrderTest.java
index 7bb8cf2b16..69d7253a72 100644
--- 
a/server/tserver/src/test/java/org/apache/accumulo/tserver/WalRemovalOrderTest.java
+++ 
b/server/tserver/src/test/java/org/apache/accumulo/tserver/WalRemovalOrderTest.java
@@ -18,6 +18,10 @@
  */
 package org.apache.accumulo.tserver;
 
+import static org.easymock.EasyMock.createMock;
+import static org.easymock.EasyMock.expect;
+import static org.easymock.EasyMock.replay;
+import static org.easymock.EasyMock.verify;
 import static org.junit.jupiter.api.Assertions.assertEquals;
 
 import java.util.Collections;
@@ -25,33 +29,38 @@ import java.util.LinkedHashSet;
 import java.util.List;
 import java.util.Set;
 
-import org.apache.accumulo.core.conf.AccumuloConfiguration;
 import org.apache.accumulo.core.conf.DefaultConfiguration;
-import org.apache.accumulo.server.fs.VolumeManager;
+import org.apache.accumulo.core.tabletserver.log.LogEntry;
+import org.apache.accumulo.server.ServerContext;
 import org.apache.accumulo.tserver.log.DfsLogger;
-import org.apache.accumulo.tserver.log.DfsLogger.ServerResources;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
 import org.junit.jupiter.api.Test;
 
 import com.google.common.collect.Sets;
 
 public class WalRemovalOrderTest {
 
-  private static DfsLogger mockLogger(String filename) {
-    ServerResources conf = new ServerResources() {
-      @Override
-      public AccumuloConfiguration getConfiguration() {
-        return DefaultConfiguration.getInstance();
-      }
+  private ServerContext context;
 
-      @Override
-      public VolumeManager getVolumeManager() {
-        throw new UnsupportedOperationException();
-      }
-    };
-    return new DfsLogger(null, conf, filename, null);
+  @BeforeEach
+  private void createMocks() {
+    context = createMock(ServerContext.class);
+    
expect(context.getConfiguration()).andReturn(DefaultConfiguration.getInstance()).anyTimes();
+    replay(context);
+  }
+
+  @AfterEach
+  private void verifyMocks() {
+    verify(context);
+  }
+
+  private DfsLogger mockLogger(String filename) {
+    var mockLogEntry = LogEntry.fromPath(filename + 
"+1234/11111111-1111-1111-1111-111111111111");
+    return new DfsLogger(context, mockLogEntry);
   }
 
-  private static LinkedHashSet<DfsLogger> mockLoggers(String... logs) {
+  private LinkedHashSet<DfsLogger> mockLoggers(String... logs) {
     LinkedHashSet<DfsLogger> logSet = new LinkedHashSet<>();
 
     for (String log : logs) {
@@ -61,7 +70,7 @@ public class WalRemovalOrderTest {
     return logSet;
   }
 
-  private static void runTest(LinkedHashSet<DfsLogger> closedLogs, 
Set<DfsLogger> inUseLogs,
+  private void runTest(LinkedHashSet<DfsLogger> closedLogs, Set<DfsLogger> 
inUseLogs,
       Set<DfsLogger> expected) {
     Set<DfsLogger> eligible = 
TabletServer.findOldestUnreferencedWals(List.copyOf(closedLogs),
         candidates -> candidates.removeAll(inUseLogs));
diff --git 
a/test/src/main/java/org/apache/accumulo/test/MissingWalHeaderCompletesRecoveryIT.java
 
b/test/src/main/java/org/apache/accumulo/test/MissingWalHeaderCompletesRecoveryIT.java
index f6711ff223..1eb207dcca 100644
--- 
a/test/src/main/java/org/apache/accumulo/test/MissingWalHeaderCompletesRecoveryIT.java
+++ 
b/test/src/main/java/org/apache/accumulo/test/MissingWalHeaderCompletesRecoveryIT.java
@@ -137,7 +137,7 @@ public class MissingWalHeaderCompletesRecoveryIT extends 
ConfigurableMacBase {
       TableId tableId = 
TableId.of(client.tableOperations().tableIdMap().get(tableName));
       assertNotNull(tableId, "Table ID was null");
 
-      LogEntry logEntry = new LogEntry(emptyWalog.toURI().toString());
+      LogEntry logEntry = LogEntry.fromPath(emptyWalog.toURI().toString());
 
       log.info("Taking {} offline", tableName);
       client.tableOperations().offline(tableName, true);
@@ -195,7 +195,7 @@ public class MissingWalHeaderCompletesRecoveryIT extends 
ConfigurableMacBase {
       TableId tableId = 
TableId.of(client.tableOperations().tableIdMap().get(tableName));
       assertNotNull(tableId, "Table ID was null");
 
-      LogEntry logEntry = new LogEntry(partialHeaderWalog.toURI().toString());
+      LogEntry logEntry = 
LogEntry.fromPath(partialHeaderWalog.toURI().toString());
 
       log.info("Taking {} offline", tableName);
       client.tableOperations().offline(tableName, true);
diff --git 
a/test/src/main/java/org/apache/accumulo/test/manager/MergeStateIT.java 
b/test/src/main/java/org/apache/accumulo/test/manager/MergeStateIT.java
index 40c2e07b5f..981a2ff8c5 100644
--- a/test/src/main/java/org/apache/accumulo/test/manager/MergeStateIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/manager/MergeStateIT.java
@@ -42,7 +42,6 @@ import org.apache.accumulo.core.metadata.TServerInstance;
 import org.apache.accumulo.core.metadata.TabletLocationState;
 import org.apache.accumulo.core.metadata.schema.Ample.DataLevel;
 import 
org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.CurrentLocationColumnFamily;
-import 
org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.LogColumnFamily;
 import 
org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.TabletColumnFamily;
 import org.apache.accumulo.core.metadata.schema.TabletMetadata.Location;
 import org.apache.accumulo.core.security.Authorizations;
@@ -203,7 +202,7 @@ public class MergeStateIT extends ConfigurableMacBase {
       // Add a walog which should keep the state from transitioning to MERGING
       KeyExtent ke = new KeyExtent(tableId, new Text("t"), new Text("p"));
       m = new Mutation(ke.toMetaRow());
-      LogEntry logEntry = new LogEntry("localhost:1234/" + UUID.randomUUID());
+      LogEntry logEntry = LogEntry.fromPath("localhost+1234/" + 
UUID.randomUUID());
       logEntry.addToMutation(m);
       update(accumuloClient, m);
 
@@ -214,7 +213,7 @@ public class MergeStateIT extends ConfigurableMacBase {
 
       // Delete the walog which will now allow a transition to MERGING
       m = new Mutation(ke.toMetaRow());
-      m.putDelete(LogColumnFamily.NAME, logEntry.getColumnQualifier());
+      logEntry.deleteFromMutation(m);
       update(accumuloClient, m);
 
       // now we can split

Reply via email to