You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@accumulo.apache.org by cs...@apache.org on 2024/04/06 19:52:35 UTC

(accumulo) branch main updated: Update TabletMetadata to be immutable (#4437)

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

cshannon 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 1ae2b17e2b Update TabletMetadata to be immutable (#4437)
1ae2b17e2b is described below

commit 1ae2b17e2bb94c922b5a3a7482cf92fb5ae23883
Author: Christopher L. Shannon <cs...@apache.org>
AuthorDate: Sat Apr 6 15:52:31 2024 -0400

    Update TabletMetadata to be immutable (#4437)
    
    This commit refactors all of the fields to be final for TabletMetadata.
    The object is already treated as immutable in practice, but
    previously didn't declare fields final so this could lead to potential
    future bugs and also means not being able to guarantee the state of a
    field which is now possible.
---
 .../core/metadata/schema/TabletMetadata.java       | 302 +++++++++++++++------
 .../core/metadata/schema/TabletMetadataTest.java   | 101 ++++++-
 2 files changed, 313 insertions(+), 90 deletions(-)

diff --git a/core/src/main/java/org/apache/accumulo/core/metadata/schema/TabletMetadata.java b/core/src/main/java/org/apache/accumulo/core/metadata/schema/TabletMetadata.java
index b98d9e8540..f2ad719fd8 100644
--- a/core/src/main/java/org/apache/accumulo/core/metadata/schema/TabletMetadata.java
+++ b/core/src/main/java/org/apache/accumulo/core/metadata/schema/TabletMetadata.java
@@ -74,38 +74,70 @@ import org.slf4j.LoggerFactory;
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
+import com.google.common.base.Supplier;
+import com.google.common.base.Suppliers;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.ImmutableSortedMap;
+import com.google.common.collect.ImmutableSortedMap.Builder;
 import com.google.common.net.HostAndPort;
 
 public class TabletMetadata {
   private static final Logger log = LoggerFactory.getLogger(TabletMetadata.class);
 
-  private TableId tableId;
-  private Text prevEndRow;
-  private boolean sawPrevEndRow = false;
-  private Text oldPrevEndRow;
-  private boolean sawOldPrevEndRow = false;
-  private Text endRow;
-  private Location location;
-  private Map<StoredTabletFile,DataFileValue> files;
-  private List<StoredTabletFile> scans;
-  private Map<StoredTabletFile,Long> loadedFiles;
-  private EnumSet<ColumnType> fetchedCols;
-  private KeyExtent extent;
-  private Location last;
-  private SuspendingTServer suspend;
-  private String dirName;
-  private MetadataTime time;
-  private String cloned;
-  private SortedMap<Key,Value> keyValues;
-  private OptionalLong flush = OptionalLong.empty();
-  private List<LogEntry> logs;
-  private OptionalLong compact = OptionalLong.empty();
-  private Double splitRatio = null;
-  private Map<ExternalCompactionId,ExternalCompactionMetadata> extCompactions;
-  private boolean merged;
+  private final TableId tableId;
+  private final Text prevEndRow;
+  private final boolean sawPrevEndRow;
+  private final Text oldPrevEndRow;
+  private final boolean sawOldPrevEndRow;
+  private final Text endRow;
+  private final Location location;
+  private final Map<StoredTabletFile,DataFileValue> files;
+  private final List<StoredTabletFile> scans;
+  private final Map<StoredTabletFile,Long> loadedFiles;
+  private final EnumSet<ColumnType> fetchedCols;
+  private final Supplier<KeyExtent> extent;
+  private final Location last;
+  private final SuspendingTServer suspend;
+  private final String dirName;
+  private final MetadataTime time;
+  private final String cloned;
+  private final SortedMap<Key,Value> keyValues;
+  private final OptionalLong flush;
+  private final List<LogEntry> logs;
+  private final OptionalLong compact;
+  private final Double splitRatio;
+  private final Map<ExternalCompactionId,ExternalCompactionMetadata> extCompactions;
+  private final boolean merged;
+
+  private TabletMetadata(Builder tmBuilder) {
+    this.tableId = tmBuilder.tableId;
+    this.prevEndRow = tmBuilder.prevEndRow;
+    this.sawPrevEndRow = tmBuilder.sawPrevEndRow;
+    this.oldPrevEndRow = tmBuilder.oldPrevEndRow;
+    this.sawOldPrevEndRow = tmBuilder.sawOldPrevEndRow;
+    this.endRow = tmBuilder.endRow;
+    this.location = tmBuilder.location;
+    this.files = Objects.requireNonNull(tmBuilder.files.build());
+    this.scans = Objects.requireNonNull(tmBuilder.scans.build());
+    this.loadedFiles = tmBuilder.loadedFiles.build();
+    this.fetchedCols = Objects.requireNonNull(tmBuilder.fetchedCols);
+    this.last = tmBuilder.last;
+    this.suspend = tmBuilder.suspend;
+    this.dirName = tmBuilder.dirName;
+    this.time = tmBuilder.time;
+    this.cloned = tmBuilder.cloned;
+    this.keyValues = Optional.ofNullable(tmBuilder.keyValues).map(ImmutableSortedMap.Builder::build)
+        .orElse(null);
+    this.flush = tmBuilder.flush;
+    this.logs = Objects.requireNonNull(tmBuilder.logs.build());
+    this.compact = Objects.requireNonNull(tmBuilder.compact);
+    this.splitRatio = tmBuilder.splitRatio;
+    this.extCompactions = Objects.requireNonNull(tmBuilder.extCompactions.build());
+    this.merged = tmBuilder.merged;
+    this.extent =
+        Suppliers.memoize(() -> new KeyExtent(getTableId(), getEndRow(), getPrevEndRow()));
+  }
 
   public enum LocationType {
     CURRENT, FUTURE, LAST
@@ -231,10 +263,7 @@ public class TabletMetadata {
   }
 
   public KeyExtent getExtent() {
-    if (extent == null) {
-      extent = new KeyExtent(getTableId(), getEndRow(), getPrevEndRow());
-    }
-    return extent;
+    return extent.get();
   }
 
   private void ensureFetched(ColumnType col) {
@@ -390,16 +419,7 @@ public class TabletMetadata {
       EnumSet<ColumnType> fetchedColumns, boolean buildKeyValueMap) {
     Objects.requireNonNull(rowIter);
 
-    TabletMetadata te = new TabletMetadata();
-    final ImmutableSortedMap.Builder<Key,Value> kvBuilder =
-        buildKeyValueMap ? ImmutableSortedMap.naturalOrder() : null;
-
-    final var filesBuilder = ImmutableMap.<StoredTabletFile,DataFileValue>builder();
-    final var scansBuilder = ImmutableList.<StoredTabletFile>builder();
-    final var logsBuilder = ImmutableList.<LogEntry>builder();
-    final var extCompBuilder =
-        ImmutableMap.<ExternalCompactionId,ExternalCompactionMetadata>builder();
-    final var loadedFilesBuilder = ImmutableMap.<StoredTabletFile,Long>builder();
+    final var tmBuilder = new Builder();
     ByteSequence row = null;
 
     while (rowIter.hasNext()) {
@@ -410,14 +430,14 @@ public class TabletMetadata {
       final String qual = key.getColumnQualifierData().toString();
 
       if (buildKeyValueMap) {
-        kvBuilder.put(key, kv.getValue());
+        tmBuilder.keyValue(key, kv.getValue());
       }
 
       if (row == null) {
         row = key.getRowData();
         KeyExtent ke = KeyExtent.fromMetaRow(key.getRow());
-        te.endRow = ke.endRow();
-        te.tableId = ke.tableId();
+        tmBuilder.endRow(ke.endRow());
+        tmBuilder.table(ke.tableId());
       } else if (!row.equals(key.getRowData())) {
         throw new IllegalArgumentException(
             "Input contains more than one row : " + row + " " + key.getRowData());
@@ -427,15 +447,15 @@ public class TabletMetadata {
         case TabletColumnFamily.STR_NAME:
           switch (qual) {
             case PREV_ROW_QUAL:
-              te.prevEndRow = TabletColumnFamily.decodePrevEndRow(kv.getValue());
-              te.sawPrevEndRow = true;
+              tmBuilder.prevEndRow(TabletColumnFamily.decodePrevEndRow(kv.getValue()));
+              tmBuilder.sawPrevEndRow(true);
               break;
             case OLD_PREV_ROW_QUAL:
-              te.oldPrevEndRow = TabletColumnFamily.decodePrevEndRow(kv.getValue());
-              te.sawOldPrevEndRow = true;
+              tmBuilder.oldPrevEndRow(TabletColumnFamily.decodePrevEndRow(kv.getValue()));
+              tmBuilder.sawOldPrevEndRow(true);
               break;
             case SPLIT_RATIO_QUAL:
-              te.splitRatio = Double.parseDouble(val);
+              tmBuilder.splitRatio(Double.parseDouble(val));
               break;
           }
           break;
@@ -444,88 +464,70 @@ public class TabletMetadata {
             case DIRECTORY_QUAL:
               Preconditions.checkArgument(ServerColumnFamily.isValidDirCol(val),
                   "Saw invalid dir name %s %s", key, val);
-              te.dirName = val;
+              tmBuilder.dirName(val);
               break;
             case TIME_QUAL:
-              te.time = MetadataTime.parse(val);
+              tmBuilder.time(MetadataTime.parse(val));
               break;
             case FLUSH_QUAL:
-              te.flush = OptionalLong.of(Long.parseLong(val));
+              tmBuilder.flush(Long.parseLong(val));
               break;
             case COMPACT_QUAL:
-              te.compact = OptionalLong.of(Long.parseLong(val));
+              tmBuilder.compact(Long.parseLong(val));
               break;
           }
           break;
         case DataFileColumnFamily.STR_NAME:
-          filesBuilder.put(new StoredTabletFile(qual), new DataFileValue(val));
+          tmBuilder.file(new StoredTabletFile(qual), new DataFileValue(val));
           break;
         case BulkFileColumnFamily.STR_NAME:
-          loadedFilesBuilder.put(new StoredTabletFile(qual),
+          tmBuilder.loadedFile(new StoredTabletFile(qual),
               BulkFileColumnFamily.getBulkLoadTid(val));
           break;
         case CurrentLocationColumnFamily.STR_NAME:
-          te.setLocationOnce(val, qual, LocationType.CURRENT);
+          tmBuilder.location(val, qual, LocationType.CURRENT);
           break;
         case FutureLocationColumnFamily.STR_NAME:
-          te.setLocationOnce(val, qual, LocationType.FUTURE);
+          tmBuilder.location(val, qual, LocationType.FUTURE);
           break;
         case LastLocationColumnFamily.STR_NAME:
-          te.last = Location.last(val, qual);
+          tmBuilder.last(Location.last(val, qual));
           break;
         case SuspendLocationColumn.STR_NAME:
-          te.suspend = SuspendingTServer.fromValue(kv.getValue());
+          tmBuilder.suspend(SuspendingTServer.fromValue(kv.getValue()));
           break;
         case ScanFileColumnFamily.STR_NAME:
-          scansBuilder.add(new StoredTabletFile(qual));
+          tmBuilder.scan(new StoredTabletFile(qual));
           break;
         case ClonedColumnFamily.STR_NAME:
-          te.cloned = val;
+          tmBuilder.cloned(val);
           break;
         case LogColumnFamily.STR_NAME:
-          logsBuilder.add(LogEntry.fromMetaWalEntry(kv));
+          tmBuilder.log(LogEntry.fromMetaWalEntry(kv));
           break;
         case ExternalCompactionColumnFamily.STR_NAME:
-          extCompBuilder.put(ExternalCompactionId.of(qual),
+          tmBuilder.extCompaction(ExternalCompactionId.of(qual),
               ExternalCompactionMetadata.fromJson(val));
           break;
         case MergedColumnFamily.STR_NAME:
-          te.merged = true;
+          tmBuilder.merged(true);
           break;
         default:
           throw new IllegalStateException("Unexpected family " + fam);
       }
     }
 
-    te.files = filesBuilder.build();
-    te.loadedFiles = loadedFilesBuilder.build();
-    te.fetchedCols = fetchedColumns;
-    te.scans = scansBuilder.build();
-    te.logs = logsBuilder.build();
-    te.extCompactions = extCompBuilder.build();
-    if (buildKeyValueMap) {
-      te.keyValues = kvBuilder.build();
-    }
-    return te;
-  }
-
-  private void setLocationOnce(String val, String qual, LocationType lt) {
-    if (location != null) {
-      throw new IllegalStateException("Attempted to set second location for tableId: " + tableId
-          + " endrow: " + endRow + " -- " + location + " " + qual + " " + val);
-    }
-    location = new Location(val, qual, lt);
+    return tmBuilder.build(fetchedColumns);
   }
 
   @VisibleForTesting
   static TabletMetadata create(String id, String prevEndRow, String endRow) {
-    TabletMetadata te = new TabletMetadata();
-    te.tableId = TableId.of(id);
-    te.sawPrevEndRow = true;
-    te.prevEndRow = prevEndRow == null ? null : new Text(prevEndRow);
-    te.endRow = endRow == null ? null : new Text(endRow);
-    te.fetchedCols = EnumSet.of(ColumnType.PREV_ROW);
-    return te;
+    final var tmBuilder = new Builder();
+    tmBuilder.table(TableId.of(id));
+    tmBuilder.sawPrevEndRow(true);
+    tmBuilder.prevEndRow(prevEndRow == null ? null : new Text(prevEndRow));
+    tmBuilder.endRow(endRow == null ? null : new Text(endRow));
+    return tmBuilder.build(EnumSet.of(ColumnType.PREV_ROW));
   }
 
   /**
@@ -558,4 +560,132 @@ public class TabletMetadata {
         .map(sld -> sld.getAddress(ServiceLockData.ThriftService.TSERV))
         .map(address -> new TServerInstance(address, stat.getEphemeralOwner()));
   }
+
+  static class Builder {
+    private TableId tableId;
+    private Text prevEndRow;
+    private boolean sawPrevEndRow;
+    private Text oldPrevEndRow;
+    private boolean sawOldPrevEndRow;
+    private Text endRow;
+    private Location location;
+    private final ImmutableMap.Builder<StoredTabletFile,DataFileValue> files =
+        ImmutableMap.builder();
+    private final ImmutableList.Builder<StoredTabletFile> scans = ImmutableList.builder();
+    private final ImmutableMap.Builder<StoredTabletFile,Long> loadedFiles = ImmutableMap.builder();
+    private EnumSet<ColumnType> fetchedCols;
+    private Location last;
+    private SuspendingTServer suspend;
+    private String dirName;
+    private MetadataTime time;
+    private String cloned;
+    private ImmutableSortedMap.Builder<Key,Value> keyValues;
+    private OptionalLong flush = OptionalLong.empty();
+    private final ImmutableList.Builder<LogEntry> logs = ImmutableList.builder();
+    private OptionalLong compact = OptionalLong.empty();
+    private Double splitRatio = null;
+    private final ImmutableMap.Builder<ExternalCompactionId,
+        ExternalCompactionMetadata> extCompactions = ImmutableMap.builder();
+    private boolean merged;
+
+    void table(TableId tableId) {
+      this.tableId = tableId;
+    }
+
+    void endRow(Text endRow) {
+      this.endRow = endRow;
+    }
+
+    void prevEndRow(Text prevEndRow) {
+      this.prevEndRow = prevEndRow;
+    }
+
+    void sawPrevEndRow(boolean sawPrevEndRow) {
+      this.sawPrevEndRow = sawPrevEndRow;
+    }
+
+    void oldPrevEndRow(Text oldPrevEndRow) {
+      this.oldPrevEndRow = oldPrevEndRow;
+    }
+
+    void sawOldPrevEndRow(boolean sawOldPrevEndRow) {
+      this.sawOldPrevEndRow = sawOldPrevEndRow;
+    }
+
+    void splitRatio(Double splitRatio) {
+      this.splitRatio = splitRatio;
+    }
+
+    void dirName(String dirName) {
+      this.dirName = dirName;
+    }
+
+    void time(MetadataTime time) {
+      this.time = time;
+    }
+
+    void flush(long flush) {
+      this.flush = OptionalLong.of(flush);
+    }
+
+    void compact(long compact) {
+      this.compact = OptionalLong.of(compact);
+    }
+
+    void file(StoredTabletFile stf, DataFileValue dfv) {
+      this.files.put(stf, dfv);
+    }
+
+    void loadedFile(StoredTabletFile stf, Long tid) {
+      this.loadedFiles.put(stf, tid);
+    }
+
+    void location(String val, String qual, LocationType lt) {
+      if (location != null) {
+        throw new IllegalStateException("Attempted to set second location for tableId: " + tableId
+            + " endrow: " + endRow + " -- " + location + " " + qual + " " + val);
+      }
+      this.location = new Location(val, qual, lt);
+    }
+
+    void last(Location last) {
+      this.last = last;
+    }
+
+    void suspend(SuspendingTServer suspend) {
+      this.suspend = suspend;
+    }
+
+    void scan(StoredTabletFile stf) {
+      this.scans.add(stf);
+    }
+
+    void cloned(String cloned) {
+      this.cloned = cloned;
+    }
+
+    void log(LogEntry log) {
+      this.logs.add(log);
+    }
+
+    void extCompaction(ExternalCompactionId id, ExternalCompactionMetadata metadata) {
+      this.extCompactions.put(id, metadata);
+    }
+
+    void merged(boolean merged) {
+      this.merged = merged;
+    }
+
+    void keyValue(Key key, Value value) {
+      if (this.keyValues == null) {
+        this.keyValues = ImmutableSortedMap.naturalOrder();
+      }
+      this.keyValues.put(key, value);
+    }
+
+    TabletMetadata build(EnumSet<ColumnType> fetchedCols) {
+      this.fetchedCols = fetchedCols;
+      return new TabletMetadata(this);
+    }
+  }
 }
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 448d3bacfd..ac94fc8f69 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
@@ -26,6 +26,9 @@ import static org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSec
 import static org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.ServerColumnFamily.DIRECTORY_COLUMN;
 import static org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.ServerColumnFamily.FLUSH_COLUMN;
 import static org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.ServerColumnFamily.TIME_COLUMN;
+import static org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.SuspendLocationColumn.SUSPEND_COLUMN;
+import static org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.TabletColumnFamily.OLD_PREV_ROW_COLUMN;
+import static org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.TabletColumnFamily.SPLIT_RATIO_COLUMN;
 import static org.apache.accumulo.core.metadata.schema.TabletMetadata.ColumnType.LAST;
 import static org.apache.accumulo.core.metadata.schema.TabletMetadata.ColumnType.LOCATION;
 import static org.apache.accumulo.core.metadata.schema.TabletMetadata.ColumnType.PREV_ROW;
@@ -54,6 +57,7 @@ import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.dataImpl.KeyExtent;
 import org.apache.accumulo.core.fate.FateTxId;
+import org.apache.accumulo.core.metadata.ReferencedTabletFile;
 import org.apache.accumulo.core.metadata.StoredTabletFile;
 import org.apache.accumulo.core.metadata.SuspendingTServer;
 import org.apache.accumulo.core.metadata.TServerInstance;
@@ -62,14 +66,18 @@ import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.Bu
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.ClonedColumnFamily;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.CurrentLocationColumnFamily;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.DataFileColumnFamily;
+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.ScanFileColumnFamily;
-import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.SuspendLocationColumn;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.TabletColumnFamily;
+import org.apache.accumulo.core.metadata.schema.TabletMetadata.Builder;
 import org.apache.accumulo.core.metadata.schema.TabletMetadata.ColumnType;
 import org.apache.accumulo.core.metadata.schema.TabletMetadata.LocationType;
+import org.apache.accumulo.core.spi.compaction.CompactionExecutorId;
+import org.apache.accumulo.core.spi.compaction.CompactionKind;
 import org.apache.accumulo.core.tabletserver.log.LogEntry;
+import org.apache.accumulo.core.util.compaction.CompactionExecutorIdImpl;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.Text;
 import org.junit.jupiter.api.Test;
@@ -119,6 +127,24 @@ public class TabletMetadataTest {
 
     MERGED_COLUMN.put(mutation, new Value());
 
+    OLD_PREV_ROW_COLUMN.put(mutation, TabletColumnFamily.encodePrevEndRow(new Text("oldPrev")));
+    long suspensionTime = System.currentTimeMillis();
+    TServerInstance ser1 = new TServerInstance(HostAndPort.fromParts("server1", 8555), "s001");
+    Value suspend = SuspendingTServer.toValue(ser1, suspensionTime);
+    SUSPEND_COLUMN.put(mutation, suspend);
+    double splitRatio = .3;
+    SPLIT_RATIO_COLUMN.put(mutation, new Value(Double.toString(splitRatio)));
+
+    ExternalCompactionId ecid = ExternalCompactionId.generate(UUID.randomUUID());
+    ReferencedTabletFile tmpFile =
+        ReferencedTabletFile.of(new Path("file:///accumulo/tables/t-0/b-0/c1.rf"));
+    CompactionExecutorId ceid = CompactionExecutorIdImpl.externalId("G1");
+    Set<StoredTabletFile> jobFiles =
+        Set.of(StoredTabletFile.of(new Path("file:///accumulo/tables/t-0/b-0/b2.rf")));
+    ExternalCompactionMetadata ecMeta = new ExternalCompactionMetadata(jobFiles, jobFiles, tmpFile,
+        "localhost:4444", CompactionKind.SYSTEM, (short) 2, ceid, false, false, 44L);
+    mutation.put(ExternalCompactionColumnFamily.STR_NAME, ecid.canonical(), ecMeta.toJson());
+
     SortedMap<Key,Value> rowMap = toRowMap(mutation);
 
     TabletMetadata tm = TabletMetadata.convertRow(rowMap.entrySet().iterator(),
@@ -149,6 +175,11 @@ public class TabletMetadataTest {
     assertEquals("M123456789", tm.getTime().encode());
     assertEquals(Set.of(sf1, sf2), Set.copyOf(tm.getScans()));
     assertTrue(tm.hasMerged());
+    assertEquals(new Text("oldPrev"), tm.getOldPrevEndRow());
+    assertTrue(tm.sawOldPrevEndRow());
+    assertEquals(SuspendingTServer.fromValue(suspend), tm.getSuspend());
+    assertEquals(splitRatio, tm.getSplitRatio());
+    assertEquals(ecMeta.toJson(), tm.getExternalCompactions().get(ecid).toJson());
   }
 
   @Test
@@ -257,9 +288,8 @@ public class TabletMetadataTest {
 
     // test SUSPENDED
     mutation = TabletColumnFamily.createPrevRowMutation(extent);
-    mutation.at().family(SuspendLocationColumn.SUSPEND_COLUMN.getColumnFamily())
-        .qualifier(SuspendLocationColumn.SUSPEND_COLUMN.getColumnQualifier())
-        .put(SuspendingTServer.toValue(ser2, 1000L));
+    mutation.at().family(SUSPEND_COLUMN.getColumnFamily())
+        .qualifier(SUSPEND_COLUMN.getColumnQualifier()).put(SuspendingTServer.toValue(ser2, 1000L));
     rowMap = toRowMap(mutation);
 
     tm = TabletMetadata.convertRow(rowMap.entrySet().iterator(), colsToFetch, false);
@@ -316,6 +346,69 @@ public class TabletMetadataTest {
     assertTrue(closeCalled.get());
   }
 
+  @Test
+  public void testTmBuilderImmutable() {
+    TabletMetadata.Builder b = new Builder();
+    var tm = b.build(EnumSet.allOf(ColumnType.class));
+
+    ExternalCompactionId ecid = ExternalCompactionId.generate(UUID.randomUUID());
+    ReferencedTabletFile tmpFile =
+        ReferencedTabletFile.of(new Path("file:///accumulo/tables/t-0/b-0/c1.rf"));
+    CompactionExecutorId ceid = CompactionExecutorIdImpl.externalId("G1");
+    StoredTabletFile stf = StoredTabletFile.of(new Path("file:///accumulo/tables/t-0/b-0/b2.rf"));
+    Set<StoredTabletFile> jobFiles = Set.of(stf);
+    ExternalCompactionMetadata ecMeta = new ExternalCompactionMetadata(jobFiles, jobFiles, tmpFile,
+        "localhost:4444", CompactionKind.SYSTEM, (short) 2, ceid, false, false, 44L);
+
+    // Verify the various collections are immutable and non-null (except for getKeyValues) if
+    // nothing set on the builder
+    assertTrue(tm.getExternalCompactions().isEmpty());
+    assertThrows(UnsupportedOperationException.class,
+        () -> tm.getExternalCompactions().put(ecid, ecMeta));
+    assertTrue(tm.getFiles().isEmpty());
+    assertTrue(tm.getFilesMap().isEmpty());
+    assertThrows(UnsupportedOperationException.class, () -> tm.getFiles().add(stf));
+    assertThrows(UnsupportedOperationException.class,
+        () -> tm.getFilesMap().put(stf, new DataFileValue(0, 0, 0)));
+    assertTrue(tm.getLogs().isEmpty());
+    assertThrows(UnsupportedOperationException.class,
+        () -> tm.getLogs().add(LogEntry.fromPath("localhost+8020/" + UUID.randomUUID())));
+    assertTrue(tm.getScans().isEmpty());
+    assertThrows(UnsupportedOperationException.class, () -> tm.getScans().add(stf));
+    assertTrue(tm.getLoaded().isEmpty());
+    assertThrows(UnsupportedOperationException.class, () -> tm.getLoaded().put(stf, 0L));
+    assertThrows(IllegalStateException.class, tm::getKeyValues);
+
+    // Set some data in the collections and very they are not empty but still immutable
+    b.extCompaction(ecid, ecMeta);
+    b.file(stf, new DataFileValue(0, 0, 0));
+    b.log(LogEntry.fromPath("localhost+8020/" + UUID.randomUUID()));
+    b.scan(stf);
+    b.loadedFile(stf, 0L);
+    b.keyValue(new Key(), new Value());
+    var tm2 = b.build(EnumSet.allOf(ColumnType.class));
+
+    assertEquals(1, tm2.getExternalCompactions().size());
+    assertThrows(UnsupportedOperationException.class,
+        () -> tm2.getExternalCompactions().put(ecid, ecMeta));
+    assertEquals(1, tm2.getFiles().size());
+    assertEquals(1, tm2.getFilesMap().size());
+    assertThrows(UnsupportedOperationException.class, () -> tm2.getFiles().add(stf));
+    assertThrows(UnsupportedOperationException.class,
+        () -> tm2.getFilesMap().put(stf, new DataFileValue(0, 0, 0)));
+    assertEquals(1, tm2.getLogs().size());
+    assertThrows(UnsupportedOperationException.class,
+        () -> tm2.getLogs().add(LogEntry.fromPath("localhost+8020/" + UUID.randomUUID())));
+    assertEquals(1, tm2.getScans().size());
+    assertThrows(UnsupportedOperationException.class, () -> tm2.getScans().add(stf));
+    assertEquals(1, tm2.getLoaded().size());
+    assertThrows(UnsupportedOperationException.class, () -> tm2.getLoaded().put(stf, 0L));
+    assertEquals(1, tm2.getKeyValues().size());
+    assertThrows(UnsupportedOperationException.class,
+        () -> tm2.getKeyValues().put(new Key(), new Value()));
+
+  }
+
   private SortedMap<Key,Value> toRowMap(Mutation mutation) {
     SortedMap<Key,Value> rowMap = new TreeMap<>();
     mutation.getUpdates().forEach(cu -> {