You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@accumulo.apache.org by ed...@apache.org on 2022/10/18 18:07:26 UTC

[accumulo] branch main updated: remove extra node in prop path (#3003)

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

edcoleman 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 13e12e9e88 remove extra node in prop path (#3003)
13e12e9e88 is described below

commit 13e12e9e88908cbe9765e5c6902e8160fa24c501
Author: EdColeman <de...@etcoleman.com>
AuthorDate: Tue Oct 18 14:07:21 2022 -0400

    remove extra node in prop path (#3003)
    
    * uses config as node name, remove extra node in path
---
 .../java/org/apache/accumulo/core/Constants.java   |   4 +-
 .../server/conf/store/NamespacePropKey.java        |  27 +----
 .../accumulo/server/conf/store/PropStoreKey.java   |  49 ++++++---
 .../accumulo/server/conf/store/SystemPropKey.java  |  17 +--
 .../accumulo/server/conf/store/TablePropKey.java   |  21 +---
 .../server/conf/store/impl/ZooPropLoader.java      |   3 +
 .../server/conf/store/impl/ZooPropStore.java       |  10 +-
 .../server/conf/util/ConfigPropertyUpgrader.java   |  28 +++--
 .../server/conf/util/ConfigTransformer.java        | 121 ++++++++++++--------
 .../accumulo/server/conf/util/TransformToken.java  |  15 +--
 .../accumulo/server/conf/util/ZooInfoViewer.java   |  24 ++--
 .../accumulo/server/init/ZooKeeperInitializer.java |   3 -
 .../accumulo/server/tables/TableManager.java       |  11 +-
 .../server/conf/store/PropStoreKeyTest.java        |  55 +++++++---
 .../server/conf/store/impl/PropStoreEventTest.java |   3 +-
 .../server/conf/store/impl/ZooPropLoaderTest.java  | 122 +++++++++++++++++----
 .../server/conf/store/impl/ZooPropStoreTest.java   |  46 ++++++--
 .../conf/util/ConfigPropertyUpgraderTest.java      | 102 +++++++++++++++++
 .../server/conf/util/ZooInfoViewerTest.java        |  54 +++++++--
 .../accumulo/test/conf/PropStoreConfigIT.java      |   8 +-
 .../test/conf/util/ConfigTransformerIT.java        |  13 ++-
 .../accumulo/test/conf/util/TransformTokenIT.java  |  12 +-
 22 files changed, 518 insertions(+), 230 deletions(-)

diff --git a/core/src/main/java/org/apache/accumulo/core/Constants.java b/core/src/main/java/org/apache/accumulo/core/Constants.java
index 289d24acfb..47f7c06ad1 100644
--- a/core/src/main/java/org/apache/accumulo/core/Constants.java
+++ b/core/src/main/java/org/apache/accumulo/core/Constants.java
@@ -40,7 +40,6 @@ public class Constants {
   public static final byte[] ZTABLES_INITIAL_ID = {'0'};
   public static final String ZTABLE_NAME = "/name";
   public static final String ZTABLE_DELETE_MARKER = "/deleting";
-  public static final String ZTABLE_CONF = "/conf";
   public static final String ZTABLE_STATE = "/state";
   public static final String ZTABLE_FLUSH_ID = "/flush-id";
   public static final String ZTABLE_COMPACT_ID = "/compact-id";
@@ -49,7 +48,6 @@ public class Constants {
 
   public static final String ZNAMESPACES = "/namespaces";
   public static final String ZNAMESPACE_NAME = "/name";
-  public static final String ZNAMESPACE_CONF = "/conf";
 
   public static final String ZMANAGERS = "/managers";
   public static final String ZMANAGER_LOCK = ZMANAGERS + "/lock";
@@ -64,6 +62,8 @@ public class Constants {
   public static final String ZMONITOR_LOCK = ZMONITOR + "/lock";
   public static final String ZMONITOR_HTTP_ADDR = ZMONITOR + "/http_addr";
 
+  // used by < 2.1 table and namespace configurations
+  public static final String ZCONF_LEGACY = "/conf";
   public static final String ZCONFIG = "/config";
 
   public static final String ZTSERVERS = "/tservers";
diff --git a/server/base/src/main/java/org/apache/accumulo/server/conf/store/NamespacePropKey.java b/server/base/src/main/java/org/apache/accumulo/server/conf/store/NamespacePropKey.java
index 95a7c21bf7..451442061f 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/conf/store/NamespacePropKey.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/conf/store/NamespacePropKey.java
@@ -18,15 +18,13 @@
  */
 package org.apache.accumulo.server.conf.store;
 
+import static org.apache.accumulo.core.Constants.ZCONFIG;
 import static org.apache.accumulo.core.Constants.ZNAMESPACES;
-import static org.apache.accumulo.core.Constants.ZNAMESPACE_CONF;
-import static org.apache.accumulo.core.Constants.ZTABLE_CONF;
 
 import org.apache.accumulo.core.data.InstanceId;
 import org.apache.accumulo.core.data.NamespaceId;
 import org.apache.accumulo.fate.zookeeper.ZooUtil;
 import org.apache.accumulo.server.ServerContext;
-import org.checkerframework.checker.nullness.qual.NonNull;
 
 public class NamespacePropKey extends PropStoreKey<NamespaceId> {
 
@@ -34,32 +32,15 @@ public class NamespacePropKey extends PropStoreKey<NamespaceId> {
     super(instanceId, path, id);
   }
 
-  private static String getNodeName(final InstanceId instanceId, final NamespaceId id) {
-    return ZooUtil.getRoot(instanceId) + ZNAMESPACES + "/" + id.canonical() + ZTABLE_CONF + "/"
-        + PROP_NODE_NAME;
-  }
-
   public static NamespacePropKey of(final ServerContext context, final NamespaceId id) {
     return of(context.getInstanceID(), id);
   }
 
   public static NamespacePropKey of(final InstanceId instanceId, final NamespaceId id) {
-    return new NamespacePropKey(instanceId, getNodePath(instanceId, id), id);
-  }
-
-  private static String getNodePath(final InstanceId instanceId, final NamespaceId id) {
-    return ZooUtil.getRoot(instanceId) + ZNAMESPACES + "/" + id.canonical() + ZNAMESPACE_CONF + "/"
-        + PROP_NODE_NAME;
-  }
-
-  @Override
-  public @NonNull String getNodePath() {
-    return getNodeName(instanceId, id);
+    return new NamespacePropKey(instanceId, buildNodePath(instanceId, id), id);
   }
 
-  @Override
-  @NonNull
-  public String getBasePath() {
-    return ZooUtil.getRoot(instanceId) + ZNAMESPACES + "/" + id.canonical() + ZNAMESPACE_CONF;
+  private static String buildNodePath(final InstanceId instanceId, final NamespaceId id) {
+    return ZooUtil.getRoot(instanceId) + ZNAMESPACES + "/" + id.canonical() + ZCONFIG;
   }
 }
diff --git a/server/base/src/main/java/org/apache/accumulo/server/conf/store/PropStoreKey.java b/server/base/src/main/java/org/apache/accumulo/server/conf/store/PropStoreKey.java
index 73b1db00bb..45f82476fb 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/conf/store/PropStoreKey.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/conf/store/PropStoreKey.java
@@ -18,6 +18,7 @@
  */
 package org.apache.accumulo.server.conf.store;
 
+import static org.apache.accumulo.core.Constants.ZCONFIG;
 import static org.apache.accumulo.core.Constants.ZNAMESPACES;
 import static org.apache.accumulo.core.Constants.ZTABLES;
 
@@ -28,6 +29,7 @@ import org.apache.accumulo.core.data.AbstractId;
 import org.apache.accumulo.core.data.InstanceId;
 import org.apache.accumulo.core.data.NamespaceId;
 import org.apache.accumulo.core.data.TableId;
+import org.apache.accumulo.fate.zookeeper.ZooUtil;
 import org.checkerframework.checker.nullness.qual.NonNull;
 import org.checkerframework.checker.nullness.qual.Nullable;
 import org.slf4j.Logger;
@@ -44,11 +46,9 @@ import org.slf4j.LoggerFactory;
 public abstract class PropStoreKey<ID_TYPE extends AbstractId<ID_TYPE>>
     implements Comparable<PropStoreKey<ID_TYPE>> {
 
-  public static final String PROP_NODE_NAME = "encoded_props";
-
   private static final Logger log = LoggerFactory.getLogger(PropStoreKey.class);
 
-  // indices for path.split();
+  // indices for path.split() on config node paths;
   public static final int TYPE_TOKEN_POSITION = 3;
   public static final int IID_TOKEN_POSITION = 2;
   public static final int ID_TOKEN_POSITION = 4;
@@ -56,6 +56,10 @@ public abstract class PropStoreKey<ID_TYPE extends AbstractId<ID_TYPE>>
   // remove starting slash from constant.
   public static final String TABLES_NODE_NAME = ZTABLES.substring(1);
   public static final String NAMESPACE_NODE_NAME = ZNAMESPACES.substring(1);
+  // expected token length for table and namespace config
+  public static final int EXPECTED_CONFIG_LEN = 6;
+  // expected token length for sys config
+  public static final int EXPECTED_SYS_CONFIG_LEN = 4;
 
   protected final InstanceId instanceId;
   protected final ID_TYPE id;
@@ -72,10 +76,6 @@ public abstract class PropStoreKey<ID_TYPE extends AbstractId<ID_TYPE>>
     return path;
   }
 
-  public @NonNull abstract String getBasePath();
-
-  public @NonNull abstract String getNodePath();
-
   public @NonNull ID_TYPE getId() {
     return id;
   }
@@ -90,24 +90,39 @@ public abstract class PropStoreKey<ID_TYPE extends AbstractId<ID_TYPE>>
   public static @Nullable PropStoreKey<?> fromPath(final String path) {
     String[] tokens = path.split("/");
 
-    InstanceId instanceId;
-    try {
-      instanceId = InstanceId.of(tokens[IID_TOKEN_POSITION]);
-    } catch (ArrayIndexOutOfBoundsException ex) {
-      log.warn("Path '{}' is an invalid path for a property cache key", path);
+    if (tokens.length < 1
+        || tokens.length != EXPECTED_CONFIG_LEN && tokens.length != EXPECTED_SYS_CONFIG_LEN) {
+      log.warn("Path '{}' is an invalid path for a property cache key - bad length", path);
       return null;
     }
-    if (tokens.length < 1 || !tokens[tokens.length - 1].equals(PROP_NODE_NAME)) {
-      // without tokens or it does not end with PROP_NAME_NAME
+
+    InstanceId instanceId = InstanceId.of(tokens[IID_TOKEN_POSITION]);
+
+    // needs to start with /accumulo/[instanceId]
+    if (!path.startsWith(ZooUtil.getRoot(instanceId))) {
+      log.warn(
+          "Path '{}' is invalid for a property cache key, expected to start with /accumulo/{}}",
+          path, instanceId);
       return null;
     }
-    if (tokens[TYPE_TOKEN_POSITION].equals(TABLES_NODE_NAME)) {
+
+    String nodeName = "/" + tokens[tokens.length - 1];
+    if (tokens.length == EXPECTED_CONFIG_LEN && tokens[TYPE_TOKEN_POSITION].equals(TABLES_NODE_NAME)
+        && nodeName.equals(ZCONFIG)) {
       return TablePropKey.of(instanceId, TableId.of(tokens[ID_TOKEN_POSITION]));
     }
-    if (tokens[TYPE_TOKEN_POSITION].equals(NAMESPACE_NODE_NAME)) {
+
+    if (tokens.length == EXPECTED_CONFIG_LEN
+        && tokens[TYPE_TOKEN_POSITION].equals(NAMESPACE_NODE_NAME) && nodeName.equals(ZCONFIG)) {
       return NamespacePropKey.of(instanceId, NamespaceId.of(tokens[ID_TOKEN_POSITION]));
     }
-    return SystemPropKey.of(instanceId);
+
+    if (tokens.length == EXPECTED_SYS_CONFIG_LEN && nodeName.equals(ZCONFIG)) {
+      return SystemPropKey.of(instanceId);
+    }
+    // without tokens or it does not end with PROP_NAME_NAME
+    log.warn("Path '{}' is an invalid path for a property cache key", path);
+    return null;
   }
 
   @Override
diff --git a/server/base/src/main/java/org/apache/accumulo/server/conf/store/SystemPropKey.java b/server/base/src/main/java/org/apache/accumulo/server/conf/store/SystemPropKey.java
index b3b28ec709..6fe811b8cf 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/conf/store/SystemPropKey.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/conf/store/SystemPropKey.java
@@ -23,7 +23,6 @@ import static org.apache.accumulo.core.Constants.ZCONFIG;
 import org.apache.accumulo.core.data.InstanceId;
 import org.apache.accumulo.fate.zookeeper.ZooUtil;
 import org.apache.accumulo.server.ServerContext;
-import org.checkerframework.checker.nullness.qual.NonNull;
 
 public class SystemPropKey extends PropStoreKey<InstanceId> {
 
@@ -31,26 +30,16 @@ public class SystemPropKey extends PropStoreKey<InstanceId> {
     super(instanceId, path, instanceId);
   }
 
-  @Override
-  public @NonNull String getNodePath() {
-    return getNodeName(instanceId);
-  }
-
-  @Override
-  public @NonNull String getBasePath() {
-    return ZooUtil.getRoot(instanceId) + ZCONFIG;
-  }
-
   public static SystemPropKey of(final ServerContext context) {
     return of(context.getInstanceID());
   }
 
   public static SystemPropKey of(final InstanceId instanceId) {
-    return new SystemPropKey(instanceId, getNodeName(instanceId));
+    return new SystemPropKey(instanceId, buildNodePath(instanceId));
   }
 
-  private static String getNodeName(final InstanceId instanceId) {
-    return ZooUtil.getRoot(instanceId) + ZCONFIG + "/" + PROP_NODE_NAME;
+  private static String buildNodePath(final InstanceId instanceId) {
+    return ZooUtil.getRoot(instanceId) + ZCONFIG;
   }
 
 }
diff --git a/server/base/src/main/java/org/apache/accumulo/server/conf/store/TablePropKey.java b/server/base/src/main/java/org/apache/accumulo/server/conf/store/TablePropKey.java
index 7060d2aa67..fcc6501e61 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/conf/store/TablePropKey.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/conf/store/TablePropKey.java
@@ -18,14 +18,13 @@
  */
 package org.apache.accumulo.server.conf.store;
 
+import static org.apache.accumulo.core.Constants.ZCONFIG;
 import static org.apache.accumulo.core.Constants.ZTABLES;
-import static org.apache.accumulo.core.Constants.ZTABLE_CONF;
 
 import org.apache.accumulo.core.data.InstanceId;
 import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.fate.zookeeper.ZooUtil;
 import org.apache.accumulo.server.ServerContext;
-import org.checkerframework.checker.nullness.qual.NonNull;
 
 public class TablePropKey extends PropStoreKey<TableId> {
 
@@ -34,26 +33,14 @@ public class TablePropKey extends PropStoreKey<TableId> {
   }
 
   public static TablePropKey of(final InstanceId instanceId, final TableId tableId) {
-    return new TablePropKey(instanceId, getNodePath(instanceId, tableId), tableId);
+    return new TablePropKey(instanceId, buildNodePath(instanceId, tableId), tableId);
   }
 
   private TablePropKey(final InstanceId instanceId, final String path, final TableId tableId) {
     super(instanceId, path, tableId);
   }
 
-  private static String getNodePath(final InstanceId instanceId, final TableId id) {
-    return ZooUtil.getRoot(instanceId) + ZTABLES + "/" + id.canonical() + ZTABLE_CONF + "/"
-        + PROP_NODE_NAME;
+  private static String buildNodePath(final InstanceId instanceId, final TableId id) {
+    return ZooUtil.getRoot(instanceId) + ZTABLES + "/" + id.canonical() + ZCONFIG;
   }
-
-  @Override
-  public @NonNull String getNodePath() {
-    return getNodePath(instanceId, id);
-  }
-
-  @Override
-  public @NonNull String getBasePath() {
-    return ZooUtil.getRoot(instanceId) + ZTABLES + "/" + id.canonical() + ZTABLE_CONF;
-  }
-
 }
diff --git a/server/base/src/main/java/org/apache/accumulo/server/conf/store/impl/ZooPropLoader.java b/server/base/src/main/java/org/apache/accumulo/server/conf/store/impl/ZooPropLoader.java
index d57fbec925..5eae033768 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/conf/store/impl/ZooPropLoader.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/conf/store/impl/ZooPropLoader.java
@@ -63,6 +63,9 @@ public class ZooPropLoader implements CacheLoader<PropStoreKey<?>,VersionedPrope
 
       Stat stat = new Stat();
       byte[] bytes = zrw.getData(propStoreKey.getPath(), propStoreWatcher, stat);
+      if (stat.getDataLength() == 0) {
+        return new VersionedProperties();
+      }
       VersionedProperties vProps = propCodec.fromBytes(stat.getVersion(), bytes);
 
       metrics.addLoadTime(
diff --git a/server/base/src/main/java/org/apache/accumulo/server/conf/store/impl/ZooPropStore.java b/server/base/src/main/java/org/apache/accumulo/server/conf/store/impl/ZooPropStore.java
index d9a8e2d516..a26fcc86a7 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/conf/store/impl/ZooPropStore.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/conf/store/impl/ZooPropStore.java
@@ -190,7 +190,8 @@ public class ZooPropStore implements PropStore, PropChangeListener {
     }
 
     if (propStoreKey instanceof SystemPropKey) {
-      return new ConfigTransformer(zrw, codec, propStoreWatcher).transform(propStoreKey);
+      return new ConfigTransformer(zrw, codec, propStoreWatcher).transform(propStoreKey,
+          propStoreKey.getPath(), false);
     }
 
     throw new IllegalStateException(
@@ -222,6 +223,10 @@ public class ZooPropStore implements PropStore, PropChangeListener {
     try {
       Stat stat = new Stat();
       byte[] bytes = zooReader.getData(propStoreKey.getPath(), watcher, stat);
+      if (stat.getDataLength() == 0) {
+        // node exists - but is empty - no props have been stored on node.
+        return null;
+      }
       return codec.fromBytes(stat.getVersion(), bytes);
     } catch (KeeperException.NoNodeException ex) {
       // ignore no node - allow other exceptions to propagate
@@ -438,6 +443,9 @@ public class ZooPropStore implements PropStore, PropChangeListener {
     try {
       Stat stat = new Stat();
       byte[] bytes = zrw.getData(propStoreKey.getPath(), stat);
+      if (stat.getDataLength() == 0) {
+        return new VersionedProperties();
+      }
       return codec.fromBytes(stat.getVersion(), bytes);
     } catch (InterruptedException ex) {
       Thread.currentThread().interrupt();
diff --git a/server/base/src/main/java/org/apache/accumulo/server/conf/util/ConfigPropertyUpgrader.java b/server/base/src/main/java/org/apache/accumulo/server/conf/util/ConfigPropertyUpgrader.java
index e9f0020215..ca8812edff 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/conf/util/ConfigPropertyUpgrader.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/conf/util/ConfigPropertyUpgrader.java
@@ -18,6 +18,8 @@
  */
 package org.apache.accumulo.server.conf.util;
 
+import static org.apache.accumulo.core.Constants.ZCONFIG;
+
 import java.util.Set;
 import java.util.TreeSet;
 
@@ -41,6 +43,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import com.google.auto.service.AutoService;
+import com.google.common.annotations.VisibleForTesting;
 
 @AutoService(KeywordExecutable.class)
 public class ConfigPropertyUpgrader implements KeywordExecutable {
@@ -88,21 +91,25 @@ public class ConfigPropertyUpgrader implements KeywordExecutable {
     upgradeTableProps(instanceId, zrw, transformer);
   }
 
-  private void upgradeSysProps(final InstanceId instanceId, final ConfigTransformer transformer) {
+  @VisibleForTesting
+  void upgradeSysProps(final InstanceId instanceId, final ConfigTransformer transformer) {
     log.info("Upgrade system config properties for {}", instanceId);
-    transformer.transform(SystemPropKey.of(instanceId));
+    String legacyPath = ZooUtil.getRoot(instanceId) + ZCONFIG;
+    transformer.transform(SystemPropKey.of(instanceId), legacyPath, false);
   }
 
-  private void upgradeNamespaceProps(final InstanceId instanceId, final ZooReaderWriter zrw,
+  @VisibleForTesting
+  void upgradeNamespaceProps(final InstanceId instanceId, final ZooReaderWriter zrw,
       final ConfigTransformer transformer) {
     String zkPathNamespaceBase = ZooUtil.getRoot(instanceId) + Constants.ZNAMESPACES;
     try {
       // sort is cosmetic - only improves readability and consistency in logs
       Set<String> namespaces = new TreeSet<>(zrw.getChildren(zkPathNamespaceBase));
       for (String namespace : namespaces) {
-        String zkPropBasePath = zkPathNamespaceBase + "/" + namespace + Constants.ZNAMESPACE_CONF;
-        log.info("Upgrading namespace {} base path: {}", namespace, zkPropBasePath);
-        transformer.transform(NamespacePropKey.of(instanceId, NamespaceId.of(namespace)));
+        String legacyPath = zkPathNamespaceBase + "/" + namespace + Constants.ZCONF_LEGACY;
+        log.info("Upgrading namespace {} base path: {}", namespace, legacyPath);
+        transformer.transform(NamespacePropKey.of(instanceId, NamespaceId.of(namespace)),
+            legacyPath, true);
       }
     } catch (KeeperException ex) {
       throw new IllegalStateException(
@@ -113,16 +120,17 @@ public class ConfigPropertyUpgrader implements KeywordExecutable {
     }
   }
 
-  private void upgradeTableProps(final InstanceId instanceId, final ZooReaderWriter zrw,
+  @VisibleForTesting
+  void upgradeTableProps(final InstanceId instanceId, final ZooReaderWriter zrw,
       ConfigTransformer transformer) {
     String zkPathTableBase = ZooUtil.getRoot(instanceId) + Constants.ZTABLES;
     try {
       // sort is cosmetic - only improves readability and consistency in logs
       Set<String> tables = new TreeSet<>(zrw.getChildren(zkPathTableBase));
       for (String table : tables) {
-        String zkPropBasePath = zkPathTableBase + "/" + table + Constants.ZNAMESPACE_CONF;
-        log.info("Upgrading table {} base path: {}", table, zkPropBasePath);
-        transformer.transform(TablePropKey.of(instanceId, TableId.of(table)));
+        String legacyPath = zkPathTableBase + "/" + table + Constants.ZCONF_LEGACY;
+        log.info("Upgrading table {} base path: {}", table, legacyPath);
+        transformer.transform(TablePropKey.of(instanceId, TableId.of(table)), legacyPath, true);
       }
     } catch (KeeperException ex) {
       throw new IllegalStateException(
diff --git a/server/base/src/main/java/org/apache/accumulo/server/conf/util/ConfigTransformer.java b/server/base/src/main/java/org/apache/accumulo/server/conf/util/ConfigTransformer.java
index 3a75823774..205777ed2f 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/conf/util/ConfigTransformer.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/conf/util/ConfigTransformer.java
@@ -35,6 +35,7 @@ import java.util.TreeSet;
 
 import org.apache.accumulo.core.conf.DeprecatedPropertyUtil;
 import org.apache.accumulo.core.util.DurationFormat;
+import org.apache.accumulo.core.util.Pair;
 import org.apache.accumulo.fate.util.Retry;
 import org.apache.accumulo.fate.zookeeper.ZooReaderWriter;
 import org.apache.accumulo.fate.zookeeper.ZooUtil;
@@ -104,26 +105,25 @@ public class ConfigTransformer {
    *
    * @return the encoded properties.
    */
-  public VersionedProperties transform(final PropStoreKey<?> propStoreKey) {
-    TransformToken token = TransformToken.createToken(propStoreKey, zrw);
-    return transform(propStoreKey, token);
+  public VersionedProperties transform(final PropStoreKey<?> propStoreKey, final String legacyPath,
+      final boolean deleteLegacyNode) {
+    TransformToken token = TransformToken.createToken(legacyPath, zrw);
+    return transform(propStoreKey, token, legacyPath, deleteLegacyNode);
   }
 
   // Allow external (mocked) TransformToken to be used
   @VisibleForTesting
-  VersionedProperties transform(final PropStoreKey<?> propStoreKey, final TransformToken token) {
-
-    log.info("checking for legacy property upgrade transform for {}", propStoreKey);
-
+  VersionedProperties transform(final PropStoreKey<?> propStoreKey, final TransformToken token,
+      final String legacyPath, final boolean deleteLegacyNode) {
+    log.trace("checking for legacy property upgrade transform for {}", propStoreKey);
     VersionedProperties results;
     Instant start = Instant.now();
     try {
-
       // check for node - just return if it exists.
       results = ZooPropStore.readFromZk(propStoreKey, propStoreWatcher, zrw);
       if (results != null) {
-        log.debug(
-            "Found existing node at {}. skipping legacy prop conversion - version: {}, timestamp: {}",
+        log.trace(
+            "Found existing node with properties at {}. skipping legacy prop conversion - version: {}, timestamp: {}",
             propStoreKey, results.getDataVersion(), results.getTimestamp());
         return results;
       }
@@ -137,8 +137,8 @@ public class ConfigTransformer {
               propStoreKey.getPath());
           results = ZooPropStore.readFromZk(propStoreKey, propStoreWatcher, zrw);
           if (results != null) {
-            log.debug(
-                "Found existing node after getting token at {}. skipping legacy prop conversion - version: {}, timestamp: {}",
+            log.trace(
+                "Found existing node with properties after getting token at {}. skipping legacy prop conversion - version: {}, timestamp: {}",
                 propStoreKey, results.getDataVersion(), results.getTimestamp());
             return results;
           }
@@ -152,14 +152,11 @@ public class ConfigTransformer {
         }
       }
 
-      Set<LegacyPropNode> upgradeNodes = readLegacyProps(propStoreKey);
-      if (upgradeNodes == null) {
-        log.info("Found existing node after reading legacy props {}, skipping conversion",
+      Set<LegacyPropNode> upgradeNodes = readLegacyProps(legacyPath);
+      if (upgradeNodes.size() == 0) {
+        log.trace("No existing legacy props {}, skipping conversion, writing default prop node",
             propStoreKey);
-        results = ZooPropStore.readFromZk(propStoreKey, propStoreWatcher, zrw);
-        if (results != null) {
-          return results;
-        }
+        return writeNode(propStoreKey, Map.of());
       }
 
       upgradeNodes = convertDeprecatedProps(propStoreKey, upgradeNodes);
@@ -176,17 +173,27 @@ public class ConfigTransformer {
             "legacy conversion failed. Lost transform token for " + propStoreKey);
       }
 
-      int errorCount = deleteLegacyProps(upgradeNodes);
-      log.debug("deleted legacy props - error count: {}", errorCount);
-      log.debug("property transform for {} took {} ms", propStoreKey,
-          new DurationFormat(Duration.between(start, Instant.now()).toMillis(), ""));
+      Pair<Integer,Integer> deleteCounts = deleteLegacyProps(upgradeNodes);
+      log.info("property transform for {} took {} ms, delete count: {}, error count: {}",
+          propStoreKey, new DurationFormat(Duration.between(start, Instant.now()).toMillis(), ""),
+          deleteCounts.getFirst(), deleteCounts.getSecond());
 
       return results;
 
     } catch (Exception ex) {
-      log.info("Exception on upgrading legacy properties for: " + propStoreKey, ex);
+      log.info("Issue on upgrading legacy properties for: " + propStoreKey, ex);
     } finally {
       token.releaseToken();
+      if (deleteLegacyNode) {
+        log.trace("Delete legacy property base node: {}", legacyPath);
+        try {
+          zrw.delete(legacyPath);
+        } catch (KeeperException.NotEmptyException ex) {
+          log.info("Delete for legacy prop node {} - not empty", legacyPath);
+        } catch (KeeperException | InterruptedException ex) {
+          Thread.currentThread().interrupt();
+        }
+      }
     }
     return null;
   }
@@ -212,7 +219,7 @@ public class ConfigTransformer {
     return renamedNodes;
   }
 
-  private @Nullable Set<LegacyPropNode> readLegacyProps(PropStoreKey<?> propStoreKey) {
+  private @NonNull Set<LegacyPropNode> readLegacyProps(final String basePath) {
 
     Set<LegacyPropNode> legacyProps = new TreeSet<>();
 
@@ -220,22 +227,16 @@ public class ConfigTransformer {
     var tokenName = TransformToken.TRANSFORM_TOKEN.substring(1);
 
     try {
-      var keyBasePath = propStoreKey.getBasePath();
-      List<String> childNames = zrw.getChildren(keyBasePath);
+      List<String> childNames = zrw.getChildren(basePath);
       for (String propName : childNames) {
-        log.trace("processing ZooKeeper child node: {} for: {}", propName, propStoreKey);
+        log.trace("processing ZooKeeper child node: {} at path: {}", propName, basePath);
         if (tokenName.equals(propName)) {
           continue;
         }
-        if (PropStoreKey.PROP_NODE_NAME.equals(propName)) {
-          log.debug(
-              "encoded property node exists for {}. Legacy conversion ignoring conversion of this node",
-              propStoreKey);
-          return null;
-        }
+
         log.trace("Adding: {} to list for legacy conversion", propName);
 
-        var path = keyBasePath + "/" + propName;
+        var path = basePath + "/" + propName;
         Stat stat = new Stat();
         byte[] bytes = zrw.getData(path, stat);
 
@@ -262,10 +263,14 @@ public class ConfigTransformer {
     return legacyProps;
   }
 
-  private int deleteLegacyProps(Set<LegacyPropNode> nodes) {
+  private Pair<Integer,Integer> deleteLegacyProps(Set<LegacyPropNode> nodes) {
+    int deleteCount = 0;
     int errorCount = 0;
     for (LegacyPropNode n : nodes) {
       try {
+        log.trace("Delete legacy prop at path: {}, data version: {}", n.getPath(),
+            n.getNodeVersion());
+        deleteCount++;
         zrw.deleteStrict(n.getPath(), n.getNodeVersion());
       } catch (InterruptedException ex) {
         Thread.currentThread().interrupt();
@@ -275,29 +280,27 @@ public class ConfigTransformer {
         log.info("Failed to delete node during upgrade clean-up", ex);
       }
     }
-    return errorCount;
+    return new Pair<>(deleteCount, errorCount);
   }
 
   private @Nullable VersionedProperties writeConverted(final PropStoreKey<?> propStoreKey,
       final Set<LegacyPropNode> nodes) {
     final Map<String,String> props = new HashMap<>();
     nodes.forEach(node -> props.put(node.getPropName(), node.getData()));
-    VersionedProperties vProps = new VersionedProperties(props);
-    String path = propStoreKey.getPath();
+
+    VersionedProperties vProps;
     try {
-      try {
-        zrw.putPrivatePersistentData(path, codec.toBytes(vProps), ZooUtil.NodeExistsPolicy.FAIL);
-      } catch (KeeperException.NodeExistsException ex) {
-        vProps = ZooPropStore.readFromZk(propStoreKey, propStoreWatcher, zrw);
-      }
-    } catch (InterruptedException | IOException | KeeperException ex) {
+      vProps = writeNode(propStoreKey, props);
+    } catch (InterruptedException | KeeperException ex) {
       if (ex instanceof InterruptedException) {
         Thread.currentThread().interrupt();
       }
       throw new IllegalStateException(
           "failed to create node for " + propStoreKey + " on conversion", ex);
     }
+
     if (!validateWrite(propStoreKey, vProps)) {
+      log.trace("Failed property conversion validation for: {}", propStoreKey);
       // failed validation
       return null;
     }
@@ -305,6 +308,32 @@ public class ConfigTransformer {
     return vProps;
   }
 
+  private VersionedProperties writeNode(final PropStoreKey<?> propStoreKey,
+      final Map<String,String> props) throws InterruptedException, KeeperException {
+    VersionedProperties vProps;
+    try {
+      String path = propStoreKey.getPath();
+      log.trace("Writing converted properties to ZooKeeper path: {} for key: {}", path,
+          propStoreKey);
+      Stat currStat = zrw.getStatus(path);
+      if (currStat == null || currStat.getDataLength() == 0) {
+        // no node or node with no props stored
+        vProps = new VersionedProperties(props);
+        zrw.putPrivatePersistentData(path, codec.toBytes(vProps),
+            ZooUtil.NodeExistsPolicy.OVERWRITE);
+        // re-read to update property version
+        vProps = ZooPropStore.readFromZk(propStoreKey, propStoreWatcher, zrw);
+      } else {
+        // props exist - return stored value
+        vProps = ZooPropStore.readFromZk(propStoreKey, propStoreWatcher, zrw);
+      }
+    } catch (IOException ex) {
+      throw new IllegalStateException(
+          "failed to create node for " + propStoreKey + " on conversion", ex);
+    }
+    return vProps;
+  }
+
   private boolean validateWrite(final PropStoreKey<?> propStoreKey,
       final VersionedProperties vProps) {
     try {
@@ -313,6 +342,8 @@ public class ConfigTransformer {
         throw new IllegalStateException(
             "failed to get stat to validate created node for " + propStoreKey);
       }
+      log.debug("Property conversion validation - version received: {}, version expected: {}",
+          stat.getVersion(), vProps.getDataVersion());
       return stat.getVersion() == vProps.getDataVersion();
     } catch (KeeperException ex) {
       throw new IllegalStateException("failed to validate created node for " + propStoreKey, ex);
diff --git a/server/base/src/main/java/org/apache/accumulo/server/conf/util/TransformToken.java b/server/base/src/main/java/org/apache/accumulo/server/conf/util/TransformToken.java
index 5b1a0d6de0..8de41198a2 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/conf/util/TransformToken.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/conf/util/TransformToken.java
@@ -25,7 +25,6 @@ import java.util.Objects;
 import java.util.UUID;
 
 import org.apache.accumulo.fate.zookeeper.ZooReaderWriter;
-import org.apache.accumulo.server.conf.store.PropStoreKey;
 import org.apache.zookeeper.KeeperException;
 import org.apache.zookeeper.data.Stat;
 import org.checkerframework.checker.nullness.qual.NonNull;
@@ -55,8 +54,8 @@ public class TransformToken {
   private final ZooReaderWriter zrw;
   private boolean haveToken = false;
 
-  private TransformToken(final @NonNull PropStoreKey<?> key, final ZooReaderWriter zrw) {
-    path = key.getBasePath() + TRANSFORM_TOKEN;
+  private TransformToken(final @NonNull String basePath, final ZooReaderWriter zrw) {
+    path = basePath + TRANSFORM_TOKEN;
     this.zrw = zrw;
 
     boolean t = getTokenOwnership();
@@ -69,18 +68,16 @@ public class TransformToken {
    * already exists, the lock is unlocked and the caller can decide to either wait for the resource
    * to be created by the thread that created the lock, or try calling to {@code lock} to succeed
    *
-   * @param key
-   *          a PropStoreKey that defines the storage location of the created lock and the
-   *          associated property nodes.
+   * @param path
+   *          the parent node of the legacy properties and the associated property children nodes.
    * @param zrw
    *          a ZooReaderWriter
    * @return an TransformLock instance.
    * @throws IllegalStateException
    *           is the lock creation fails due to an underlying ZooKeeper exception.
    */
-  public static TransformToken createToken(final @NonNull PropStoreKey<?> key,
-      final ZooReaderWriter zrw) {
-    return new TransformToken(key, zrw);
+  public static TransformToken createToken(final @NonNull String path, final ZooReaderWriter zrw) {
+    return new TransformToken(path, zrw);
   }
 
   /**
diff --git a/server/base/src/main/java/org/apache/accumulo/server/conf/util/ZooInfoViewer.java b/server/base/src/main/java/org/apache/accumulo/server/conf/util/ZooInfoViewer.java
index d7d2283025..784aecf625 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/conf/util/ZooInfoViewer.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/conf/util/ZooInfoViewer.java
@@ -342,7 +342,7 @@ public class ZooInfoViewer implements KeywordExecutable {
     filteredIds.forEach((nid, name) -> {
       try {
         var key = NamespacePropKey.of(iid, nid);
-        log.trace("fetch props from path: {}", key.getNodePath());
+        log.trace("fetch props from path: {}", key.getPath());
         var props = ZooPropStore.readFromZk(key, nullWatcher, zooReader);
         results.put(name, props);
       } catch (InterruptedException ex) {
@@ -379,7 +379,7 @@ public class ZooInfoViewer implements KeywordExecutable {
     filteredIds.forEach((tid, name) -> {
       try {
         var key = TablePropKey.of(iid, tid);
-        log.trace("fetch props from path: {}", key.getNodePath());
+        log.trace("fetch props from path: {}", key.getPath());
         var props = ZooPropStore.readFromZk(key, nullWatcher, zooReader);
         results.put(name, props);
       } catch (InterruptedException ex) {
@@ -424,16 +424,20 @@ public class ZooInfoViewer implements KeywordExecutable {
       final Map<String,VersionedProperties> props) {
     log.trace("Printing: {}", props);
     props.forEach((n, p) -> {
-      writer.printf("Name: %s, Data Version:%s, Data Timestamp: %s:\n", n, p.getDataVersion(),
-          tsFormat.format(p.getTimestamp()));
-      Map<String,String> pMap = p.asMap();
-      if (pMap.isEmpty()) {
-        writer.println("-- none --");
+      if (p == null) {
+        writer.printf("Name: '%s' : no property node present\n", n);
       } else {
-        TreeMap<String,String> sorted = new TreeMap<>(pMap);
-        sorted.forEach((name, value) -> writer.printf("%s%s=%s\n", INDENT, name, value));
+        writer.printf("Name: %s, Data Version:%s, Data Timestamp: %s:\n", n, p.getDataVersion(),
+            tsFormat.format(p.getTimestamp()));
+        Map<String,String> pMap = p.asMap();
+        if (pMap.isEmpty()) {
+          writer.println("-- none --");
+        } else {
+          TreeMap<String,String> sorted = new TreeMap<>(pMap);
+          sorted.forEach((name, value) -> writer.printf("%s%s=%s\n", INDENT, name, value));
+        }
+        writer.println();
       }
-      writer.println();
     });
   }
 
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 08771f6efe..2853b5a6b2 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
@@ -73,9 +73,6 @@ public class ZooKeeperInitializer {
 
       String zkInstanceRoot = Constants.ZROOT + "/" + instanceId;
       zoo.putPersistentData(zkInstanceRoot, EMPTY_BYTE_ARRAY, ZooUtil.NodeExistsPolicy.SKIP);
-      zoo.putPersistentData(zkInstanceRoot + Constants.ZCONFIG, EMPTY_BYTE_ARRAY,
-          ZooUtil.NodeExistsPolicy.SKIP);
-
       var sysPropPath = SystemPropKey.of(instanceId).getPath();
       VersionedProperties vProps = new VersionedProperties();
       // skip if the encoded props node exists
diff --git a/server/base/src/main/java/org/apache/accumulo/server/tables/TableManager.java b/server/base/src/main/java/org/apache/accumulo/server/tables/TableManager.java
index d0795f2b3a..16ce184e66 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/tables/TableManager.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/tables/TableManager.java
@@ -75,8 +75,6 @@ public class TableManager {
     zoo.putPersistentData(zPath, new byte[0], existsPolicy);
     zoo.putPersistentData(zPath + Constants.ZNAMESPACE_NAME, namespace.getBytes(UTF_8),
         existsPolicy);
-    zoo.putPersistentData(zPath + Constants.ZNAMESPACE_CONF, new byte[0], existsPolicy);
-
     var propKey = NamespacePropKey.of(instanceId, namespaceId);
     if (!propStore.exists(propKey)) {
       propStore.create(propKey, Map.of());
@@ -101,7 +99,6 @@ public class TableManager {
     tableName = qualifiedTableName.getSecond();
     String zTablePath = Constants.ZROOT + "/" + instanceId + Constants.ZTABLES + "/" + tableId;
     zoo.putPersistentData(zTablePath, new byte[0], existsPolicy);
-    zoo.putPersistentData(zTablePath + Constants.ZTABLE_CONF, new byte[0], existsPolicy);
     zoo.putPersistentData(zTablePath + Constants.ZTABLE_NAMESPACE,
         namespaceId.canonical().getBytes(UTF_8), existsPolicy);
     zoo.putPersistentData(zTablePath + Constants.ZTABLE_NAME, tableName.getBytes(UTF_8),
@@ -221,9 +218,9 @@ public class TableManager {
         TableState.NEW, NodeExistsPolicy.OVERWRITE);
 
     String srcTablePath = Constants.ZROOT + "/" + instanceID + Constants.ZTABLES + "/" + srcTableId
-        + Constants.ZTABLE_CONF;
-    String newTablePath = Constants.ZROOT + "/" + instanceID + Constants.ZTABLES + "/" + tableId
-        + Constants.ZTABLE_CONF;
+        + Constants.ZCONFIG;
+    String newTablePath =
+        Constants.ZROOT + "/" + instanceID + Constants.ZTABLES + "/" + tableId + Constants.ZCONFIG;
     zoo.recursiveCopyPersistentOverwrite(srcTablePath, newTablePath);
 
     PropUtil.setProperties(context, TablePropKey.of(context, tableId), propertiesToSet);
@@ -296,7 +293,7 @@ public class TableManager {
         case NodeDeleted:
           if (zPath != null && tableId != null
               && (zPath.equals(tablesPrefix + "/" + tableId + Constants.ZTABLE_STATE)
-                  || zPath.equals(tablesPrefix + "/" + tableId + Constants.ZTABLE_CONF)
+                  || zPath.equals(tablesPrefix + "/" + tableId + Constants.ZCONFIG)
                   || zPath.equals(tablesPrefix + "/" + tableId + Constants.ZTABLE_NAME)))
             tableStateCache.remove(tableId);
           break;
diff --git a/server/base/src/test/java/org/apache/accumulo/server/conf/store/PropStoreKeyTest.java b/server/base/src/test/java/org/apache/accumulo/server/conf/store/PropStoreKeyTest.java
index 601409f5a0..f3d040275d 100644
--- a/server/base/src/test/java/org/apache/accumulo/server/conf/store/PropStoreKeyTest.java
+++ b/server/base/src/test/java/org/apache/accumulo/server/conf/store/PropStoreKeyTest.java
@@ -21,13 +21,13 @@ package org.apache.accumulo.server.conf.store;
 import static org.apache.accumulo.core.Constants.ZCONFIG;
 import static org.apache.accumulo.core.Constants.ZNAMESPACES;
 import static org.apache.accumulo.core.Constants.ZTABLES;
-import static org.apache.accumulo.server.conf.store.PropStoreKey.PROP_NODE_NAME;
 import static org.easymock.EasyMock.createMock;
 import static org.easymock.EasyMock.expect;
 import static org.easymock.EasyMock.replay;
 import static org.easymock.EasyMock.verify;
 import static org.junit.jupiter.api.Assertions.assertEquals;
 import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertNull;
 import static org.junit.jupiter.api.Assertions.assertTrue;
 
 import java.util.UUID;
@@ -49,7 +49,7 @@ public class PropStoreKeyTest {
   public void systemType() {
     var propKey = SystemPropKey.of(instanceId);
     log.info("name: {}", propKey);
-    assertTrue(propKey.getPath().endsWith(ZCONFIG + "/" + PROP_NODE_NAME));
+    assertTrue(propKey.getPath().endsWith(ZCONFIG));
   }
 
   @Test
@@ -60,7 +60,7 @@ public class PropStoreKeyTest {
 
     var propKey = SystemPropKey.of(context);
     log.info("propKey: {}", propKey);
-    assertTrue(propKey.getPath().endsWith(ZCONFIG + "/" + PROP_NODE_NAME));
+    assertTrue(propKey.getPath().endsWith(ZCONFIG));
     verify(context);
   }
 
@@ -68,8 +68,7 @@ public class PropStoreKeyTest {
   public void namespaceType() {
     var propKey = NamespacePropKey.of(instanceId, NamespaceId.of("a"));
     log.info("propKey: {}", propKey);
-    assertTrue(
-        propKey.getPath().endsWith(PROP_NODE_NAME) && propKey.getPath().contains(ZNAMESPACES));
+    assertTrue(propKey.getPath().endsWith(ZCONFIG) && propKey.getPath().contains(ZNAMESPACES));
     log.info("propKey: {}", propKey);
   }
 
@@ -80,8 +79,7 @@ public class PropStoreKeyTest {
     replay(context);
 
     var propKey = NamespacePropKey.of(context, NamespaceId.of("a"));
-    assertTrue(
-        propKey.getPath().endsWith(PROP_NODE_NAME) && propKey.getPath().contains(ZNAMESPACES));
+    assertTrue(propKey.getPath().endsWith(ZCONFIG) && propKey.getPath().contains(ZNAMESPACES));
     verify(context);
   }
 
@@ -89,7 +87,7 @@ public class PropStoreKeyTest {
   public void tableType() {
     var propKey = TablePropKey.of(instanceId, TableId.of("a"));
     log.info("propKey: {}", propKey);
-    assertTrue(propKey.getPath().endsWith(PROP_NODE_NAME) && propKey.getPath().contains(ZTABLES));
+    assertTrue(propKey.getPath().endsWith(ZCONFIG) && propKey.getPath().contains(ZTABLES));
     log.info("propKey: {}", propKey);
   }
 
@@ -98,28 +96,51 @@ public class PropStoreKeyTest {
 
     var iid = "3f9976c6-3bf1-41ab-9751-1b0a9be3551d";
 
-    PropStoreKey<?> t1 =
-        PropStoreKey.fromPath("/accumulo/" + iid + "/tables/t1/conf/encoded_props");
+    PropStoreKey<?> t1 = PropStoreKey.fromPath("/accumulo/" + iid + "/tables/t1" + ZCONFIG);
     assertNotNull(t1);
     assertEquals(TableId.of("t1"), t1.getId());
 
-    PropStoreKey<?> n1 =
-        PropStoreKey.fromPath("/accumulo/" + iid + "/namespaces/n1/conf/encoded_props");
+    PropStoreKey<?> n1 = PropStoreKey.fromPath("/accumulo/" + iid + "/namespaces/n1" + ZCONFIG);
     assertNotNull(n1);
     assertEquals(NamespaceId.of("n1"), n1.getId());
     assertNotNull(n1.getId());
 
-    PropStoreKey<?> s1 = PropStoreKey.fromPath("/accumulo/" + iid + "/config/encoded_props");
+    PropStoreKey<?> s1 = PropStoreKey.fromPath("/accumulo/" + iid + ZCONFIG);
     assertNotNull(s1);
     // system config returns instance id as id placeholder
     assertEquals(iid, s1.getId().canonical());
   }
 
+  @Test
+  public void invalidKeysTest() {
+    var iid = "3f9976c6-3bf1-41ab-9751-1b0a9be3551d";
+
+    // too short
+    assertNull(PropStoreKey.fromPath("/accumulo"));
+
+    // not a system config
+    assertTrue(PropStoreKey.fromPath("/accumulo/" + iid + ZCONFIG) instanceof SystemPropKey);
+    assertNull(PropStoreKey.fromPath("/foo"));
+    assertNull(PropStoreKey.fromPath("/accumulo/" + iid + "/foo"));
+    assertNull(PropStoreKey.fromPath("/accumulo/" + iid + ZCONFIG + "/foo"));
+
+    assertTrue(PropStoreKey
+        .fromPath("/accumulo/" + iid + ZTABLES + "/a" + ZCONFIG) instanceof TablePropKey);
+    assertNull(PropStoreKey.fromPath("/accumulo/" + iid + ZTABLES + ZCONFIG));
+    assertNull(PropStoreKey.fromPath("/accumulo/" + iid + "/invalid/a" + ZCONFIG));
+    assertNull(PropStoreKey.fromPath("/accumulo/" + iid + ZTABLES + "/a" + ZCONFIG + "/foo"));
+
+    assertTrue(PropStoreKey
+        .fromPath("/accumulo/" + iid + ZNAMESPACES + "/a" + ZCONFIG) instanceof NamespacePropKey);
+    assertNull(PropStoreKey.fromPath("/accumulo/" + iid + ZNAMESPACES + ZCONFIG));
+    assertNull(PropStoreKey.fromPath("/accumulo/" + iid + "/invalid/a" + ZCONFIG));
+    assertNull(PropStoreKey.fromPath("/accumulo/" + iid + ZNAMESPACES + "/a" + ZCONFIG + "/foo"));
+  }
+
   @Test
   public void getBasePathTest() {
-    assertTrue(SystemPropKey.of(instanceId).getBasePath().endsWith("/config"));
-    assertTrue(
-        NamespacePropKey.of(instanceId, NamespaceId.of("123")).getBasePath().endsWith("/conf"));
-    assertTrue(TablePropKey.of(instanceId, TableId.of("456")).getBasePath().endsWith("/conf"));
+    assertTrue(SystemPropKey.of(instanceId).getPath().endsWith("/config"));
+    assertTrue(NamespacePropKey.of(instanceId, NamespaceId.of("123")).getPath().endsWith(ZCONFIG));
+    assertTrue(TablePropKey.of(instanceId, TableId.of("456")).getPath().endsWith(ZCONFIG));
   }
 }
diff --git a/server/base/src/test/java/org/apache/accumulo/server/conf/store/impl/PropStoreEventTest.java b/server/base/src/test/java/org/apache/accumulo/server/conf/store/impl/PropStoreEventTest.java
index ab2e4cb45c..03fb902635 100644
--- a/server/base/src/test/java/org/apache/accumulo/server/conf/store/impl/PropStoreEventTest.java
+++ b/server/base/src/test/java/org/apache/accumulo/server/conf/store/impl/PropStoreEventTest.java
@@ -226,7 +226,7 @@ public class PropStoreEventTest {
 
     watcher.signalCacheChangeEvent(tablePropKey);
 
-    Thread.sleep(150);
+    Thread.sleep(250);
     assertEquals(1, listener.getCacheChangeEventCount());
   }
 
@@ -246,6 +246,7 @@ public class PropStoreEventTest {
       s.setCtime(System.currentTimeMillis());
       s.setMtime(System.currentTimeMillis());
       s.setVersion(12);
+      s.setDataLength(propCodec.toBytes(vProps).length);
       stat.setValue(s);
       return propCodec.toBytes(vProps);
     }).once();
diff --git a/server/base/src/test/java/org/apache/accumulo/server/conf/store/impl/ZooPropLoaderTest.java b/server/base/src/test/java/org/apache/accumulo/server/conf/store/impl/ZooPropLoaderTest.java
index 76de2b7780..2d020cb90c 100644
--- a/server/base/src/test/java/org/apache/accumulo/server/conf/store/impl/ZooPropLoaderTest.java
+++ b/server/base/src/test/java/org/apache/accumulo/server/conf/store/impl/ZooPropLoaderTest.java
@@ -30,6 +30,7 @@ import static org.easymock.EasyMock.createMock;
 import static org.easymock.EasyMock.eq;
 import static org.easymock.EasyMock.expect;
 import static org.easymock.EasyMock.expectLastCall;
+import static org.easymock.EasyMock.isA;
 import static org.easymock.EasyMock.newCapture;
 import static org.easymock.EasyMock.replay;
 import static org.easymock.EasyMock.verify;
@@ -108,10 +109,18 @@ public class ZooPropLoaderTest {
   @Test
   public void loadTest() throws Exception {
 
-    VersionedProperties defaultProps = new VersionedProperties();
-
-    expect(zrw.getData(eq(propStoreKey.getPath()), anyObject(), anyObject()))
-        .andReturn(propCodec.toBytes(defaultProps)).anyTimes();
+    final VersionedProperties defaultProps = new VersionedProperties();
+    final byte[] bytes = propCodec.toBytes(defaultProps);
+    Capture<Stat> stat = newCapture();
+    expect(zrw.getData(eq(propStoreKey.getPath()), anyObject(), capture(stat))).andAnswer(() -> {
+      Stat s = stat.getValue();
+      s.setCtime(System.currentTimeMillis());
+      s.setMtime(System.currentTimeMillis());
+      s.setVersion((int) defaultProps.getDataVersion());
+      s.setDataLength(bytes.length);
+      stat.setValue(s);
+      return (bytes);
+    }).once();
 
     cacheMetrics.addLoadTime(anyLong());
     expectLastCall().times(1);
@@ -132,12 +141,23 @@ public class ZooPropLoaderTest {
   @Test
   public void loadAndCacheTest() throws Exception {
 
-    VersionedProperties defaultProps = new VersionedProperties();
+    final VersionedProperties defaultProps = new VersionedProperties();
+    final byte[] bytes = propCodec.toBytes(defaultProps);
 
     expect(zrw.getStatus(propStoreKey.getPath())).andThrow(new KeeperException.NoNodeException())
         .anyTimes();
-    expect(zrw.getData(eq(propStoreKey.getPath()), anyObject(), anyObject()))
-        .andReturn(propCodec.toBytes(defaultProps)).once();
+
+    Capture<Stat> stat = newCapture();
+    expect(zrw.getData(eq(propStoreKey.getPath()), isA(PropStoreWatcher.class), capture(stat)))
+        .andAnswer(() -> {
+          Stat s = stat.getValue();
+          s.setCtime(System.currentTimeMillis());
+          s.setMtime(System.currentTimeMillis());
+          s.setVersion((int) defaultProps.getDataVersion());
+          s.setDataLength(bytes.length);
+          stat.setValue(s);
+          return (bytes);
+        }).once();
 
     cacheMetrics.addLoadTime(anyLong());
     expectLastCall().times(1);
@@ -192,9 +212,19 @@ public class ZooPropLoaderTest {
   public void expireTest() throws Exception {
 
     VersionedProperties defaultProps = new VersionedProperties();
+    byte[] bytes = propCodec.toBytes(defaultProps);
 
-    expect(zrw.getData(eq(propStoreKey.getPath()), anyObject(), anyObject()))
-        .andReturn(propCodec.toBytes(defaultProps)).times(2);
+    Capture<Stat> stat = newCapture();
+    expect(zrw.getData(eq(propStoreKey.getPath()), isA(PropStoreWatcher.class), capture(stat)))
+        .andAnswer(() -> {
+          Stat s = stat.getValue();
+          s.setCtime(System.currentTimeMillis());
+          s.setMtime(System.currentTimeMillis());
+          s.setVersion((int) defaultProps.getDataVersion());
+          s.setDataLength(bytes.length);
+          stat.setValue(s);
+          return (bytes);
+        }).times(2);
 
     cacheMetrics.addLoadTime(anyLong());
     expectLastCall().times(2);
@@ -227,9 +257,19 @@ public class ZooPropLoaderTest {
   public void reloadExceptionTest() throws Exception {
 
     final VersionedProperties defaultProps = new VersionedProperties();
+    final byte[] bytes = propCodec.toBytes(defaultProps);
 
-    expect(zrw.getData(eq(propStoreKey.getPath()), anyObject(), anyObject()))
-        .andReturn(propCodec.toBytes(defaultProps)).once();
+    Capture<Stat> stat = newCapture();
+    expect(zrw.getData(eq(propStoreKey.getPath()), isA(PropStoreWatcher.class), capture(stat)))
+        .andAnswer(() -> {
+          Stat s = stat.getValue();
+          s.setCtime(System.currentTimeMillis());
+          s.setMtime(System.currentTimeMillis());
+          s.setVersion((int) defaultProps.getDataVersion());
+          s.setDataLength(bytes.length);
+          stat.setValue(s);
+          return (bytes);
+        }).once();
 
     expect(zrw.getData(eq(propStoreKey.getPath()), anyObject(), anyObject()))
         .andThrow(new KeeperException.NoNodeException("forced no node")).anyTimes();
@@ -283,12 +323,32 @@ public class ZooPropLoaderTest {
     final var sysPropKey = SystemPropKey.of(instanceId);
     final var tablePropKey = TablePropKey.of(instanceId, TableId.of("t1"));
 
-    VersionedProperties defaultProps = new VersionedProperties();
+    final VersionedProperties defaultProps = new VersionedProperties();
+    final byte[] bytes = propCodec.toBytes(defaultProps);
 
-    expect(zrw.getData(eq(sysPropKey.getPath()), anyObject(), anyObject()))
-        .andReturn(propCodec.toBytes(defaultProps)).once();
-    expect(zrw.getData(eq(tablePropKey.getPath()), anyObject(), anyObject()))
-        .andReturn(propCodec.toBytes(defaultProps)).once();
+    Capture<Stat> stat = newCapture();
+    expect(zrw.getData(eq(sysPropKey.getPath()), isA(PropStoreWatcher.class), capture(stat)))
+        .andAnswer(() -> {
+          Stat s = stat.getValue();
+          s.setCtime(System.currentTimeMillis());
+          s.setMtime(System.currentTimeMillis());
+          s.setVersion((int) defaultProps.getDataVersion());
+          s.setDataLength(bytes.length);
+          stat.setValue(s);
+          return (bytes);
+        }).once();
+
+    Capture<Stat> stat1 = newCapture();
+    expect(zrw.getData(eq(tablePropKey.getPath()), isA(PropStoreWatcher.class), capture(stat1)))
+        .andAnswer(() -> {
+          Stat s = stat1.getValue();
+          s.setCtime(System.currentTimeMillis());
+          s.setMtime(System.currentTimeMillis());
+          s.setVersion((int) defaultProps.getDataVersion());
+          s.setDataLength(bytes.length);
+          stat1.setValue(s);
+          return (bytes);
+        }).once();
 
     cacheMetrics.addLoadTime(anyLong());
     expectLastCall().times(2);
@@ -314,12 +374,32 @@ public class ZooPropLoaderTest {
     final var sysPropKey = SystemPropKey.of(instanceId);
     final var tablePropKey = TablePropKey.of(instanceId, TableId.of("t1"));
 
-    VersionedProperties defaultProps = new VersionedProperties();
+    final VersionedProperties defaultProps = new VersionedProperties();
+    final byte[] bytes = propCodec.toBytes(defaultProps);
 
-    expect(zrw.getData(eq(sysPropKey.getPath()), anyObject(), anyObject()))
-        .andReturn(propCodec.toBytes(defaultProps)).once();
-    expect(zrw.getData(eq(tablePropKey.getPath()), anyObject(), anyObject()))
-        .andReturn(propCodec.toBytes(defaultProps)).once();
+    Capture<Stat> stat = newCapture();
+    expect(zrw.getData(eq(sysPropKey.getPath()), isA(PropStoreWatcher.class), capture(stat)))
+        .andAnswer(() -> {
+          Stat s = stat.getValue();
+          s.setCtime(System.currentTimeMillis());
+          s.setMtime(System.currentTimeMillis());
+          s.setVersion((int) defaultProps.getDataVersion());
+          s.setDataLength(bytes.length);
+          stat.setValue(s);
+          return (bytes);
+        }).once();
+
+    Capture<Stat> stat1 = newCapture();
+    expect(zrw.getData(eq(tablePropKey.getPath()), isA(PropStoreWatcher.class), capture(stat1)))
+        .andAnswer(() -> {
+          Stat s = stat1.getValue();
+          s.setCtime(System.currentTimeMillis());
+          s.setMtime(System.currentTimeMillis());
+          s.setVersion((int) defaultProps.getDataVersion());
+          s.setDataLength(bytes.length);
+          stat1.setValue(s);
+          return (bytes);
+        }).once();
 
     cacheMetrics.addLoadTime(anyLong());
     expectLastCall().times(2);
diff --git a/server/base/src/test/java/org/apache/accumulo/server/conf/store/impl/ZooPropStoreTest.java b/server/base/src/test/java/org/apache/accumulo/server/conf/store/impl/ZooPropStoreTest.java
index a65d868a51..f2e22f760b 100644
--- a/server/base/src/test/java/org/apache/accumulo/server/conf/store/impl/ZooPropStoreTest.java
+++ b/server/base/src/test/java/org/apache/accumulo/server/conf/store/impl/ZooPropStoreTest.java
@@ -27,6 +27,7 @@ import static org.easymock.EasyMock.createMock;
 import static org.easymock.EasyMock.eq;
 import static org.easymock.EasyMock.expect;
 import static org.easymock.EasyMock.expectLastCall;
+import static org.easymock.EasyMock.isA;
 import static org.easymock.EasyMock.newCapture;
 import static org.easymock.EasyMock.replay;
 import static org.easymock.EasyMock.verify;
@@ -119,8 +120,17 @@ public class ZooPropStoreTest {
     var vProps = new VersionedProperties(Map.of(Property.TABLE_BLOOM_ENABLED.getKey(), "true"));
 
     // expect one ZooKeeper call - subsequent calls should load from cache.
-    expect(zrw.getData(eq(propStoreKey.getPath()), anyObject(PropStoreWatcher.class), anyObject()))
-        .andReturn(VersionedPropCodec.getDefault().toBytes(vProps)).once();
+    Capture<Stat> stat = newCapture();
+    expect(zrw.getData(eq(propStoreKey.getPath()), isA(PropStoreWatcher.class), capture(stat)))
+        .andAnswer(() -> {
+          Stat s = stat.getValue();
+          s.setCtime(System.currentTimeMillis());
+          s.setMtime(System.currentTimeMillis());
+          s.setVersion((int) vProps.getDataVersion());
+          s.setDataLength(propCodec.toBytes(vProps).length);
+          stat.setValue(s);
+          return propCodec.toBytes(vProps);
+        }).once();
 
     replay(context, zrw);
 
@@ -152,6 +162,7 @@ public class ZooPropStoreTest {
           s.setCtime(System.currentTimeMillis());
           s.setMtime(System.currentTimeMillis());
           s.setVersion(expectedVersion);
+          s.setDataLength(propCodec.toBytes(new VersionedProperties(props)).length);
           stat.setValue(s);
           return propCodec.toBytes(new VersionedProperties(props));
         }).once();
@@ -178,9 +189,18 @@ public class ZooPropStoreTest {
     var initialProps = new VersionedProperties(0, Instant.now(),
         Map.of(TABLE_BULK_MAX_TABLETS.getKey(), "1234", TABLE_FILE_BLOCK_SIZE.getKey(), "512M"));
 
-    // not cached - will load from ZooKeeper
-    expect(zrw.getData(eq(propStoreKey.getPath()), anyObject(Stat.class)))
-        .andReturn(propCodec.toBytes(initialProps)).once();
+    // not cached - first will load from ZooKeeper
+    Capture<Stat> stat = newCapture();
+
+    expect(zrw.getData(eq(propStoreKey.getPath()), capture(stat))).andAnswer(() -> {
+      Stat s = stat.getValue();
+      s.setCtime(System.currentTimeMillis());
+      s.setMtime(System.currentTimeMillis());
+      s.setVersion((int) initialProps.getDataVersion());
+      s.setDataLength(propCodec.toBytes(initialProps).length);
+      stat.setValue(s);
+      return propCodec.toBytes(initialProps);
+    }).once();
 
     Capture<byte[]> bytes = newCapture();
     expect(zrw.overwritePersistentData(eq(propStoreKey.getPath()), capture(bytes), eq(0)))
@@ -222,6 +242,7 @@ public class ZooPropStoreTest {
     expect(zrw.getData(eq(propStoreKey.getPath()), capture(stat))).andAnswer(() -> {
       Stat s = stat.getValue();
       s.setVersion(123);
+      s.setDataLength(propCodec.toBytes(initialProps).length);
       stat.setValue(s);
       return propCodec.toBytes(initialProps);
     }).once();
@@ -264,6 +285,7 @@ public class ZooPropStoreTest {
       s.setMtime(System.currentTimeMillis());
       s.setCzxid(1234);
       s.setVersion(19);
+      s.setDataLength(12345);
       stat.setValue(s);
       return new byte[100];
     }).once();
@@ -373,8 +395,18 @@ public class ZooPropStoreTest {
     var vProps = new VersionedProperties(Map.of(Property.TABLE_BLOOM_ENABLED.getKey(), "true"));
 
     // expect first call to load cache.
-    expect(zrw.getData(eq(propStoreKey.getPath()), anyObject(PropStoreWatcher.class), anyObject()))
-        .andReturn(VersionedPropCodec.getDefault().toBytes(vProps)).once();
+    // expect one ZooKeeper call - subsequent calls should load from cache.
+    Capture<Stat> stat = newCapture();
+    expect(zrw.getData(eq(propStoreKey.getPath()), isA(PropStoreWatcher.class), capture(stat)))
+        .andAnswer(() -> {
+          Stat s = stat.getValue();
+          s.setCtime(System.currentTimeMillis());
+          s.setMtime(System.currentTimeMillis());
+          s.setVersion((int) vProps.getDataVersion());
+          s.setDataLength(propCodec.toBytes(vProps).length);
+          stat.setValue(s);
+          return propCodec.toBytes(vProps);
+        }).once();
 
     zrw.delete(eq(propStoreKey.getPath()));
     expectLastCall().once();
diff --git a/server/base/src/test/java/org/apache/accumulo/server/conf/util/ConfigPropertyUpgraderTest.java b/server/base/src/test/java/org/apache/accumulo/server/conf/util/ConfigPropertyUpgraderTest.java
new file mode 100644
index 0000000000..025a035403
--- /dev/null
+++ b/server/base/src/test/java/org/apache/accumulo/server/conf/util/ConfigPropertyUpgraderTest.java
@@ -0,0 +1,102 @@
+/*
+ * 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.server.conf.util;
+
+import static org.easymock.EasyMock.anyString;
+import static org.easymock.EasyMock.createMock;
+import static org.easymock.EasyMock.expect;
+import static org.easymock.EasyMock.replay;
+import static org.easymock.EasyMock.verify;
+
+import java.util.List;
+import java.util.UUID;
+
+import org.apache.accumulo.core.Constants;
+import org.apache.accumulo.core.data.InstanceId;
+import org.apache.accumulo.core.data.NamespaceId;
+import org.apache.accumulo.core.data.TableId;
+import org.apache.accumulo.fate.zookeeper.ZooReaderWriter;
+import org.apache.accumulo.fate.zookeeper.ZooUtil;
+import org.apache.accumulo.server.conf.codec.VersionedProperties;
+import org.apache.accumulo.server.conf.store.NamespacePropKey;
+import org.apache.accumulo.server.conf.store.SystemPropKey;
+import org.apache.accumulo.server.conf.store.TablePropKey;
+import org.junit.jupiter.api.Test;
+
+class ConfigPropertyUpgraderTest {
+
+  @Test
+  void upgradeSysProps() {
+
+    InstanceId iid = InstanceId.of(UUID.randomUUID());
+
+    ConfigPropertyUpgrader upgrader = new ConfigPropertyUpgrader();
+    ConfigTransformer transformer = createMock(ConfigTransformer.class);
+    expect(transformer.transform(SystemPropKey.of(iid), SystemPropKey.of(iid).getPath(), false))
+        .andReturn(new VersionedProperties()).once();
+
+    replay(transformer);
+    upgrader.upgradeSysProps(iid, transformer);
+    verify(transformer);
+  }
+
+  @Test
+  void upgradeNamespaceProps() throws Exception {
+
+    InstanceId iid = InstanceId.of(UUID.randomUUID());
+
+    ConfigPropertyUpgrader upgrader = new ConfigPropertyUpgrader();
+
+    ConfigTransformer transformer = createMock(ConfigTransformer.class);
+    String nsRoot = ZooUtil.getRoot(iid) + Constants.ZNAMESPACES;
+    expect(transformer.transform(NamespacePropKey.of(iid, NamespaceId.of("a")),
+        nsRoot + "/a" + Constants.ZCONF_LEGACY, true)).andReturn(new VersionedProperties()).once();
+    expect(transformer.transform(NamespacePropKey.of(iid, NamespaceId.of("b")),
+        nsRoot + "/b" + Constants.ZCONF_LEGACY, true)).andReturn(new VersionedProperties()).once();
+
+    ZooReaderWriter zrw = createMock(ZooReaderWriter.class);
+    expect(zrw.getChildren(anyString())).andReturn(List.of("a", "b")).once();
+
+    replay(transformer, zrw);
+    upgrader.upgradeNamespaceProps(iid, zrw, transformer);
+    verify(transformer, zrw);
+  }
+
+  @Test
+  void upgradeTableProps() throws Exception {
+
+    InstanceId iid = InstanceId.of(UUID.randomUUID());
+
+    ConfigPropertyUpgrader upgrader = new ConfigPropertyUpgrader();
+
+    ConfigTransformer transformer = createMock(ConfigTransformer.class);
+    String nsRoot = ZooUtil.getRoot(iid) + Constants.ZTABLES;
+    expect(transformer.transform(TablePropKey.of(iid, TableId.of("a")),
+        nsRoot + "/a" + Constants.ZCONF_LEGACY, true)).andReturn(new VersionedProperties()).once();
+    expect(transformer.transform(TablePropKey.of(iid, TableId.of("b")),
+        nsRoot + "/b" + Constants.ZCONF_LEGACY, true)).andReturn(new VersionedProperties()).once();
+
+    ZooReaderWriter zrw = createMock(ZooReaderWriter.class);
+    expect(zrw.getChildren(anyString())).andReturn(List.of("a", "b")).once();
+
+    replay(transformer, zrw);
+    upgrader.upgradeTableProps(iid, zrw, transformer);
+    verify(transformer, zrw);
+  }
+}
diff --git a/server/base/src/test/java/org/apache/accumulo/server/conf/util/ZooInfoViewerTest.java b/server/base/src/test/java/org/apache/accumulo/server/conf/util/ZooInfoViewerTest.java
index b4a1e35224..5f4a807540 100644
--- a/server/base/src/test/java/org/apache/accumulo/server/conf/util/ZooInfoViewerTest.java
+++ b/server/base/src/test/java/org/apache/accumulo/server/conf/util/ZooInfoViewerTest.java
@@ -26,10 +26,12 @@ import static org.apache.accumulo.core.Constants.ZROOT;
 import static org.apache.accumulo.core.Constants.ZTABLES;
 import static org.apache.accumulo.core.Constants.ZTABLE_NAME;
 import static org.apache.accumulo.core.Constants.ZTABLE_NAMESPACE;
-import static org.easymock.EasyMock.anyObject;
+import static org.easymock.EasyMock.capture;
 import static org.easymock.EasyMock.createMock;
 import static org.easymock.EasyMock.eq;
 import static org.easymock.EasyMock.expect;
+import static org.easymock.EasyMock.isA;
+import static org.easymock.EasyMock.newCapture;
 import static org.easymock.EasyMock.replay;
 import static org.easymock.EasyMock.verify;
 import static org.junit.jupiter.api.Assertions.assertEquals;
@@ -54,8 +56,9 @@ import org.apache.accumulo.server.conf.codec.VersionedProperties;
 import org.apache.accumulo.server.conf.store.NamespacePropKey;
 import org.apache.accumulo.server.conf.store.SystemPropKey;
 import org.apache.accumulo.server.conf.store.TablePropKey;
-import org.apache.zookeeper.Watcher;
+import org.apache.accumulo.server.conf.store.impl.PropStoreWatcher;
 import org.apache.zookeeper.data.Stat;
+import org.easymock.Capture;
 import org.junit.jupiter.api.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -298,8 +301,17 @@ public class ZooInfoViewerTest {
 
     var sysPropBytes = propCodec
         .toBytes(new VersionedProperties(123, Instant.now(), Map.of("s1", "sv1", "s2", "sv2")));
-    expect(zooReader.getData(eq(SystemPropKey.of(iid).getNodePath()), anyObject(Watcher.class),
-        anyObject(Stat.class))).andReturn(sysPropBytes).anyTimes();
+    Capture<Stat> sStat = newCapture();
+    expect(zooReader.getData(eq(SystemPropKey.of(iid).getPath()), isA(PropStoreWatcher.class),
+        capture(sStat))).andAnswer(() -> {
+          Stat s = sStat.getValue();
+          s.setCtime(System.currentTimeMillis());
+          s.setMtime(System.currentTimeMillis());
+          s.setVersion(0); // default version
+          s.setDataLength(sysPropBytes.length);
+          sStat.setValue(s);
+          return sysPropBytes;
+        }).once();
 
     var nsBasePath = ZooUtil.getRoot(iid) + ZNAMESPACES;
     expect(zooReader.getChildren(nsBasePath)).andReturn(List.of("a")).anyTimes();
@@ -308,25 +320,45 @@ public class ZooInfoViewerTest {
     var nsPropBytes =
         propCodec.toBytes(new VersionedProperties(123, Instant.now(), Map.of("n1", "nv1")));
     NamespaceId nsId = NamespaceId.of("a");
-    expect(zooReader.getData(eq(NamespacePropKey.of(iid, nsId).getNodePath()),
-        anyObject(Watcher.class), anyObject(Stat.class))).andReturn(nsPropBytes).anyTimes();
+    Capture<Stat> nsStat = newCapture();
+    expect(zooReader.getData(eq(NamespacePropKey.of(iid, nsId).getPath()),
+        isA(PropStoreWatcher.class), capture(nsStat))).andAnswer(() -> {
+          Stat s = nsStat.getValue();
+          s.setCtime(System.currentTimeMillis());
+          s.setMtime(System.currentTimeMillis());
+          s.setVersion(0); // default version
+          s.setDataLength(nsPropBytes.length);
+          nsStat.setValue(s);
+          return nsPropBytes;
+        }).once();
 
     var tBasePath = ZooUtil.getRoot(iid) + ZTABLES;
     expect(zooReader.getChildren(tBasePath)).andReturn(List.of("t")).anyTimes();
     expect(zooReader.getData(eq(tBasePath + "/t" + ZTABLE_NAME)))
         .andReturn("t_table".getBytes(UTF_8)).anyTimes();
-    var tPropBytes =
-        propCodec.toBytes(new VersionedProperties(123, Instant.now(), Map.of("t1", "tv1")));
+
+    var tProps = new VersionedProperties(123, Instant.now(), Map.of("t1", "tv1"));
+    var tPropBytes = propCodec.toBytes(tProps);
     TableId tid = TableId.of("t");
-    expect(zooReader.getData(eq(TablePropKey.of(iid, tid).getNodePath()), anyObject(Watcher.class),
-        anyObject(Stat.class))).andReturn(tPropBytes).anyTimes();
+    Capture<Stat> stat = newCapture();
+    expect(zooReader.getData(eq(TablePropKey.of(iid, tid).getPath()), isA(PropStoreWatcher.class),
+        capture(stat))).andAnswer(() -> {
+          Stat s = stat.getValue();
+          s.setCtime(System.currentTimeMillis());
+          s.setMtime(System.currentTimeMillis());
+          s.setVersion((int) tProps.getDataVersion());
+          s.setDataLength(tPropBytes.length);
+          stat.setValue(s);
+          return tPropBytes;
+        }).once();
+
     expect(zooReader.getData(tBasePath + "/t" + ZTABLE_NAMESPACE))
         .andReturn("+default".getBytes(UTF_8)).anyTimes();
 
     replay(zooReader);
 
     NamespacePropKey nsKey = NamespacePropKey.of(iid, nsId);
-    log.trace("namespace base path: {}", nsKey.getBasePath());
+    log.trace("namespace base path: {}", nsKey.getPath());
 
     String testFileName = "./target/zoo-info-viewer-" + System.currentTimeMillis() + ".txt";
 
diff --git a/test/src/main/java/org/apache/accumulo/test/conf/PropStoreConfigIT.java b/test/src/main/java/org/apache/accumulo/test/conf/PropStoreConfigIT.java
index 9afd650ba3..1e20fc7943 100644
--- a/test/src/main/java/org/apache/accumulo/test/conf/PropStoreConfigIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/conf/PropStoreConfigIT.java
@@ -173,7 +173,7 @@ public class PropStoreConfigIT extends AccumuloClusterHarness {
       assertTrue(
           noAcl.get(0).toString().contains("world") || noAcl.get(1).toString().contains("world"));
 
-      var sysAcl = zrw.getACL(SystemPropKey.of(serverContext).getNodePath());
+      var sysAcl = zrw.getACL(SystemPropKey.of(serverContext).getPath());
       assertEquals(1, sysAcl.size());
       assertFalse(sysAcl.get(0).toString().contains("world"));
 
@@ -181,7 +181,8 @@ public class PropStoreConfigIT extends AccumuloClusterHarness {
           .entrySet()) {
         log.debug("Check acl on namespace name: {}, id: {}", nsEntry.getKey(), nsEntry.getValue());
         var namespaceAcl = zrw.getACL(
-            NamespacePropKey.of(serverContext, NamespaceId.of(nsEntry.getValue())).getNodePath());
+            NamespacePropKey.of(serverContext, NamespaceId.of(nsEntry.getValue())).getPath());
+        log.debug("namespace permissions: {}", namespaceAcl);
         assertEquals(1, namespaceAcl.size());
         assertFalse(namespaceAcl.get(0).toString().contains("world"));
       }
@@ -189,7 +190,8 @@ public class PropStoreConfigIT extends AccumuloClusterHarness {
       for (Map.Entry<String,String> tEntry : client.tableOperations().tableIdMap().entrySet()) {
         log.debug("Check acl on table name: {}, id: {}", tEntry.getKey(), tEntry.getValue());
         var tableAcl =
-            zrw.getACL(TablePropKey.of(serverContext, TableId.of(tEntry.getValue())).getNodePath());
+            zrw.getACL(TablePropKey.of(serverContext, TableId.of(tEntry.getValue())).getPath());
+        log.debug("Received ACLs of: {}", tableAcl);
         assertEquals(1, tableAcl.size());
         assertFalse(tableAcl.get(0).toString().contains("world"));
       }
diff --git a/test/src/main/java/org/apache/accumulo/test/conf/util/ConfigTransformerIT.java b/test/src/main/java/org/apache/accumulo/test/conf/util/ConfigTransformerIT.java
index 0e8d7c0d43..2da71350d7 100644
--- a/test/src/main/java/org/apache/accumulo/test/conf/util/ConfigTransformerIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/conf/util/ConfigTransformerIT.java
@@ -131,14 +131,14 @@ public class ConfigTransformerIT {
 
     var sysPropKey = SystemPropKey.of(instanceId);
 
-    List<String> sysLegacy = zrw.getChildren(sysPropKey.getBasePath());
+    List<String> sysLegacy = zrw.getChildren(sysPropKey.getPath());
     log.info("Before: {}", sysLegacy);
 
     var vProps = propStore.get(sysPropKey);
     assertNotNull(vProps);
     log.info("Converted: {}", vProps);
 
-    sysLegacy = zrw.getChildren(sysPropKey.getBasePath());
+    sysLegacy = zrw.getChildren(sysPropKey.getPath());
     log.info("After: {}", sysLegacy);
 
   }
@@ -149,10 +149,10 @@ public class ConfigTransformerIT {
     var sysPropKey = SystemPropKey.of(instanceId);
 
     ConfigTransformer transformer = new ConfigTransformer(zrw, codec, watcher);
-    List<String> sysLegacy = zrw.getChildren(sysPropKey.getBasePath());
+    List<String> sysLegacy = zrw.getChildren(sysPropKey.getPath());
     log.info("Before: {}", sysLegacy);
 
-    var converted = transformer.transform(sysPropKey);
+    var converted = transformer.transform(sysPropKey, sysPropKey.getPath(), false);
 
     assertEquals(sysLegacy.size(), converted.asMap().size());
   }
@@ -167,9 +167,10 @@ public class ConfigTransformerIT {
 
     ConfigTransformer transformer = new ConfigTransformer(zrw, codec, watcher, retry);
     // manually create a lock so transformer fails
-    zrw.putEphemeralData(sysPropKey.getBasePath() + TransformToken.TRANSFORM_TOKEN, new byte[0]);
+    zrw.putEphemeralData(sysPropKey.getPath() + TransformToken.TRANSFORM_TOKEN, new byte[0]);
 
-    assertThrows(IllegalStateException.class, () -> transformer.transform(sysPropKey));
+    assertThrows(IllegalStateException.class,
+        () -> transformer.transform(sysPropKey, sysPropKey.getPath(), false));
 
   }
 
diff --git a/test/src/main/java/org/apache/accumulo/test/conf/util/TransformTokenIT.java b/test/src/main/java/org/apache/accumulo/test/conf/util/TransformTokenIT.java
index 66c458dff5..a97b4a568a 100644
--- a/test/src/main/java/org/apache/accumulo/test/conf/util/TransformTokenIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/conf/util/TransformTokenIT.java
@@ -112,18 +112,18 @@ public class TransformTokenIT {
 
     var sysPropKey = SystemPropKey.of(instanceId);
 
-    TransformToken token = TransformToken.createToken(sysPropKey, zrw);
+    TransformToken token = TransformToken.createToken(sysPropKey.getPath(), zrw);
 
     assertTrue(token.haveTokenOwnership());
     token.releaseToken();
     assertFalse(token.haveTokenOwnership());
 
     // relock by getting a new lock
-    TransformToken lock2 = TransformToken.createToken(sysPropKey, zrw);
+    TransformToken lock2 = TransformToken.createToken(sysPropKey.getPath(), zrw);
     assertTrue(lock2.haveTokenOwnership());
 
     // fail with a current lock node present
-    TransformToken lock3 = TransformToken.createToken(sysPropKey, zrw);
+    TransformToken lock3 = TransformToken.createToken(sysPropKey.getPath(), zrw);
     assertFalse(lock3.haveTokenOwnership());
     // and confirm lock still present
     assertTrue(lock2.haveTokenOwnership());
@@ -135,9 +135,9 @@ public class TransformTokenIT {
     replay(context, watcher);
 
     var sysPropKey = SystemPropKey.of(instanceId);
-    var tokenPath = sysPropKey.getBasePath() + TransformToken.TRANSFORM_TOKEN;
+    var tokenPath = sysPropKey.getPath() + TransformToken.TRANSFORM_TOKEN;
 
-    TransformToken lock = TransformToken.createToken(sysPropKey, zrw);
+    TransformToken lock = TransformToken.createToken(sysPropKey.getPath(), zrw);
 
     // force change in lock
     assertTrue(lock.haveTokenOwnership());
@@ -147,7 +147,7 @@ public class TransformTokenIT {
 
     // clean-up and get new lock
     zrw.delete(tokenPath);
-    TransformToken lock3 = TransformToken.createToken(sysPropKey, zrw);
+    TransformToken lock3 = TransformToken.createToken(sysPropKey.getPath(), zrw);
     assertTrue(lock3.haveTokenOwnership());
     zrw.delete(tokenPath);
     assertThrows(IllegalStateException.class, lock::releaseToken,