keith-turner commented on code in PR #3876:
URL: https://github.com/apache/accumulo/pull/3876#discussion_r1370482709


##########
server/manager/src/main/java/org/apache/accumulo/manager/upgrade/Upgrader11to12.java:
##########
@@ -0,0 +1,208 @@
+/*
+ * 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.manager.upgrade;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.apache.accumulo.core.metadata.RootTable.ZROOT_TABLET;
+import static 
org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.DataFileColumnFamily;
+import static 
org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.ExternalCompactionColumnFamily;
+import static 
org.apache.accumulo.core.metadata.schema.UpgraderDeprecatedConstants.ChoppedColumnFamily;
+import static 
org.apache.accumulo.server.AccumuloDataVersion.METADATA_FILE_JSON_ENCODING;
+
+import org.apache.accumulo.core.client.Accumulo;
+import org.apache.accumulo.core.client.AccumuloClient;
+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.data.Key;
+import org.apache.accumulo.core.data.Mutation;
+import org.apache.accumulo.core.data.Value;
+import org.apache.accumulo.core.fate.zookeeper.ZooUtil;
+import org.apache.accumulo.core.metadata.StoredTabletFile;
+import org.apache.accumulo.core.metadata.schema.Ample;
+import org.apache.accumulo.core.metadata.schema.RootTabletMetadata;
+import org.apache.accumulo.core.security.Authorizations;
+import org.apache.accumulo.server.ServerContext;
+import org.apache.hadoop.fs.Path;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.data.Stat;
+import org.checkerframework.checker.nullness.qual.NonNull;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.gson.JsonSyntaxException;
+
+public class Upgrader11to12 implements Upgrader {
+
+  private static final Logger log = 
LoggerFactory.getLogger(Upgrader11to12.class);
+
+  @Override
+  public void upgradeZookeeper(@NonNull ServerContext context) {
+    log.debug("Upgrade ZooKeeper: upgrading to data version {}", 
METADATA_FILE_JSON_ENCODING);
+    var rootBase = ZooUtil.getRoot(context.getInstanceID()) + ZROOT_TABLET;
+
+    try {
+      var zrw = context.getZooReaderWriter();
+      Stat stat = new Stat();
+      byte[] rootData = zrw.getData(rootBase, stat);
+
+      String json = new String(rootData, UTF_8);
+      if (RootTabletMetadata.Data.needsConversion(json)) {
+        RootTabletMetadata rtm = RootTabletMetadata.upgrade(json);
+        zrw.overwritePersistentData(rootBase, rtm.toJson().getBytes(UTF_8), 
stat.getVersion());
+      }
+    } catch (InterruptedException ex) {
+      Thread.currentThread().interrupt();
+      throw new IllegalStateException(
+          "Could not read root metadata from ZooKeeper due to interrupt", ex);
+    } catch (KeeperException ex) {
+      throw new IllegalStateException(
+          "Could not read root metadata from ZooKeeper because of ZooKeeper 
exception", ex);
+    }
+  }
+
+  @Override
+  public void upgradeRoot(@NonNull ServerContext context) {
+    log.debug("Upgrade root: upgrading to data version {}", 
METADATA_FILE_JSON_ENCODING);
+    var rootName = Ample.DataLevel.METADATA.metaTable();
+    processReferences(context, rootName);
+  }
+
+  @Override
+  public void upgradeMetadata(@NonNull ServerContext context) {
+    log.debug("Upgrade metadata: upgrading to data version {}", 
METADATA_FILE_JSON_ENCODING);
+    var metaName = Ample.DataLevel.USER.metaTable();
+    processReferences(context, metaName);
+  }
+
+  private void processReferences(ServerContext context, String tableName) {
+    // not using ample to avoid StoredTabletFile because old file ref is 
incompatible
+    try (AccumuloClient c = 
Accumulo.newClient().from(context.getProperties()).build();
+        BatchWriter batchWriter = c.createBatchWriter(tableName);
+        Scanner scanner = context.createScanner(tableName, 
Authorizations.EMPTY)) {
+
+      scanner.fetchColumnFamily(DataFileColumnFamily.NAME);
+      scanner.fetchColumnFamily(ChoppedColumnFamily.NAME);
+      scanner.fetchColumnFamily(ExternalCompactionColumnFamily.NAME);
+      scanner.forEach((k, v) -> {
+        var family = k.getColumnFamily();
+        if (family.equals(DataFileColumnFamily.NAME)) {
+          upgradeDataFileCF(k, v, batchWriter, tableName);
+        } else if (family.equals(ChoppedColumnFamily.NAME)) {
+          removeChoppedCF(k, batchWriter, tableName);
+        } else if (family.equals(ExternalCompactionColumnFamily.NAME)) {
+          removeExternalCompactionCF(k, batchWriter, tableName);
+        } else {
+          log.warn("Received unexpected column family processing references: " 
+ family);
+        }
+      });
+    } catch (MutationsRejectedException mex) {
+      log.warn("Failed to update reference for table: " + tableName);
+      log.warn("Constraint violations: {}", 
mex.getConstraintViolationSummaries());
+      throw new IllegalStateException("Failed to process table: " + tableName, 
mex);
+    } catch (Exception ex) {
+      throw new IllegalStateException("Failed to process table: " + tableName, 
ex);
+    }
+  }
+
+  @VisibleForTesting
+  void upgradeDataFileCF(final Key key, final Value value, final BatchWriter 
batchWriter,
+      final String tableName) {
+    String file = key.getColumnQualifier().toString();
+    // filter out references if they are in the correct format already.
+    if (fileNeedsConversion(file)) {
+      var fileJson = StoredTabletFile.of(new Path(file)).getMetadataText();
+      try {
+        Mutation update = new Mutation(key.getRow());
+        
update.at().family(DataFileColumnFamily.STR_NAME).qualifier(fileJson).put(value);
+        log.trace("table: {}, adding: {}", tableName, update.prettyPrint());
+        batchWriter.addMutation(update);
+
+        Mutation delete = new Mutation(key.getRow());
+        
delete.at().family(DataFileColumnFamily.STR_NAME).qualifier(file).delete();
+        log.trace("table {}: deleting: {}", tableName, delete.prettyPrint());
+        batchWriter.addMutation(delete);
+      } catch (MutationsRejectedException ex) {
+        // include constraint violation info in log - but stop upgrade
+        log.warn(
+            "Failed to update file reference for table: " + tableName + ". 
row: " + key.getRow());
+        log.warn("Constraint violations: {}", 
ex.getConstraintViolationSummaries());
+        throw new IllegalStateException("File conversion failed. Aborting 
upgrade", ex);
+      }
+    }
+  }
+
+  @VisibleForTesting
+  void removeChoppedCF(final Key key, final BatchWriter batchWriter, final 
String tableName) {
+    Mutation delete = null;
+    try {
+      delete = new 
Mutation(key.getRow()).at().family(ChoppedColumnFamily.STR_NAME)
+          .qualifier(ChoppedColumnFamily.STR_NAME).delete();
+      log.warn(
+          "Deleting chopped reference from:{}. Previous split or delete may 
not have completed cleanly. Ref: {}",
+          tableName, delete.prettyPrint());
+      batchWriter.addMutation(delete);
+    } catch (MutationsRejectedException ex) {
+      log.warn("Failed to delete obsolete chopped CF reference for table: " + 
tableName + ". Ref: "
+          + delete.prettyPrint() + ". Will try to continue. Ref may need to be 
manually removed");
+      log.warn("Constraint violations: {}", 
ex.getConstraintViolationSummaries());
+    }
+  }
+
+  @VisibleForTesting
+  void removeExternalCompactionCF(final Key key, final BatchWriter batchWriter,
+      final String tableName) {
+    Mutation delete = null;
+    try {
+      delete = new 
Mutation(key.getRow()).at().family(ExternalCompactionColumnFamily.NAME)
+          .qualifier(key.getColumnQualifier()).delete();
+      log.warn(

Review Comment:
   Its not unexpected rf problematic to have these external compactions, so the 
warn level does not seem necessary.
   
   ```suggestion
         log.debug(
   ```



##########
server/manager/src/main/java/org/apache/accumulo/manager/upgrade/Upgrader11to12.java:
##########
@@ -0,0 +1,208 @@
+/*
+ * 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.manager.upgrade;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.apache.accumulo.core.metadata.RootTable.ZROOT_TABLET;
+import static 
org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.DataFileColumnFamily;
+import static 
org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.ExternalCompactionColumnFamily;
+import static 
org.apache.accumulo.core.metadata.schema.UpgraderDeprecatedConstants.ChoppedColumnFamily;
+import static 
org.apache.accumulo.server.AccumuloDataVersion.METADATA_FILE_JSON_ENCODING;
+
+import org.apache.accumulo.core.client.Accumulo;
+import org.apache.accumulo.core.client.AccumuloClient;
+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.data.Key;
+import org.apache.accumulo.core.data.Mutation;
+import org.apache.accumulo.core.data.Value;
+import org.apache.accumulo.core.fate.zookeeper.ZooUtil;
+import org.apache.accumulo.core.metadata.StoredTabletFile;
+import org.apache.accumulo.core.metadata.schema.Ample;
+import org.apache.accumulo.core.metadata.schema.RootTabletMetadata;
+import org.apache.accumulo.core.security.Authorizations;
+import org.apache.accumulo.server.ServerContext;
+import org.apache.hadoop.fs.Path;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.data.Stat;
+import org.checkerframework.checker.nullness.qual.NonNull;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.gson.JsonSyntaxException;
+
+public class Upgrader11to12 implements Upgrader {
+
+  private static final Logger log = 
LoggerFactory.getLogger(Upgrader11to12.class);
+
+  @Override
+  public void upgradeZookeeper(@NonNull ServerContext context) {
+    log.debug("Upgrade ZooKeeper: upgrading to data version {}", 
METADATA_FILE_JSON_ENCODING);
+    var rootBase = ZooUtil.getRoot(context.getInstanceID()) + ZROOT_TABLET;
+
+    try {
+      var zrw = context.getZooReaderWriter();
+      Stat stat = new Stat();
+      byte[] rootData = zrw.getData(rootBase, stat);
+
+      String json = new String(rootData, UTF_8);
+      if (RootTabletMetadata.Data.needsConversion(json)) {
+        RootTabletMetadata rtm = RootTabletMetadata.upgrade(json);
+        zrw.overwritePersistentData(rootBase, rtm.toJson().getBytes(UTF_8), 
stat.getVersion());

Review Comment:
   Since the root tablet metadata is so important, may be good to log the root 
tablet metadata before and after.



##########
server/manager/src/main/java/org/apache/accumulo/manager/upgrade/Upgrader11to12.java:
##########
@@ -0,0 +1,208 @@
+/*
+ * 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.manager.upgrade;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.apache.accumulo.core.metadata.RootTable.ZROOT_TABLET;
+import static 
org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.DataFileColumnFamily;
+import static 
org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.ExternalCompactionColumnFamily;
+import static 
org.apache.accumulo.core.metadata.schema.UpgraderDeprecatedConstants.ChoppedColumnFamily;
+import static 
org.apache.accumulo.server.AccumuloDataVersion.METADATA_FILE_JSON_ENCODING;
+
+import org.apache.accumulo.core.client.Accumulo;
+import org.apache.accumulo.core.client.AccumuloClient;
+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.data.Key;
+import org.apache.accumulo.core.data.Mutation;
+import org.apache.accumulo.core.data.Value;
+import org.apache.accumulo.core.fate.zookeeper.ZooUtil;
+import org.apache.accumulo.core.metadata.StoredTabletFile;
+import org.apache.accumulo.core.metadata.schema.Ample;
+import org.apache.accumulo.core.metadata.schema.RootTabletMetadata;
+import org.apache.accumulo.core.security.Authorizations;
+import org.apache.accumulo.server.ServerContext;
+import org.apache.hadoop.fs.Path;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.data.Stat;
+import org.checkerframework.checker.nullness.qual.NonNull;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.gson.JsonSyntaxException;
+
+public class Upgrader11to12 implements Upgrader {
+
+  private static final Logger log = 
LoggerFactory.getLogger(Upgrader11to12.class);
+
+  @Override
+  public void upgradeZookeeper(@NonNull ServerContext context) {
+    log.debug("Upgrade ZooKeeper: upgrading to data version {}", 
METADATA_FILE_JSON_ENCODING);
+    var rootBase = ZooUtil.getRoot(context.getInstanceID()) + ZROOT_TABLET;
+
+    try {
+      var zrw = context.getZooReaderWriter();
+      Stat stat = new Stat();
+      byte[] rootData = zrw.getData(rootBase, stat);
+
+      String json = new String(rootData, UTF_8);
+      if (RootTabletMetadata.Data.needsConversion(json)) {
+        RootTabletMetadata rtm = RootTabletMetadata.upgrade(json);
+        zrw.overwritePersistentData(rootBase, rtm.toJson().getBytes(UTF_8), 
stat.getVersion());
+      }
+    } catch (InterruptedException ex) {
+      Thread.currentThread().interrupt();
+      throw new IllegalStateException(
+          "Could not read root metadata from ZooKeeper due to interrupt", ex);
+    } catch (KeeperException ex) {
+      throw new IllegalStateException(
+          "Could not read root metadata from ZooKeeper because of ZooKeeper 
exception", ex);
+    }
+  }
+
+  @Override
+  public void upgradeRoot(@NonNull ServerContext context) {
+    log.debug("Upgrade root: upgrading to data version {}", 
METADATA_FILE_JSON_ENCODING);
+    var rootName = Ample.DataLevel.METADATA.metaTable();
+    processReferences(context, rootName);
+  }
+
+  @Override
+  public void upgradeMetadata(@NonNull ServerContext context) {
+    log.debug("Upgrade metadata: upgrading to data version {}", 
METADATA_FILE_JSON_ENCODING);
+    var metaName = Ample.DataLevel.USER.metaTable();
+    processReferences(context, metaName);
+  }
+
+  private void processReferences(ServerContext context, String tableName) {
+    // not using ample to avoid StoredTabletFile because old file ref is 
incompatible
+    try (AccumuloClient c = 
Accumulo.newClient().from(context.getProperties()).build();
+        BatchWriter batchWriter = c.createBatchWriter(tableName);
+        Scanner scanner = context.createScanner(tableName, 
Authorizations.EMPTY)) {
+
+      scanner.fetchColumnFamily(DataFileColumnFamily.NAME);
+      scanner.fetchColumnFamily(ChoppedColumnFamily.NAME);
+      scanner.fetchColumnFamily(ExternalCompactionColumnFamily.NAME);
+      scanner.forEach((k, v) -> {
+        var family = k.getColumnFamily();
+        if (family.equals(DataFileColumnFamily.NAME)) {
+          upgradeDataFileCF(k, v, batchWriter, tableName);
+        } else if (family.equals(ChoppedColumnFamily.NAME)) {
+          removeChoppedCF(k, batchWriter, tableName);
+        } else if (family.equals(ExternalCompactionColumnFamily.NAME)) {
+          removeExternalCompactionCF(k, batchWriter, tableName);
+        } else {
+          log.warn("Received unexpected column family processing references: " 
+ family);

Review Comment:
   I would throw an exception here instead of logging a warning because its 
completely unexpected.



##########
server/manager/src/main/java/org/apache/accumulo/manager/upgrade/Upgrader11to12.java:
##########
@@ -0,0 +1,208 @@
+/*
+ * 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.manager.upgrade;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.apache.accumulo.core.metadata.RootTable.ZROOT_TABLET;
+import static 
org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.DataFileColumnFamily;
+import static 
org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.ExternalCompactionColumnFamily;
+import static 
org.apache.accumulo.core.metadata.schema.UpgraderDeprecatedConstants.ChoppedColumnFamily;
+import static 
org.apache.accumulo.server.AccumuloDataVersion.METADATA_FILE_JSON_ENCODING;
+
+import org.apache.accumulo.core.client.Accumulo;
+import org.apache.accumulo.core.client.AccumuloClient;
+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.data.Key;
+import org.apache.accumulo.core.data.Mutation;
+import org.apache.accumulo.core.data.Value;
+import org.apache.accumulo.core.fate.zookeeper.ZooUtil;
+import org.apache.accumulo.core.metadata.StoredTabletFile;
+import org.apache.accumulo.core.metadata.schema.Ample;
+import org.apache.accumulo.core.metadata.schema.RootTabletMetadata;
+import org.apache.accumulo.core.security.Authorizations;
+import org.apache.accumulo.server.ServerContext;
+import org.apache.hadoop.fs.Path;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.data.Stat;
+import org.checkerframework.checker.nullness.qual.NonNull;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.gson.JsonSyntaxException;
+
+public class Upgrader11to12 implements Upgrader {
+
+  private static final Logger log = 
LoggerFactory.getLogger(Upgrader11to12.class);
+
+  @Override
+  public void upgradeZookeeper(@NonNull ServerContext context) {
+    log.debug("Upgrade ZooKeeper: upgrading to data version {}", 
METADATA_FILE_JSON_ENCODING);
+    var rootBase = ZooUtil.getRoot(context.getInstanceID()) + ZROOT_TABLET;
+
+    try {
+      var zrw = context.getZooReaderWriter();
+      Stat stat = new Stat();
+      byte[] rootData = zrw.getData(rootBase, stat);
+
+      String json = new String(rootData, UTF_8);
+      if (RootTabletMetadata.Data.needsConversion(json)) {
+        RootTabletMetadata rtm = RootTabletMetadata.upgrade(json);
+        zrw.overwritePersistentData(rootBase, rtm.toJson().getBytes(UTF_8), 
stat.getVersion());
+      }
+    } catch (InterruptedException ex) {
+      Thread.currentThread().interrupt();
+      throw new IllegalStateException(
+          "Could not read root metadata from ZooKeeper due to interrupt", ex);
+    } catch (KeeperException ex) {
+      throw new IllegalStateException(
+          "Could not read root metadata from ZooKeeper because of ZooKeeper 
exception", ex);
+    }
+  }
+
+  @Override
+  public void upgradeRoot(@NonNull ServerContext context) {
+    log.debug("Upgrade root: upgrading to data version {}", 
METADATA_FILE_JSON_ENCODING);
+    var rootName = Ample.DataLevel.METADATA.metaTable();
+    processReferences(context, rootName);
+  }
+
+  @Override
+  public void upgradeMetadata(@NonNull ServerContext context) {
+    log.debug("Upgrade metadata: upgrading to data version {}", 
METADATA_FILE_JSON_ENCODING);
+    var metaName = Ample.DataLevel.USER.metaTable();
+    processReferences(context, metaName);
+  }
+
+  private void processReferences(ServerContext context, String tableName) {
+    // not using ample to avoid StoredTabletFile because old file ref is 
incompatible
+    try (AccumuloClient c = 
Accumulo.newClient().from(context.getProperties()).build();
+        BatchWriter batchWriter = c.createBatchWriter(tableName);
+        Scanner scanner = context.createScanner(tableName, 
Authorizations.EMPTY)) {
+
+      scanner.fetchColumnFamily(DataFileColumnFamily.NAME);
+      scanner.fetchColumnFamily(ChoppedColumnFamily.NAME);
+      scanner.fetchColumnFamily(ExternalCompactionColumnFamily.NAME);
+      scanner.forEach((k, v) -> {
+        var family = k.getColumnFamily();
+        if (family.equals(DataFileColumnFamily.NAME)) {
+          upgradeDataFileCF(k, v, batchWriter, tableName);
+        } else if (family.equals(ChoppedColumnFamily.NAME)) {
+          removeChoppedCF(k, batchWriter, tableName);
+        } else if (family.equals(ExternalCompactionColumnFamily.NAME)) {
+          removeExternalCompactionCF(k, batchWriter, tableName);
+        } else {
+          log.warn("Received unexpected column family processing references: " 
+ family);
+        }
+      });
+    } catch (MutationsRejectedException mex) {
+      log.warn("Failed to update reference for table: " + tableName);
+      log.warn("Constraint violations: {}", 
mex.getConstraintViolationSummaries());
+      throw new IllegalStateException("Failed to process table: " + tableName, 
mex);
+    } catch (Exception ex) {
+      throw new IllegalStateException("Failed to process table: " + tableName, 
ex);
+    }
+  }
+
+  @VisibleForTesting
+  void upgradeDataFileCF(final Key key, final Value value, final BatchWriter 
batchWriter,
+      final String tableName) {
+    String file = key.getColumnQualifier().toString();
+    // filter out references if they are in the correct format already.
+    if (fileNeedsConversion(file)) {
+      var fileJson = StoredTabletFile.of(new Path(file)).getMetadataText();
+      try {
+        Mutation update = new Mutation(key.getRow());
+        
update.at().family(DataFileColumnFamily.STR_NAME).qualifier(fileJson).put(value);
+        log.trace("table: {}, adding: {}", tableName, update.prettyPrint());
+        batchWriter.addMutation(update);
+
+        Mutation delete = new Mutation(key.getRow());
+        
delete.at().family(DataFileColumnFamily.STR_NAME).qualifier(file).delete();
+        log.trace("table {}: deleting: {}", tableName, delete.prettyPrint());
+        batchWriter.addMutation(delete);
+      } catch (MutationsRejectedException ex) {
+        // include constraint violation info in log - but stop upgrade
+        log.warn(
+            "Failed to update file reference for table: " + tableName + ". 
row: " + key.getRow());
+        log.warn("Constraint violations: {}", 
ex.getConstraintViolationSummaries());
+        throw new IllegalStateException("File conversion failed. Aborting 
upgrade", ex);
+      }
+    }
+  }
+
+  @VisibleForTesting
+  void removeChoppedCF(final Key key, final BatchWriter batchWriter, final 
String tableName) {
+    Mutation delete = null;
+    try {
+      delete = new 
Mutation(key.getRow()).at().family(ChoppedColumnFamily.STR_NAME)
+          .qualifier(ChoppedColumnFamily.STR_NAME).delete();
+      log.warn(
+          "Deleting chopped reference from:{}. Previous split or delete may 
not have completed cleanly. Ref: {}",
+          tableName, delete.prettyPrint());
+      batchWriter.addMutation(delete);
+    } catch (MutationsRejectedException ex) {
+      log.warn("Failed to delete obsolete chopped CF reference for table: " + 
tableName + ". Ref: "
+          + delete.prettyPrint() + ". Will try to continue. Ref may need to be 
manually removed");
+      log.warn("Constraint violations: {}", 
ex.getConstraintViolationSummaries());
+    }
+  }
+
+  @VisibleForTesting
+  void removeExternalCompactionCF(final Key key, final BatchWriter batchWriter,
+      final String tableName) {
+    Mutation delete = null;
+    try {
+      delete = new 
Mutation(key.getRow()).at().family(ExternalCompactionColumnFamily.NAME)
+          .qualifier(key.getColumnQualifier()).delete();
+      log.warn(
+          "Deleting external compaction reference from:{}. Previous compaction 
may not have completed. Ref: {}",
+          tableName, delete.prettyPrint());
+      batchWriter.addMutation(delete);
+    } catch (MutationsRejectedException ex) {
+      log.warn("Failed to delete obsolete external compaction CF reference for 
table: " + tableName

Review Comment:
   Also need to throw an exception here.



##########
core/src/main/java/org/apache/accumulo/core/metadata/schema/RootTabletMetadata.java:
##########
@@ -73,10 +77,20 @@ private static class Data {
      */
     private final TreeMap<String,TreeMap<String,String>> columnValues;
 
-    public Data(int version, TreeMap<String,TreeMap<String,String>> 
columnValues) {
+    private Data(int version, TreeMap<String,TreeMap<String,String>> 
columnValues) {
       this.version = version;
       this.columnValues = columnValues;
     }
+
+    public int getVersion() {
+      return version;
+    }
+
+    public static boolean needsConversion(final String json) {

Review Comment:
   ```suggestion
       public static boolean needsUpgrade(final String json) {
   ```



##########
core/src/test/java/org/apache/accumulo/core/metadata/schema/RootTabletMetadataTest.java:
##########
@@ -0,0 +1,66 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   https://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.accumulo.core.metadata.schema;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+import org.apache.accumulo.core.metadata.StoredTabletFile;
+import org.apache.hadoop.fs.Path;
+import org.junit.jupiter.api.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class RootTabletMetadataTest {

Review Comment:
   Could add the following test also
   
    * Pass version 5 data and ensure an exception is thrown
    * Pass version 2 data to upgrade and ensure it behaves as expected



##########
server/manager/src/main/java/org/apache/accumulo/manager/upgrade/Upgrader11to12.java:
##########
@@ -0,0 +1,259 @@
+/*
+ * 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.manager.upgrade;
+
+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.util.Map;
+import java.util.TreeMap;
+
+import org.apache.accumulo.core.client.Accumulo;
+import org.apache.accumulo.core.client.AccumuloClient;
+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.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.dataImpl.KeyExtent;
+import org.apache.accumulo.core.fate.zookeeper.ZooUtil;
+import org.apache.accumulo.core.metadata.StoredTabletFile;
+import org.apache.accumulo.core.metadata.schema.Ample;
+import org.apache.accumulo.core.metadata.schema.MetadataSchema;
+import org.apache.accumulo.core.metadata.schema.RootTabletMetadata;
+import org.apache.accumulo.core.metadata.schema.UpgraderDeprecatedConstants;
+import org.apache.accumulo.core.security.Authorizations;
+import org.apache.accumulo.core.util.ComparablePair;
+import org.apache.accumulo.server.ServerContext;
+import org.apache.hadoop.io.Text;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.data.Stat;
+import org.checkerframework.checker.nullness.qual.NonNull;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.gson.JsonSyntaxException;
+
+public class Upgrader11to12 implements Upgrader {
+
+  private static final Logger log = 
LoggerFactory.getLogger(Upgrader11to12.class);
+
+  @Override
+  public void upgradeZookeeper(@NonNull ServerContext context) {
+    log.debug("Upgrade ZooKeeper: upgrading to data version {}", 
METADATA_FILE_JSON_ENCODING);
+    var rootBase = ZooUtil.getRoot(context.getInstanceID()) + ZROOT_TABLET;
+
+    try {
+      var zrw = context.getZooReaderWriter();
+      Stat stat = new Stat();
+      byte[] rootData = zrw.getData(rootBase, stat);
+
+      String json = new String(rootData, UTF_8);
+      if (RootTabletMetadata.Data.needsConversion(json)) {
+        RootTabletMetadata rtm = RootTabletMetadata.upgrade(json);
+        zrw.overwritePersistentData(rootBase, rtm.toJson().getBytes(UTF_8), 
stat.getVersion());
+      }
+    } catch (InterruptedException ex) {
+      Thread.currentThread().interrupt();
+      throw new IllegalStateException(
+          "Could not read root metadata from ZooKeeper due to interrupt", ex);
+    } catch (KeeperException ex) {
+      throw new IllegalStateException(
+          "Could not read root metadata from ZooKeeper because of ZooKeeper 
exception", ex);
+    }
+  }
+
+  @Override
+  public void upgradeRoot(@NonNull ServerContext context) {
+    log.debug("Upgrade root: upgrading to data version {}", 
METADATA_FILE_JSON_ENCODING);
+
+    var rootName = Ample.DataLevel.METADATA.metaTable();
+    var filesToConvert = getFileReferences(context, rootName);
+    convertFileReferences(context, rootName, filesToConvert);
+    deleteObsoleteReferences(context, rootName);
+  }
+
+  @Override
+  public void upgradeMetadata(@NonNull ServerContext context) {
+    log.debug("Upgrade metadata: upgrading to data version {}", 
METADATA_FILE_JSON_ENCODING);
+
+    var metaName = Ample.DataLevel.USER.metaTable();
+    var filesToConvert = getFileReferences(context, metaName);
+    convertFileReferences(context, metaName, filesToConvert);
+    deleteObsoleteReferences(context, metaName);
+  }
+
+  private Map<ComparablePair<KeyExtent,String>,Value>
+      getFileReferences(@NonNull final ServerContext context, @NonNull final 
String tableName) {
+
+    log.trace("gather file references for table: {}", tableName);
+
+    Map<ComparablePair<KeyExtent,String>,Value> filesToConvert = new 
TreeMap<>();
+
+    try (Scanner scanner = context.createScanner(tableName, 
Authorizations.EMPTY)) {
+      
scanner.fetchColumnFamily(MetadataSchema.TabletsSection.DataFileColumnFamily.STR_NAME);
+      scanner.forEach((k, v) -> {
+        KeyExtent ke = KeyExtent.fromMetaRow(k.getRow());
+        String file = k.getColumnQualifier().toString();
+        // filter out references that are in the correct format already.
+        if (fileNeedsConversion(file)) {
+          var prev = filesToConvert.put(new ComparablePair<>(ke, file), v);
+          if (prev != null) {
+            throw new IllegalStateException(
+                "upgrade for table: " + tableName + " aborted, would have 
missed: " + prev);
+          }
+        }
+      });
+    } catch (TableNotFoundException ex) {
+      throw new IllegalStateException("failed to read metadata table for 
upgrade", ex);
+    }
+    log.debug("Number of files to convert for table: {}, number of files: {}", 
tableName,
+        filesToConvert.size());
+    return filesToConvert;
+  }
+
+  private void convertFileReferences(final ServerContext context, final String 
tableName,
+      Map<ComparablePair<KeyExtent,String>,Value> filesToConvert) {
+
+    // not using ample to avoid StoredTabletFile because old file ref is 
incompatible
+    try (AccumuloClient c = 
Accumulo.newClient().from(context.getProperties()).build();

Review Comment:
   > Did not use an IsolatedScanner - not sure if that is necessary or not.
   
   Its good to use an isolated scanner for metadata as it ensure that partial 
changes from a mutation are not seen.



##########
server/manager/src/main/java/org/apache/accumulo/manager/upgrade/Upgrader11to12.java:
##########
@@ -0,0 +1,208 @@
+/*
+ * 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.manager.upgrade;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.apache.accumulo.core.metadata.RootTable.ZROOT_TABLET;
+import static 
org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.DataFileColumnFamily;
+import static 
org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.ExternalCompactionColumnFamily;
+import static 
org.apache.accumulo.core.metadata.schema.UpgraderDeprecatedConstants.ChoppedColumnFamily;
+import static 
org.apache.accumulo.server.AccumuloDataVersion.METADATA_FILE_JSON_ENCODING;
+
+import org.apache.accumulo.core.client.Accumulo;
+import org.apache.accumulo.core.client.AccumuloClient;
+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.data.Key;
+import org.apache.accumulo.core.data.Mutation;
+import org.apache.accumulo.core.data.Value;
+import org.apache.accumulo.core.fate.zookeeper.ZooUtil;
+import org.apache.accumulo.core.metadata.StoredTabletFile;
+import org.apache.accumulo.core.metadata.schema.Ample;
+import org.apache.accumulo.core.metadata.schema.RootTabletMetadata;
+import org.apache.accumulo.core.security.Authorizations;
+import org.apache.accumulo.server.ServerContext;
+import org.apache.hadoop.fs.Path;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.data.Stat;
+import org.checkerframework.checker.nullness.qual.NonNull;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.gson.JsonSyntaxException;
+
+public class Upgrader11to12 implements Upgrader {
+
+  private static final Logger log = 
LoggerFactory.getLogger(Upgrader11to12.class);
+
+  @Override
+  public void upgradeZookeeper(@NonNull ServerContext context) {
+    log.debug("Upgrade ZooKeeper: upgrading to data version {}", 
METADATA_FILE_JSON_ENCODING);
+    var rootBase = ZooUtil.getRoot(context.getInstanceID()) + ZROOT_TABLET;
+
+    try {
+      var zrw = context.getZooReaderWriter();
+      Stat stat = new Stat();
+      byte[] rootData = zrw.getData(rootBase, stat);
+
+      String json = new String(rootData, UTF_8);
+      if (RootTabletMetadata.Data.needsConversion(json)) {
+        RootTabletMetadata rtm = RootTabletMetadata.upgrade(json);
+        zrw.overwritePersistentData(rootBase, rtm.toJson().getBytes(UTF_8), 
stat.getVersion());
+      }
+    } catch (InterruptedException ex) {
+      Thread.currentThread().interrupt();
+      throw new IllegalStateException(
+          "Could not read root metadata from ZooKeeper due to interrupt", ex);
+    } catch (KeeperException ex) {
+      throw new IllegalStateException(
+          "Could not read root metadata from ZooKeeper because of ZooKeeper 
exception", ex);
+    }
+  }
+
+  @Override
+  public void upgradeRoot(@NonNull ServerContext context) {
+    log.debug("Upgrade root: upgrading to data version {}", 
METADATA_FILE_JSON_ENCODING);
+    var rootName = Ample.DataLevel.METADATA.metaTable();
+    processReferences(context, rootName);
+  }
+
+  @Override
+  public void upgradeMetadata(@NonNull ServerContext context) {
+    log.debug("Upgrade metadata: upgrading to data version {}", 
METADATA_FILE_JSON_ENCODING);
+    var metaName = Ample.DataLevel.USER.metaTable();
+    processReferences(context, metaName);
+  }
+
+  private void processReferences(ServerContext context, String tableName) {
+    // not using ample to avoid StoredTabletFile because old file ref is 
incompatible
+    try (AccumuloClient c = 
Accumulo.newClient().from(context.getProperties()).build();
+        BatchWriter batchWriter = c.createBatchWriter(tableName);
+        Scanner scanner = context.createScanner(tableName, 
Authorizations.EMPTY)) {
+
+      scanner.fetchColumnFamily(DataFileColumnFamily.NAME);
+      scanner.fetchColumnFamily(ChoppedColumnFamily.NAME);
+      scanner.fetchColumnFamily(ExternalCompactionColumnFamily.NAME);
+      scanner.forEach((k, v) -> {
+        var family = k.getColumnFamily();
+        if (family.equals(DataFileColumnFamily.NAME)) {
+          upgradeDataFileCF(k, v, batchWriter, tableName);
+        } else if (family.equals(ChoppedColumnFamily.NAME)) {
+          removeChoppedCF(k, batchWriter, tableName);
+        } else if (family.equals(ExternalCompactionColumnFamily.NAME)) {
+          removeExternalCompactionCF(k, batchWriter, tableName);
+        } else {
+          log.warn("Received unexpected column family processing references: " 
+ family);
+        }
+      });
+    } catch (MutationsRejectedException mex) {
+      log.warn("Failed to update reference for table: " + tableName);
+      log.warn("Constraint violations: {}", 
mex.getConstraintViolationSummaries());
+      throw new IllegalStateException("Failed to process table: " + tableName, 
mex);
+    } catch (Exception ex) {
+      throw new IllegalStateException("Failed to process table: " + tableName, 
ex);
+    }
+  }
+
+  @VisibleForTesting
+  void upgradeDataFileCF(final Key key, final Value value, final BatchWriter 
batchWriter,
+      final String tableName) {
+    String file = key.getColumnQualifier().toString();
+    // filter out references if they are in the correct format already.
+    if (fileNeedsConversion(file)) {
+      var fileJson = StoredTabletFile.of(new Path(file)).getMetadataText();
+      try {
+        Mutation update = new Mutation(key.getRow());
+        
update.at().family(DataFileColumnFamily.STR_NAME).qualifier(fileJson).put(value);
+        log.trace("table: {}, adding: {}", tableName, update.prettyPrint());
+        batchWriter.addMutation(update);
+
+        Mutation delete = new Mutation(key.getRow());
+        
delete.at().family(DataFileColumnFamily.STR_NAME).qualifier(file).delete();
+        log.trace("table {}: deleting: {}", tableName, delete.prettyPrint());
+        batchWriter.addMutation(delete);
+      } catch (MutationsRejectedException ex) {
+        // include constraint violation info in log - but stop upgrade
+        log.warn(
+            "Failed to update file reference for table: " + tableName + ". 
row: " + key.getRow());
+        log.warn("Constraint violations: {}", 
ex.getConstraintViolationSummaries());
+        throw new IllegalStateException("File conversion failed. Aborting 
upgrade", ex);
+      }
+    }
+  }
+
+  @VisibleForTesting
+  void removeChoppedCF(final Key key, final BatchWriter batchWriter, final 
String tableName) {
+    Mutation delete = null;
+    try {
+      delete = new 
Mutation(key.getRow()).at().family(ChoppedColumnFamily.STR_NAME)
+          .qualifier(ChoppedColumnFamily.STR_NAME).delete();
+      log.warn(
+          "Deleting chopped reference from:{}. Previous split or delete may 
not have completed cleanly. Ref: {}",
+          tableName, delete.prettyPrint());
+      batchWriter.addMutation(delete);
+    } catch (MutationsRejectedException ex) {
+      log.warn("Failed to delete obsolete chopped CF reference for table: " + 
tableName + ". Ref: "

Review Comment:
   Need to rethrow this exception.  The batch writer queues up mutations and 
does not send them immediately.  When addMuation throws an exception it may not 
have been caused by the most recently added mutation.  Could be from a mutation 
for changing a file.



##########
core/src/test/java/org/apache/accumulo/core/metadata/schema/RootTabletMetadataTest.java:
##########
@@ -0,0 +1,66 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   https://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.accumulo.core.metadata.schema;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+import org.apache.accumulo.core.metadata.StoredTabletFile;
+import org.apache.hadoop.fs.Path;
+import org.junit.jupiter.api.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class RootTabletMetadataTest {
+  private static final Logger LOG = 
LoggerFactory.getLogger(RootTabletMetadataTest.class);
+
+  @Test
+  public void convertRoot1File() {
+    String root21ZkData =
+        
"{\"version\":1,\"columnValues\":{\"file\":{\"hdfs://localhost:8020/accumulo/tables/+r/root_tablet/A000000v.rf\":\"1368,61\"},\"last\":{\"100025091780006\":\"localhost:9997\"},\"loc\":{\"100025091780006\":\"localhost:9997\"},\"srv\":{\"dir\":\"root_tablet\",\"flush\":\"3\",\"lock\":\"tservers/localhost:9997/zlock#9db8961a-4ee9-400e-8e80-3353148baadd#0000000000$100025091780006\",\"time\":\"L53\"},\"~tab\":{\"~pr\":\"\\u0000\"}}}";
+
+    RootTabletMetadata rtm = RootTabletMetadata.upgrade(root21ZkData);
+    LOG.debug("converted column values: {}", 
rtm.toTabletMetadata().getFiles());
+
+    var files = rtm.toTabletMetadata().getFiles();
+    LOG.info("FILES: {}", rtm.toTabletMetadata().getFilesMap());
+
+    assertEquals(1, files.size());
+    assertTrue(files.contains(StoredTabletFile
+        .of(new 
Path("hdfs://localhost:8020/accumulo/tables/+r/root_tablet/A000000v.rf"))));
+  }
+
+  @Test
+  public void convertRoot2Files() {
+    String root212ZkData2Files =
+        
"{\"version\":1,\"columnValues\":{\"file\":{\"hdfs://localhost:8020/accumulo/tables/+r/root_tablet/00000_00000.rf\":\"0,0\",\"hdfs://localhost:8020/accumulo/tables/+r/root_tablet/F000000c.rf\":\"926,18\"},\"last\":{\"10001a84d7d0005\":\"localhost:9997\"},\"loc\":{\"10001a84d7d0005\":\"localhost:9997\"},\"srv\":{\"dir\":\"root_tablet\",\"flush\":\"2\",\"lock\":\"tservers/localhost:9997/zlock#d21adaa4-0f97-4004-9ff8-cce9dbb6687f#0000000000$10001a84d7d0005\",\"time\":\"L6\"},\"~tab\":{\"~pr\":\"\\u0000\"}}}\n";
+
+    RootTabletMetadata rtm = RootTabletMetadata.upgrade(root212ZkData2Files);
+    LOG.debug("converted column values: {}", rtm.toTabletMetadata());
+

Review Comment:
   Can test the needsConversion function.
   
   ```suggestion
   assertTrue(RootTabletMetadata.needsConversion(root212ZkData2Files));
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


Reply via email to