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

ddanielr 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 15035d7b27 Update ScanServerRefFile format to sort on UUID (#4691)
15035d7b27 is described below

commit 15035d7b2705b606e401a09984e7694cb263e274
Author: Daniel Roberts <[email protected]>
AuthorDate: Wed Jul 24 16:11:52 2024 -0400

    Update ScanServerRefFile format to sort on UUID (#4691)
    
    Updates the ScanServerRef format and moves the UUID entry forward so
    sorting is done on UUID instead of filename.
    
    Also adds the removal of the scan serverref range from the metadata
    table
    
    Separates the scanRef table node creation into a separate method that
      can be called from the upgrader code
    
    ---------
    
    Co-authored-by: Keith Turner <[email protected]>
---
 .../core/metadata/ScanServerRefTabletFile.java     |  58 +++--
 .../server/init/FileSystemInitializer.java         | 135 +++++-----
 .../accumulo/server/init/InitialConfiguration.java |   4 +-
 .../apache/accumulo/server/init/Initialize.java    |   2 +-
 .../accumulo/server/init/ZooKeeperInitializer.java |  22 +-
 .../server/metadata/ScanServerRefStoreImpl.java    |  20 +-
 .../server/util/ScanServerMetadataEntries.java     |   4 +-
 .../accumulo/manager/upgrade/Upgrader11to12.java   |  46 ++++
 .../org/apache/accumulo/tserver/ScanServer.java    |   4 +-
 .../upgrade/ScanServerUpgrade11to12TestIT.java     | 273 +++++++++++++++++++++
 10 files changed, 464 insertions(+), 104 deletions(-)

diff --git 
a/core/src/main/java/org/apache/accumulo/core/metadata/ScanServerRefTabletFile.java
 
b/core/src/main/java/org/apache/accumulo/core/metadata/ScanServerRefTabletFile.java
index 50076eaa6f..c289c38a47 100644
--- 
a/core/src/main/java/org/apache/accumulo/core/metadata/ScanServerRefTabletFile.java
+++ 
b/core/src/main/java/org/apache/accumulo/core/metadata/ScanServerRefTabletFile.java
@@ -22,6 +22,8 @@ import java.net.URI;
 import java.util.Objects;
 import java.util.UUID;
 
+import org.apache.accumulo.core.data.Key;
+import org.apache.accumulo.core.data.Mutation;
 import org.apache.accumulo.core.data.Value;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.Text;
@@ -29,43 +31,61 @@ import org.apache.hadoop.io.Text;
 public class ScanServerRefTabletFile extends ReferencedTabletFile {
 
   private final Value NULL_VALUE = new Value(new byte[0]);
-  private final Text colf;
-  private final Text colq;
+  private final Text serverAddress;
+  private final Text uuid;
 
-  public ScanServerRefTabletFile(String file, String serverAddress, UUID 
serverLockUUID) {
+  public ScanServerRefTabletFile(UUID serverLockUUID, String serverAddress, 
String file) {
     super(new Path(URI.create(file)));
-    this.colf = new Text(serverAddress);
-    this.colq = new Text(serverLockUUID.toString());
+    this.serverAddress = new Text(serverAddress);
+    uuid = new Text(serverLockUUID.toString());
   }
 
-  public ScanServerRefTabletFile(String file, Text colf, Text colq) {
+  public ScanServerRefTabletFile(String file, String serverAddress, UUID 
serverLockUUID) {
     super(new Path(URI.create(file)));
-    this.colf = colf;
-    this.colq = colq;
+    this.serverAddress = new Text(serverAddress);
+    this.uuid = new Text(serverLockUUID.toString());
   }
 
-  public String getRow() {
-    return this.getNormalizedPathStr();
+  public ScanServerRefTabletFile(Key k) {
+    super(new Path(URI.create(k.getColumnQualifier().toString())));
+    serverAddress = k.getColumnFamily();
+    uuid = k.getRow();
   }
 
-  public Text getServerAddress() {
-    return this.colf;
+  public Mutation putMutation() {
+    Mutation mutation = new Mutation(uuid.toString());
+    mutation.put(serverAddress, getFilePath(), getValue());
+    return mutation;
+  }
+
+  public Mutation putDeleteMutation() {
+    Mutation mutation = new Mutation(uuid.toString());
+    mutation.putDelete(serverAddress, getFilePath());
+    return mutation;
   }
 
-  public Text getServerLockUUID() {
-    return this.colq;
+  public Text getFilePath() {
+    return new Text(this.getNormalizedPathStr());
+  }
+
+  public UUID getServerLockUUID() {
+    return UUID.fromString(uuid.toString());
   }
 
   public Value getValue() {
     return NULL_VALUE;
   }
 
+  public Text getServerAddress() {
+    return serverAddress;
+  }
+
   @Override
   public int hashCode() {
     final int prime = 31;
     int result = super.hashCode();
-    result = prime * result + ((colf == null) ? 0 : colf.hashCode());
-    result = prime * result + ((colq == null) ? 0 : colq.hashCode());
+    result = prime * result + ((serverAddress == null) ? 0 : 
serverAddress.hashCode());
+    result = prime * result + ((uuid == null) ? 0 : uuid.hashCode());
     return result;
   }
 
@@ -81,13 +101,13 @@ public class ScanServerRefTabletFile extends 
ReferencedTabletFile {
       return false;
     }
     ScanServerRefTabletFile other = (ScanServerRefTabletFile) obj;
-    return Objects.equals(colf, other.colf) && Objects.equals(colq, 
other.colq);
+    return Objects.equals(serverAddress, other.serverAddress) && 
Objects.equals(uuid, other.uuid);
   }
 
   @Override
   public String toString() {
-    return "ScanServerRefTabletFile [file=" + this.getRow() + ", server 
address=" + colf
-        + ", server lock uuid=" + colq + "]";
+    return "ScanServerRefTabletFile [file=" + this.getNormalizedPathStr() + ", 
server address="
+        + serverAddress + ", server lock uuid=" + uuid + "]";
   }
 
 }
diff --git 
a/server/base/src/main/java/org/apache/accumulo/server/init/FileSystemInitializer.java
 
b/server/base/src/main/java/org/apache/accumulo/server/init/FileSystemInitializer.java
index 9db54c8f12..640e374589 100644
--- 
a/server/base/src/main/java/org/apache/accumulo/server/init/FileSystemInitializer.java
+++ 
b/server/base/src/main/java/org/apache/accumulo/server/init/FileSystemInitializer.java
@@ -32,13 +32,11 @@ import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.client.admin.TimeType;
 import org.apache.accumulo.core.conf.AccumuloConfiguration;
 import org.apache.accumulo.core.conf.DefaultConfiguration;
-import org.apache.accumulo.core.conf.SiteConfiguration;
 import org.apache.accumulo.core.crypto.CryptoFactoryLoader;
-import org.apache.accumulo.core.data.InstanceId;
 import org.apache.accumulo.core.data.Key;
+import org.apache.accumulo.core.data.Mutation;
 import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.data.Value;
-import org.apache.accumulo.core.fate.zookeeper.ZooReaderWriter;
 import org.apache.accumulo.core.file.FileOperations;
 import org.apache.accumulo.core.file.FileSKVWriter;
 import org.apache.accumulo.core.metadata.AccumuloTable;
@@ -62,81 +60,109 @@ import org.apache.zookeeper.KeeperException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-class FileSystemInitializer {
+public class FileSystemInitializer {
   private static final String TABLE_TABLETS_TABLET_DIR = "table_info";
   private static final Logger log = 
LoggerFactory.getLogger(FileSystemInitializer.class);
+  private static final Text SPLIT_POINT =
+      MetadataSchema.TabletsSection.getRange().getEndKey().getRow();
 
   // config only for root table
   private final InitialConfiguration initConfig;
 
-  FileSystemInitializer(InitialConfiguration initConfig, ZooReaderWriter zoo, 
InstanceId uuid) {
+  public FileSystemInitializer(InitialConfiguration initConfig) {
     this.initConfig = initConfig;
   }
 
-  private static class Tablet {
+  public static class InitialTablet {
     TableId tableId;
     String dirName;
-    Text prevEndRow, endRow;
+    Text prevEndRow, endRow, extent;
     String[] files;
 
-    Tablet(TableId tableId, String dirName, Text prevEndRow, Text endRow, 
String... files) {
+    InitialTablet(TableId tableId, String dirName, Text prevEndRow, Text 
endRow, String... files) {
       this.tableId = tableId;
       this.dirName = dirName;
       this.prevEndRow = prevEndRow;
       this.endRow = endRow;
       this.files = files;
+      this.extent = new 
Text(MetadataSchema.TabletsSection.encodeRow(this.tableId, this.endRow));
     }
+
+    private TreeMap<Key,Value> createEntries() {
+      TreeMap<Key,Value> sorted = new TreeMap<>();
+      Value EMPTY_SIZE = new DataFileValue(0, 0).encodeAsValue();
+      sorted.put(new Key(this.extent, DIRECTORY_COLUMN.getColumnFamily(),
+          DIRECTORY_COLUMN.getColumnQualifier(), 0), new Value(this.dirName));
+      sorted.put(
+          new Key(this.extent, TIME_COLUMN.getColumnFamily(), 
TIME_COLUMN.getColumnQualifier(), 0),
+          new Value(new MetadataTime(0, TimeType.LOGICAL).encode()));
+      sorted.put(
+          new Key(this.extent, PREV_ROW_COLUMN.getColumnFamily(),
+              PREV_ROW_COLUMN.getColumnQualifier(), 0),
+          
MetadataSchema.TabletsSection.TabletColumnFamily.encodePrevEndRow(this.prevEndRow));
+      for (String file : this.files) {
+        var col =
+            new 
ColumnFQ(MetadataSchema.TabletsSection.DataFileColumnFamily.NAME, new 
Text(file));
+        sorted.put(new Key(extent, col.getColumnFamily(), 
col.getColumnQualifier(), 0), EMPTY_SIZE);
+      }
+      return sorted;
+    }
+
+    public Mutation createMutation() {
+      Mutation mutation = new Mutation(this.extent);
+      for (Map.Entry<Key,Value> entry : createEntries().entrySet()) {
+        mutation.put(entry.getKey().getColumnFamily(), 
entry.getKey().getColumnQualifier(),
+            entry.getValue());
+      }
+      return mutation;
+    }
+
   }
 
   void initialize(VolumeManager fs, String rootTabletDirUri, String 
rootTabletFileUri,
       ServerContext context) throws IOException, InterruptedException, 
KeeperException {
-    SiteConfiguration siteConfig = initConfig.getSiteConf();
     // initialize initial system tables config in zookeeper
     initSystemTablesConfig(context);
 
-    Text splitPoint = 
MetadataSchema.TabletsSection.getRange().getEndKey().getRow();
-
     VolumeChooserEnvironment chooserEnv = new VolumeChooserEnvironmentImpl(
-        VolumeChooserEnvironment.Scope.INIT, AccumuloTable.METADATA.tableId(), 
splitPoint, context);
-    String tableMetadataTabletDirName = TABLE_TABLETS_TABLET_DIR;
-    String tableMetadataTabletDirUri =
+        VolumeChooserEnvironment.Scope.INIT, AccumuloTable.METADATA.tableId(), 
null, context);
+    String defaultMetadataTabletDirName =
+        
MetadataSchema.TabletsSection.ServerColumnFamily.DEFAULT_TABLET_DIR_NAME;
+    String defaultMetadataTabletDirUri =
         fs.choose(chooserEnv, context.getBaseUris()) + 
Constants.HDFS_TABLES_DIR + Path.SEPARATOR
-            + AccumuloTable.METADATA.tableId() + Path.SEPARATOR + 
tableMetadataTabletDirName;
+            + AccumuloTable.METADATA.tableId() + Path.SEPARATOR + 
defaultMetadataTabletDirName;
 
     chooserEnv = new 
VolumeChooserEnvironmentImpl(VolumeChooserEnvironment.Scope.INIT,
-        AccumuloTable.SCAN_REF.tableId(), null, context);
-    String scanRefTableDefaultTabletDirName =
-        
MetadataSchema.TabletsSection.ServerColumnFamily.DEFAULT_TABLET_DIR_NAME;
-    String scanRefTableDefaultTabletDirUri =
+        AccumuloTable.METADATA.tableId(), SPLIT_POINT, context);
+
+    String tableMetadataTabletDirUri =
         fs.choose(chooserEnv, context.getBaseUris()) + 
Constants.HDFS_TABLES_DIR + Path.SEPARATOR
-            + AccumuloTable.SCAN_REF.tableId() + Path.SEPARATOR + 
scanRefTableDefaultTabletDirName;
+            + AccumuloTable.METADATA.tableId() + Path.SEPARATOR + 
TABLE_TABLETS_TABLET_DIR;
 
     chooserEnv = new 
VolumeChooserEnvironmentImpl(VolumeChooserEnvironment.Scope.INIT,
-        AccumuloTable.METADATA.tableId(), null, context);
-    String defaultMetadataTabletDirName =
-        
MetadataSchema.TabletsSection.ServerColumnFamily.DEFAULT_TABLET_DIR_NAME;
-    String defaultMetadataTabletDirUri =
-        fs.choose(chooserEnv, context.getBaseUris()) + 
Constants.HDFS_TABLES_DIR + Path.SEPARATOR
-            + AccumuloTable.METADATA.tableId() + Path.SEPARATOR + 
defaultMetadataTabletDirName;
+        AccumuloTable.SCAN_REF.tableId(), null, context);
+
+    String scanRefTableDefaultTabletDirUri = fs.choose(chooserEnv, 
context.getBaseUris())
+        + Constants.HDFS_TABLES_DIR + Path.SEPARATOR + 
AccumuloTable.SCAN_REF.tableId()
+        + Path.SEPARATOR + 
MetadataSchema.TabletsSection.ServerColumnFamily.DEFAULT_TABLET_DIR_NAME;
 
     // create table and default tablets directories
-    createDirectories(fs, rootTabletDirUri, tableMetadataTabletDirUri, 
defaultMetadataTabletDirUri,
+    createDirectories(fs, rootTabletDirUri, defaultMetadataTabletDirUri, 
tableMetadataTabletDirUri,
         scanRefTableDefaultTabletDirUri);
-
-    String ext = 
FileOperations.getNewFileExtension(DefaultConfiguration.getInstance());
+    InitialTablet scanRefTablet = createScanRefTablet(context);
 
     // populate the metadata tablet with info about scan ref tablets
+    String ext = 
FileOperations.getNewFileExtension(DefaultConfiguration.getInstance());
     String metadataFileName = tableMetadataTabletDirUri + Path.SEPARATOR + 
"0_1." + ext;
-    Tablet scanRefTablet =
-        new Tablet(AccumuloTable.SCAN_REF.tableId(), 
scanRefTableDefaultTabletDirName, null, null);
-    createMetadataFile(fs, metadataFileName, siteConfig, scanRefTablet);
+    createMetadataFile(fs, metadataFileName, scanRefTablet);
 
     // populate the root tablet with info about the metadata table's two 
initial tablets
-    Tablet tablesTablet = new Tablet(AccumuloTable.METADATA.tableId(), 
tableMetadataTabletDirName,
-        null, splitPoint, StoredTabletFile.of(new 
Path(metadataFileName)).getMetadata());
-    Tablet defaultTablet = new Tablet(AccumuloTable.METADATA.tableId(),
-        defaultMetadataTabletDirName, splitPoint, null);
-    createMetadataFile(fs, rootTabletFileUri, siteConfig, tablesTablet, 
defaultTablet);
+    InitialTablet tablesTablet =
+        new InitialTablet(AccumuloTable.METADATA.tableId(), 
TABLE_TABLETS_TABLET_DIR, null,
+            SPLIT_POINT, StoredTabletFile.of(new 
Path(metadataFileName)).getMetadata());
+    InitialTablet defaultTablet = new 
InitialTablet(AccumuloTable.METADATA.tableId(),
+        defaultMetadataTabletDirName, SPLIT_POINT, null);
+    createMetadataFile(fs, rootTabletFileUri, tablesTablet, defaultTablet);
   }
 
   private void createDirectories(VolumeManager fs, String... dirs) throws 
IOException {
@@ -164,7 +190,6 @@ class FileSystemInitializer {
     setTableProperties(context, AccumuloTable.ROOT.tableId(), 
initConfig.getRootMetaConf());
     setTableProperties(context, AccumuloTable.METADATA.tableId(), 
initConfig.getRootMetaConf());
     setTableProperties(context, AccumuloTable.METADATA.tableId(), 
initConfig.getMetaTableConf());
-    setTableProperties(context, AccumuloTable.SCAN_REF.tableId(), 
initConfig.getScanRefTableConf());
   }
 
   private void setTableProperties(final ServerContext context, TableId tableId,
@@ -179,12 +204,8 @@ class FileSystemInitializer {
   }
 
   private void createMetadataFile(VolumeManager volmanager, String fileName,
-      AccumuloConfiguration conf, Tablet... tablets) throws IOException {
-    // sort file contents in memory, then play back to the file
-    TreeMap<Key,Value> sorted = new TreeMap<>();
-    for (Tablet tablet : tablets) {
-      createEntriesForTablet(sorted, tablet);
-    }
+      InitialTablet... initialTablets) throws IOException {
+    AccumuloConfiguration conf = initConfig.getSiteConf();
     ReferencedTabletFile file = ReferencedTabletFile.of(new Path(fileName));
     FileSystem fs = volmanager.getFileSystemByPath(file.getPath());
 
@@ -194,28 +215,22 @@ class FileSystemInitializer {
         .forFile(file, fs, fs.getConf(), 
cs).withTableConfiguration(conf).build();
     tabletWriter.startDefaultLocalityGroup();
 
+    TreeMap<Key,Value> sorted = new TreeMap<>();
+    for (InitialTablet initialTablet : initialTablets) {
+      // sort file contents in memory, then play back to the file
+      sorted.putAll(initialTablet.createEntries());
+    }
+
     for (Map.Entry<Key,Value> entry : sorted.entrySet()) {
       tabletWriter.append(entry.getKey(), entry.getValue());
     }
-
     tabletWriter.close();
   }
 
-  private void createEntriesForTablet(TreeMap<Key,Value> map, Tablet tablet) {
-    Value EMPTY_SIZE = new DataFileValue(0, 0).encodeAsValue();
-    Text extent = new 
Text(MetadataSchema.TabletsSection.encodeRow(tablet.tableId, tablet.endRow));
-    addEntry(map, extent, DIRECTORY_COLUMN, new Value(tablet.dirName));
-    addEntry(map, extent, TIME_COLUMN, new Value(new MetadataTime(0, 
TimeType.LOGICAL).encode()));
-    addEntry(map, extent, PREV_ROW_COLUMN,
-        
MetadataSchema.TabletsSection.TabletColumnFamily.encodePrevEndRow(tablet.prevEndRow));
-    for (String file : tablet.files) {
-      addEntry(map, extent,
-          new 
ColumnFQ(MetadataSchema.TabletsSection.DataFileColumnFamily.NAME, new 
Text(file)),
-          EMPTY_SIZE);
-    }
-  }
+  public InitialTablet createScanRefTablet(ServerContext context) throws 
IOException {
+    setTableProperties(context, AccumuloTable.SCAN_REF.tableId(), 
initConfig.getScanRefTableConf());
 
-  private void addEntry(TreeMap<Key,Value> map, Text row, ColumnFQ col, Value 
value) {
-    map.put(new Key(row, col.getColumnFamily(), col.getColumnQualifier(), 0), 
value);
+    return new InitialTablet(AccumuloTable.SCAN_REF.tableId(),
+        
MetadataSchema.TabletsSection.ServerColumnFamily.DEFAULT_TABLET_DIR_NAME, null, 
null);
   }
 }
diff --git 
a/server/base/src/main/java/org/apache/accumulo/server/init/InitialConfiguration.java
 
b/server/base/src/main/java/org/apache/accumulo/server/init/InitialConfiguration.java
index 52ba4a2eda..3dac8b4ca1 100644
--- 
a/server/base/src/main/java/org/apache/accumulo/server/init/InitialConfiguration.java
+++ 
b/server/base/src/main/java/org/apache/accumulo/server/init/InitialConfiguration.java
@@ -34,7 +34,7 @@ import 
org.apache.accumulo.server.constraints.MetadataConstraints;
 import org.apache.accumulo.server.iterators.MetadataBulkLoadFilter;
 import org.apache.hadoop.conf.Configuration;
 
-class InitialConfiguration {
+public class InitialConfiguration {
 
   // config only for root table
   private final HashMap<String,String> initialRootConf = new HashMap<>();
@@ -47,7 +47,7 @@ class InitialConfiguration {
   private final Configuration hadoopConf;
   private final SiteConfiguration siteConf;
 
-  InitialConfiguration(Configuration hadoopConf, SiteConfiguration siteConf) {
+  public InitialConfiguration(Configuration hadoopConf, SiteConfiguration 
siteConf) {
     this.hadoopConf = hadoopConf;
     this.siteConf = siteConf;
 
diff --git 
a/server/base/src/main/java/org/apache/accumulo/server/init/Initialize.java 
b/server/base/src/main/java/org/apache/accumulo/server/init/Initialize.java
index 8a6716d7ec..e9babfe59c 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/init/Initialize.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/init/Initialize.java
@@ -175,7 +175,7 @@ public class Initialize implements KeywordExecutable {
       if (!createDirs(fs, instanceId, initConfig.getVolumeUris())) {
         throw new IOException("Problem creating directories on " + 
fs.getVolumes());
       }
-      var fileSystemInitializer = new FileSystemInitializer(initConfig, zoo, 
instanceId);
+      var fileSystemInitializer = new FileSystemInitializer(initConfig);
       var rootVol = fs.choose(chooserEnv, initConfig.getVolumeUris());
       var rootPath = new Path(rootVol + SEPARATOR + TABLE_DIR + SEPARATOR
           + AccumuloTable.ROOT.tableId() + SEPARATOR + rootTabletDirName);
diff --git 
a/server/base/src/main/java/org/apache/accumulo/server/init/ZooKeeperInitializer.java
 
b/server/base/src/main/java/org/apache/accumulo/server/init/ZooKeeperInitializer.java
index 97b14c26e0..50dfc76b23 100644
--- 
a/server/base/src/main/java/org/apache/accumulo/server/init/ZooKeeperInitializer.java
+++ 
b/server/base/src/main/java/org/apache/accumulo/server/init/ZooKeeperInitializer.java
@@ -118,10 +118,14 @@ public class ZooKeeperInitializer {
     TableManager.prepareNewNamespaceState(context, Namespace.ACCUMULO.id(),
         Namespace.ACCUMULO.name(), ZooUtil.NodeExistsPolicy.FAIL);
 
-    for (AccumuloTable table : AccumuloTable.values()) {
-      TableManager.prepareNewTableState(context, table.tableId(), 
Namespace.ACCUMULO.id(),
-          table.tableName(), TableState.ONLINE, ZooUtil.NodeExistsPolicy.FAIL);
-    }
+    TableManager.prepareNewTableState(context, AccumuloTable.ROOT.tableId(),
+        Namespace.ACCUMULO.id(), AccumuloTable.ROOT.tableName(), 
TableState.ONLINE,
+        ZooUtil.NodeExistsPolicy.FAIL);
+    TableManager.prepareNewTableState(context, 
AccumuloTable.METADATA.tableId(),
+        Namespace.ACCUMULO.id(), AccumuloTable.METADATA.tableName(), 
TableState.ONLINE,
+        ZooUtil.NodeExistsPolicy.FAIL);
+    // Call this separately so the upgrader code can handle the zk node 
creation for scan refs
+    initScanRefTableState(context);
 
     zoo.putPersistentData(zkInstanceRoot + Constants.ZTSERVERS, 
EMPTY_BYTE_ARRAY,
         ZooUtil.NodeExistsPolicy.FAIL);
@@ -188,4 +192,14 @@ public class ZooKeeperInitializer {
     return rootTabletJson.toJson().getBytes(UTF_8);
   }
 
+  public void initScanRefTableState(ServerContext context) {
+    try {
+      TableManager.prepareNewTableState(context, 
AccumuloTable.SCAN_REF.tableId(),
+          Namespace.ACCUMULO.id(), AccumuloTable.SCAN_REF.tableName(), 
TableState.ONLINE,
+          ZooUtil.NodeExistsPolicy.FAIL);
+    } catch (KeeperException | InterruptedException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
 }
diff --git 
a/server/base/src/main/java/org/apache/accumulo/server/metadata/ScanServerRefStoreImpl.java
 
b/server/base/src/main/java/org/apache/accumulo/server/metadata/ScanServerRefStoreImpl.java
index 1763b9d099..c3c5056a46 100644
--- 
a/server/base/src/main/java/org/apache/accumulo/server/metadata/ScanServerRefStoreImpl.java
+++ 
b/server/base/src/main/java/org/apache/accumulo/server/metadata/ScanServerRefStoreImpl.java
@@ -31,11 +31,10 @@ import 
org.apache.accumulo.core.client.MutationsRejectedException;
 import org.apache.accumulo.core.client.Scanner;
 import org.apache.accumulo.core.client.TableNotFoundException;
 import org.apache.accumulo.core.clientImpl.ClientContext;
-import org.apache.accumulo.core.data.Mutation;
+import org.apache.accumulo.core.data.Range;
 import org.apache.accumulo.core.metadata.ScanServerRefStore;
 import org.apache.accumulo.core.metadata.ScanServerRefTabletFile;
 import org.apache.accumulo.core.security.Authorizations;
-import org.apache.hadoop.io.Text;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -55,9 +54,7 @@ public class ScanServerRefStoreImpl implements 
ScanServerRefStore {
   public void put(Collection<ScanServerRefTabletFile> scanRefs) {
     try (BatchWriter writer = context.createBatchWriter(tableName)) {
       for (ScanServerRefTabletFile ref : scanRefs) {
-        Mutation m = new Mutation(ref.getRow());
-        m.put(ref.getServerAddress(), ref.getServerLockUUID(), ref.getValue());
-        writer.addMutation(m);
+        writer.addMutation(ref.putMutation());
       }
     } catch (MutationsRejectedException | TableNotFoundException e) {
       throw new IllegalStateException(
@@ -70,8 +67,7 @@ public class ScanServerRefStoreImpl implements 
ScanServerRefStore {
     try {
       Scanner scanner = context.createScanner(tableName, Authorizations.EMPTY);
       return scanner.stream().onClose(scanner::close)
-          .map(e -> new 
ScanServerRefTabletFile(e.getKey().getRowData().toString(),
-              e.getKey().getColumnFamily(), e.getKey().getColumnQualifier()));
+          .map(e -> new ScanServerRefTabletFile(e.getKey()));
     } catch (TableNotFoundException e) {
       throw new IllegalStateException(tableName + " not found!", e);
     }
@@ -82,12 +78,10 @@ public class ScanServerRefStoreImpl implements 
ScanServerRefStore {
     Objects.requireNonNull(serverAddress, "Server address must be supplied");
     Objects.requireNonNull(scanServerLockUUID, "Server uuid must be supplied");
     try (Scanner scanner = context.createScanner(tableName, 
Authorizations.EMPTY)) {
-      scanner.fetchColumn(new Text(serverAddress), new 
Text(scanServerLockUUID.toString()));
+      scanner.setRange(new Range(scanServerLockUUID.toString()));
 
       Set<ScanServerRefTabletFile> refsToDelete = 
StreamSupport.stream(scanner.spliterator(), false)
-          .map(e -> new 
ScanServerRefTabletFile(e.getKey().getRowData().toString(),
-              e.getKey().getColumnFamily(), e.getKey().getColumnQualifier()))
-          .collect(Collectors.toSet());
+          .map(e -> new 
ScanServerRefTabletFile(e.getKey())).collect(Collectors.toSet());
 
       if (!refsToDelete.isEmpty()) {
         this.delete(refsToDelete);
@@ -101,9 +95,7 @@ public class ScanServerRefStoreImpl implements 
ScanServerRefStore {
   public void delete(Collection<ScanServerRefTabletFile> refsToDelete) {
     try (BatchWriter writer = context.createBatchWriter(tableName)) {
       for (ScanServerRefTabletFile ref : refsToDelete) {
-        Mutation m = new Mutation(ref.getRow());
-        m.putDelete(ref.getServerAddress(), ref.getServerLockUUID());
-        writer.addMutation(m);
+        writer.addMutation(ref.putDeleteMutation());
       }
       log.debug("Deleted scan server file reference entries for files: {}", 
refsToDelete);
     } catch (MutationsRejectedException | TableNotFoundException e) {
diff --git 
a/server/base/src/main/java/org/apache/accumulo/server/util/ScanServerMetadataEntries.java
 
b/server/base/src/main/java/org/apache/accumulo/server/util/ScanServerMetadataEntries.java
index 71c2b0dd40..46a6b3e5cd 100644
--- 
a/server/base/src/main/java/org/apache/accumulo/server/util/ScanServerMetadataEntries.java
+++ 
b/server/base/src/main/java/org/apache/accumulo/server/util/ScanServerMetadataEntries.java
@@ -40,7 +40,7 @@ public class ScanServerMetadataEntries {
 
     // collect all uuids that are currently in the metadata table
     context.getAmple().scanServerRefs().list().forEach(ssrtf -> {
-      uuidsToDelete.add(UUID.fromString(ssrtf.getServerLockUUID().toString()));
+      uuidsToDelete.add(ssrtf.getServerLockUUID());
     });
 
     // gather the list of current live scan servers, its important that this 
is done after the above
@@ -57,7 +57,7 @@ public class ScanServerMetadataEntries {
 
       context.getAmple().scanServerRefs().list().forEach(ssrtf -> {
 
-        var uuid = UUID.fromString(ssrtf.getServerLockUUID().toString());
+        var uuid = ssrtf.getServerLockUUID();
 
         if (uuidsToDelete.contains(uuid)) {
           refsToDelete.add(ssrtf);
diff --git 
a/server/manager/src/main/java/org/apache/accumulo/manager/upgrade/Upgrader11to12.java
 
b/server/manager/src/main/java/org/apache/accumulo/manager/upgrade/Upgrader11to12.java
index ffeaf9a015..a130d11445 100644
--- 
a/server/manager/src/main/java/org/apache/accumulo/manager/upgrade/Upgrader11to12.java
+++ 
b/server/manager/src/main/java/org/apache/accumulo/manager/upgrade/Upgrader11to12.java
@@ -22,12 +22,16 @@ import static java.nio.charset.StandardCharsets.UTF_8;
 import static org.apache.accumulo.core.metadata.RootTable.ZROOT_TABLET;
 import static 
org.apache.accumulo.server.AccumuloDataVersion.METADATA_FILE_JSON_ENCODING;
 
+import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
 import java.util.Map;
 import java.util.Set;
 import java.util.TreeMap;
 
+import org.apache.accumulo.core.client.BatchDeleter;
 import org.apache.accumulo.core.client.BatchWriter;
 import org.apache.accumulo.core.client.IsolatedScanner;
 import org.apache.accumulo.core.client.MutationsRejectedException;
@@ -35,8 +39,10 @@ import org.apache.accumulo.core.client.Scanner;
 import org.apache.accumulo.core.client.TableNotFoundException;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Mutation;
+import org.apache.accumulo.core.data.Range;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.fate.zookeeper.ZooUtil;
+import org.apache.accumulo.core.metadata.AccumuloTable;
 import org.apache.accumulo.core.metadata.StoredTabletFile;
 import org.apache.accumulo.core.metadata.schema.Ample;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema;
@@ -47,6 +53,9 @@ import 
org.apache.accumulo.core.metadata.schema.RootTabletMetadata;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.util.TextUtil;
 import org.apache.accumulo.server.ServerContext;
+import org.apache.accumulo.server.init.FileSystemInitializer;
+import org.apache.accumulo.server.init.InitialConfiguration;
+import org.apache.accumulo.server.init.ZooKeeperInitializer;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.Text;
 import org.apache.zookeeper.KeeperException;
@@ -65,6 +74,9 @@ public class Upgrader11to12 implements Upgrader {
   @SuppressWarnings("deprecation")
   private static final Text CHOPPED = ChoppedColumnFamily.NAME;
 
+  public static final Collection<Range> OLD_SCAN_SERVERS_RANGES =
+      List.of(new Range("~sserv", "~sserx"), new Range("~scanref", 
"~scanreg"));
+
   @VisibleForTesting
   static final Set<Text> UPGRADE_FAMILIES =
       Set.of(DataFileColumnFamily.NAME, CHOPPED, 
ExternalCompactionColumnFamily.NAME);
@@ -131,6 +143,8 @@ public class Upgrader11to12 implements Upgrader {
     log.debug("Upgrade metadata: upgrading to data version {}", 
METADATA_FILE_JSON_ENCODING);
     var metaName = Ample.DataLevel.USER.metaTable();
     upgradeTabletsMetadata(context, metaName);
+    removeScanServerRange(context, metaName);
+    createScanServerRefTable(context);
   }
 
   private void upgradeTabletsMetadata(@NonNull ServerContext context, String 
metaName) {
@@ -216,4 +230,36 @@ public class Upgrader11to12 implements Upgrader {
     }
   }
 
+  public void removeScanServerRange(ServerContext context, String tableName) {
+    log.info("Removing Scan Server Range from table {}", tableName);
+    try (BatchDeleter batchDeleter =
+        context.createBatchDeleter(tableName, Authorizations.EMPTY, 4)) {
+      batchDeleter.setRanges(OLD_SCAN_SERVERS_RANGES);
+      batchDeleter.delete();
+    } catch (TableNotFoundException | MutationsRejectedException e) {
+      throw new RuntimeException(e);
+    }
+    log.info("Scan Server Ranges {} removed from table {}", 
OLD_SCAN_SERVERS_RANGES, tableName);
+  }
+
+  public void createScanServerRefTable(ServerContext context) {
+    ZooKeeperInitializer zkInit = new ZooKeeperInitializer();
+    zkInit.initScanRefTableState(context);
+
+    try {
+      FileSystemInitializer initializer = new FileSystemInitializer(
+          new InitialConfiguration(context.getHadoopConf(), 
context.getSiteConfiguration()));
+      FileSystemInitializer.InitialTablet scanRefTablet = 
initializer.createScanRefTablet(context);
+      // Add references to the Metadata Table
+      try (BatchWriter writer = 
context.createBatchWriter(AccumuloTable.METADATA.tableName())) {
+        writer.addMutation(scanRefTablet.createMutation());
+      } catch (MutationsRejectedException | TableNotFoundException e) {
+        log.error("Failed to write tablet refs to metadata table");
+        throw new RuntimeException(e);
+      }
+    } catch (IOException e) {
+      log.error("Problem attempting to create ScanServerRef table", e);
+    }
+    log.info("Created ScanServerRef table");
+  }
 }
diff --git 
a/server/tserver/src/main/java/org/apache/accumulo/tserver/ScanServer.java 
b/server/tserver/src/main/java/org/apache/accumulo/tserver/ScanServer.java
index 08c59fc2e3..91f6f9f055 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/ScanServer.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/ScanServer.java
@@ -630,8 +630,8 @@ public class ScanServer extends AbstractServer
 
       for (StoredTabletFile file : allFiles.keySet()) {
         if (!reservedFiles.containsKey(file)) {
-          refs.add(new ScanServerRefTabletFile(file.getNormalizedPathStr(), 
serverAddress,
-              serverLockUUID));
+          refs.add(new ScanServerRefTabletFile(serverLockUUID, serverAddress,
+              file.getNormalizedPathStr()));
           filesToReserve.add(file);
           tabletsToCheck.add(Objects.requireNonNull(allFiles.get(file)));
           LOG.trace("RFFS {} need to add scan ref for file {}", 
myReservationId, file);
diff --git 
a/test/src/main/java/org/apache/accumulo/test/upgrade/ScanServerUpgrade11to12TestIT.java
 
b/test/src/main/java/org/apache/accumulo/test/upgrade/ScanServerUpgrade11to12TestIT.java
new file mode 100644
index 0000000000..40efc8bc3c
--- /dev/null
+++ 
b/test/src/main/java/org/apache/accumulo/test/upgrade/ScanServerUpgrade11to12TestIT.java
@@ -0,0 +1,273 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   https://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.accumulo.test.upgrade;
+
+import static org.apache.accumulo.harness.AccumuloITBase.MINI_CLUSTER_ONLY;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+
+import java.io.IOException;
+import java.util.Map.Entry;
+import java.util.Set;
+import java.util.UUID;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import org.apache.accumulo.core.client.Accumulo;
+import org.apache.accumulo.core.client.AccumuloClient;
+import org.apache.accumulo.core.client.AccumuloException;
+import org.apache.accumulo.core.client.AccumuloSecurityException;
+import org.apache.accumulo.core.client.BatchScanner;
+import org.apache.accumulo.core.client.BatchWriter;
+import org.apache.accumulo.core.client.MutationsRejectedException;
+import org.apache.accumulo.core.client.Scanner;
+import org.apache.accumulo.core.client.TableNotFoundException;
+import org.apache.accumulo.core.conf.Property;
+import org.apache.accumulo.core.data.Key;
+import org.apache.accumulo.core.data.Mutation;
+import org.apache.accumulo.core.data.Range;
+import org.apache.accumulo.core.data.Value;
+import org.apache.accumulo.core.manager.state.tables.TableState;
+import org.apache.accumulo.core.metadata.AccumuloTable;
+import org.apache.accumulo.core.metadata.ScanServerRefTabletFile;
+import org.apache.accumulo.core.metadata.schema.Ample;
+import org.apache.accumulo.core.security.Authorizations;
+import org.apache.accumulo.harness.MiniClusterConfigurationCallback;
+import org.apache.accumulo.harness.SharedMiniClusterBase;
+import org.apache.accumulo.manager.upgrade.Upgrader11to12;
+import org.apache.accumulo.minicluster.ServerType;
+import org.apache.accumulo.miniclusterImpl.MiniAccumuloConfigImpl;
+import org.apache.accumulo.server.ServerContext;
+import org.apache.hadoop.io.Text;
+import org.apache.zookeeper.KeeperException;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Tag;
+import org.junit.jupiter.api.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.net.HostAndPort;
+
+@Tag(MINI_CLUSTER_ONLY)
+public class ScanServerUpgrade11to12TestIT extends SharedMiniClusterBase {
+
+  public static final Logger log = 
LoggerFactory.getLogger(ScanServerUpgrade11to12TestIT.class);
+  private static final Range META_RANGE =
+      new Range(AccumuloTable.SCAN_REF.tableId() + ";", 
AccumuloTable.SCAN_REF.tableId() + "<");
+
+  private static class ScanServerUpgradeITConfiguration
+      implements MiniClusterConfigurationCallback {
+    @Override
+    public void configureMiniCluster(MiniAccumuloConfigImpl cfg,
+        org.apache.hadoop.conf.Configuration coreSite) {
+      cfg.setNumScanServers(0);
+    }
+  }
+
+  @BeforeAll
+  public static void start() throws Exception {
+    ScanServerUpgradeITConfiguration c = new 
ScanServerUpgradeITConfiguration();
+    SharedMiniClusterBase.startMiniClusterWithConfig(c);
+  }
+
+  @AfterAll
+  public static void stop() throws Exception {
+    stopMiniCluster();
+  }
+
+  private Stream<Entry<Key,Value>> getOldScanServerRefs(String tableName) {
+    try {
+      BatchScanner scanner =
+          getCluster().getServerContext().createBatchScanner(tableName, 
Authorizations.EMPTY);
+      scanner.setRanges(Upgrader11to12.OLD_SCAN_SERVERS_RANGES);
+      return scanner.stream().onClose(scanner::close);
+    } catch (TableNotFoundException e) {
+      throw new IllegalStateException("Unable to find table " + tableName);
+    }
+  }
+
+  private void deleteScanServerRefTable() throws InterruptedException {
+    ServerContext ctx = getCluster().getServerContext();
+    // Remove the scan server table metadata in zk
+    try {
+      ctx.getTableManager().removeTable(AccumuloTable.SCAN_REF.tableId());
+    } catch (KeeperException | InterruptedException e) {
+      throw new RuntimeException("Removal of scan ref table failed" + e);
+    }
+
+    // Read from the metadata table to find any existing scan ref tablets and 
remove them
+    try (BatchWriter writer = 
ctx.createBatchWriter(AccumuloTable.METADATA.tableName())) {
+      var refTablet = checkForScanRefTablets().iterator();
+      while (refTablet.hasNext()) {
+        var entry = refTablet.next();
+        log.info("Entry:  {}", entry);
+        var mutation = new Mutation(entry.getKey().getRow());
+        mutation.putDelete(entry.getKey().getColumnFamily(), 
entry.getKey().getColumnQualifier());
+        writer.addMutation(mutation);
+      }
+      writer.flush();
+    } catch (TableNotFoundException | MutationsRejectedException e) {
+      throw new RuntimeException(e);
+    }
+
+    // Compact the metadata table to remove the tablet file for the scan ref 
table
+    try (AccumuloClient client = 
Accumulo.newClient().from(getClientProps()).build()) {
+      client.tableOperations().compact(AccumuloTable.METADATA.tableName(), 
null, null, true, true);
+    } catch (TableNotFoundException | AccumuloException | 
AccumuloSecurityException e) {
+      log.error("Failed to compact metadata table");
+      throw new RuntimeException(e);
+    }
+
+    log.info("Scan ref table is deleted, now shutting down the system");
+    try {
+      getCluster().getClusterControl().stop(ServerType.MANAGER);
+      
getCluster().getClusterControl().stopAllServers(ServerType.TABLET_SERVER);
+    } catch (IOException e) {
+      log.info("Failed to stop cluster");
+    }
+    Thread.sleep(60_000);
+  }
+
+  private void testMetadataScanServerRefRemoval(String tableName) {
+
+    HostAndPort server = HostAndPort.fromParts("127.0.0.1", 1234);
+    UUID serverLockUUID = UUID.randomUUID();
+
+    Set<ScanServerRefTabletFile> scanRefs = Stream.of("F0000070.rf", 
"F0000071.rf", "F0000072.rf")
+        .map(f -> "hdfs://localhost:8020/accumulo/tables/2a/default_tablet/" + 
f)
+        .map(f -> new ScanServerRefTabletFile(serverLockUUID, 
server.toString(), f))
+        .collect(Collectors.toSet());
+
+    ServerContext ctx = getCluster().getServerContext();
+    try (AccumuloClient client = 
Accumulo.newClient().from(getClientProps()).build()) {
+      client.tableOperations().removeProperty(tableName,
+          Property.TABLE_CONSTRAINT_PREFIX.getKey() + "1");
+      log.info("Removed constraints from table {}", tableName);
+      Thread.sleep(10_000);
+    } catch (AccumuloException | AccumuloSecurityException | 
InterruptedException e) {
+      throw new RuntimeException(e);
+    }
+
+    try (BatchWriter writer = ctx.createBatchWriter(tableName)) {
+      for (ScanServerRefTabletFile ref : scanRefs) {
+        Mutation sservMutation = new Mutation("~sserv" + ref.getFilePath());
+        sservMutation.put(ref.getServerAddress(), new 
Text(ref.getServerLockUUID().toString()),
+            new Value(""));
+        writer.addMutation(sservMutation);
+
+        Mutation scanRefMutation = new Mutation("~scanref" + 
ref.getServerLockUUID().toString());
+        scanRefMutation.put(ref.getServerAddress(), ref.getFilePath(), new 
Value(""));
+        writer.addMutation(scanRefMutation);
+      }
+      writer.flush();
+    } catch (TableNotFoundException | MutationsRejectedException e) {
+      log.warn("Failed to write mutations to metadata table");
+      throw new RuntimeException(e);
+    }
+
+    // Check that ample cannot find these scan server refs
+    assertEquals(0, ctx.getAmple().scanServerRefs().list().count());
+
+    // Ensure they exist on the metadata table
+    assertEquals(scanRefs.size() * 2L, 
getOldScanServerRefs(tableName).count());
+
+    var upgrader = new Upgrader11to12();
+    upgrader.removeScanServerRange(ctx, tableName);
+
+    // Ensure entries are now removed from the metadata table
+    assertEquals(0, getOldScanServerRefs(tableName).count());
+  }
+
+  private Stream<Entry<Key,Value>> checkForScanRefTablets() {
+    try {
+      Scanner scanner = getCluster().getServerContext()
+          .createScanner(AccumuloTable.METADATA.tableName(), 
Authorizations.EMPTY);
+      scanner.setRange(META_RANGE);
+      return scanner.stream().onClose(scanner::close);
+    } catch (TableNotFoundException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  @Test
+  public void testScanRefTableCreation() throws InterruptedException {
+    ServerContext ctx = getCluster().getServerContext();
+    deleteScanServerRefTable();
+    log.info("Attempt to start the system");
+    try {
+      
getCluster().getClusterControl().startAllServers(ServerType.TABLET_SERVER);
+      getCluster().getClusterControl().start(ServerType.MANAGER);
+      Thread.sleep(10_000);
+    } catch (IOException e) {
+      log.info("Failed to start cluster");
+    } catch (InterruptedException e) {
+      throw new RuntimeException(e);
+    }
+
+    log.info("Attempting creation of the scan ref table");
+    var upgrader = new Upgrader11to12();
+    upgrader.createScanServerRefTable(ctx);
+    assertEquals(TableState.ONLINE,
+        ctx.getTableManager().getTableState(AccumuloTable.SCAN_REF.tableId()));
+
+    while (checkForScanRefTablets().count() < 4) {
+      log.info("Waiting for the table to be hosted");
+      Thread.sleep(1_000);
+    }
+
+    log.info("Reading entries from the metadata table");
+    try (Scanner scanner = getCluster().getServerContext()
+        .createScanner(AccumuloTable.METADATA.tableName(), 
Authorizations.EMPTY)) {
+      var refTablet = scanner.stream().iterator();
+      while (refTablet.hasNext()) {
+        log.info("Metadata Entry: {}", refTablet.next());
+      }
+    } catch (TableNotFoundException e) {
+      throw new RuntimeException(e);
+    }
+
+    log.info("Reading entries from the root table");
+    try (Scanner scanner = getCluster().getServerContext()
+        .createScanner(AccumuloTable.ROOT.tableName(), Authorizations.EMPTY)) {
+      var refTablet = scanner.stream().iterator();
+      while (refTablet.hasNext()) {
+        log.info("Root Entry: {}", refTablet.next());
+      }
+    } catch (TableNotFoundException e) {
+      throw new RuntimeException(e);
+    }
+    // Create some scanRefs to test table functionality
+    assertEquals(0, ctx.getAmple().scanServerRefs().list().count());
+    HostAndPort server = HostAndPort.fromParts("127.0.0.1", 1234);
+    UUID serverLockUUID = UUID.randomUUID();
+
+    Set<ScanServerRefTabletFile> scanRefs = Stream.of("F0000070.rf", 
"F0000071.rf")
+        .map(f -> "hdfs://localhost:8020/accumulo/tables/2a/default_tablet/" + 
f)
+        .map(f -> new ScanServerRefTabletFile(f, server.toString(), 
serverLockUUID))
+        .collect(Collectors.toSet());
+    ctx.getAmple().scanServerRefs().put(scanRefs);
+    assertEquals(2, ctx.getAmple().scanServerRefs().list().count());
+  }
+
+  @Test
+  public void testMetadataScanServerRefs() {
+    testMetadataScanServerRefRemoval(Ample.DataLevel.USER.metaTable());
+  }
+
+}


Reply via email to