You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@accumulo.apache.org by ct...@apache.org on 2022/06/01 21:50:08 UTC

[accumulo] branch main updated: Clear property caches on RPC request for props (#2740)

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

ctubbsii pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/accumulo.git


The following commit(s) were added to refs/heads/main by this push:
     new 48165863bf Clear property caches on RPC request for props (#2740)
48165863bf is described below

commit 48165863bf359b2e0ddf4cddbcc6f33f7ffe4ce7
Author: Christopher Tubbs <ct...@apache.org>
AuthorDate: Wed Jun 1 17:50:02 2022 -0400

    Clear property caches on RPC request for props (#2740)
    
    Mitigation for test instability in #2739
    * Clear namespace, table, and system property cache entries when that
      particular configuration is being requested from the client
    
    API improvements for PropStore
    * Rename PropCacheKey to PropStoreKey, since it's a first-class citizen
      of the PropStore interface
    * Add getCache() method to PropStore, because it is expected that
      PropStores are caching, so this makes it explicit
    * Use .getCache().remove() instead of casting to ZooPropStore and
      simulating a ZK change event to remove an item from the cache to force
      it to reload
---
 .../server/client/ClientServiceHandler.java        |   6 ++
 .../server/conf/NamespaceConfiguration.java        |   6 +-
 .../accumulo/server/conf/SystemConfiguration.java  |   4 +-
 .../server/conf/ZooBasedConfiguration.java         |  22 ++--
 .../server/conf/store/NamespacePropKey.java        |   2 +-
 .../accumulo/server/conf/store/PropCache.java      |   7 +-
 .../server/conf/store/PropChangeListener.java      |   6 +-
 .../accumulo/server/conf/store/PropStore.java      |  32 +++---
 .../store/{PropCacheKey.java => PropStoreKey.java} |  16 +--
 .../accumulo/server/conf/store/SystemPropKey.java  |   2 +-
 .../accumulo/server/conf/store/TablePropKey.java   |   2 +-
 .../conf/store/impl/PropCacheCaffeineImpl.java     |  31 +++---
 .../server/conf/store/impl/PropStoreEventTask.java |  30 +++---
 .../server/conf/store/impl/PropStoreWatcher.java   |  62 +++++------
 .../server/conf/store/impl/ZooPropLoader.java      |  38 +++----
 .../server/conf/store/impl/ZooPropStore.java       | 114 +++++++++++----------
 .../server/conf/util/ConfigTransformer.java        |  78 +++++++-------
 .../accumulo/server/conf/util/PropSnapshot.java    |  30 +++---
 .../accumulo/server/conf/util/TransformToken.java  |   8 +-
 .../server/conf/NamespaceConfigurationTest.java    |   6 +-
 ...PropCacheKeyTest.java => PropStoreKeyTest.java} |  16 +--
 .../server/conf/store/impl/PropStoreEventTest.java |   8 +-
 .../server/conf/store/impl/ZooPropLoaderTest.java  | 100 +++++++++---------
 .../server/conf/store/impl/ZooPropStoreTest.java   |  66 ++++++------
 .../test/conf/store/PropCacheCaffeineImplZkIT.java |   6 +-
 .../test/conf/store/PropStoreZooKeeperIT.java      |  22 ++--
 .../accumulo/test/conf/store/ZooBasedConfigIT.java |  14 +--
 27 files changed, 376 insertions(+), 358 deletions(-)

diff --git a/server/base/src/main/java/org/apache/accumulo/server/client/ClientServiceHandler.java b/server/base/src/main/java/org/apache/accumulo/server/client/ClientServiceHandler.java
index 440cac12ce..5ca421a5ba 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/client/ClientServiceHandler.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/client/ClientServiceHandler.java
@@ -60,6 +60,9 @@ import org.apache.accumulo.core.security.TablePermission;
 import org.apache.accumulo.core.securityImpl.thrift.TCredentials;
 import org.apache.accumulo.core.trace.thrift.TInfo;
 import org.apache.accumulo.server.ServerContext;
+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.accumulo.server.security.SecurityOperation;
 import org.apache.accumulo.server.util.ServerBulkImportStatus;
 import org.apache.accumulo.server.util.TableDiskUsage;
@@ -305,6 +308,7 @@ public class ClientServiceHandler implements ClientService.Iface {
       ConfigurationType type) throws TException {
     switch (type) {
       case CURRENT:
+        context.getPropStore().getCache().remove(SystemPropKey.of(context));
         return conf(credentials, context.getConfiguration());
       case SITE:
         return conf(credentials, context.getSiteConfiguration());
@@ -318,6 +322,7 @@ public class ClientServiceHandler implements ClientService.Iface {
   public Map<String,String> getTableConfiguration(TInfo tinfo, TCredentials credentials,
       String tableName) throws TException, ThriftTableOperationException {
     TableId tableId = checkTableId(context, tableName, null);
+    context.getPropStore().getCache().remove(TablePropKey.of(context, tableId));
     AccumuloConfiguration config = context.getTableConfiguration(tableId);
     return conf(credentials, config);
   }
@@ -458,6 +463,7 @@ public class ClientServiceHandler implements ClientService.Iface {
       throw new ThriftTableOperationException(null, ns, null,
           TableOperationExceptionType.NAMESPACE_NOTFOUND, why);
     }
+    context.getPropStore().getCache().remove(NamespacePropKey.of(context, namespaceId));
     AccumuloConfiguration config = context.getNamespaceConfiguration(namespaceId);
     return conf(credentials, config);
   }
diff --git a/server/base/src/main/java/org/apache/accumulo/server/conf/NamespaceConfiguration.java b/server/base/src/main/java/org/apache/accumulo/server/conf/NamespaceConfiguration.java
index 2d8b19d615..e60ff220ab 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/conf/NamespaceConfiguration.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/conf/NamespaceConfiguration.java
@@ -45,7 +45,7 @@ public class NamespaceConfiguration extends ZooBasedConfiguration {
 
     String key = property.getKey();
 
-    var namespaceId = getPropCacheKey().getId();
+    var namespaceId = getPropStoreKey().getId();
     if (namespaceId != null && namespaceId.equals(Namespace.ACCUMULO.id())
         && isIteratorOrConstraint(key)) {
       // ignore iterators from parent if system namespace
@@ -81,10 +81,10 @@ public class NamespaceConfiguration extends ZooBasedConfiguration {
   }
 
   protected NamespaceId getNamespaceId() {
-    NamespaceId id = (NamespaceId) getPropCacheKey().getId();
+    NamespaceId id = (NamespaceId) getPropStoreKey().getId();
     if (id == null) {
       throw new IllegalArgumentException(
-          "Invalid request for namespace id on " + getPropCacheKey());
+          "Invalid request for namespace id on " + getPropStoreKey());
     }
     return id;
   }
diff --git a/server/base/src/main/java/org/apache/accumulo/server/conf/SystemConfiguration.java b/server/base/src/main/java/org/apache/accumulo/server/conf/SystemConfiguration.java
index 661f3fbcae..9e2429e218 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/conf/SystemConfiguration.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/conf/SystemConfiguration.java
@@ -31,9 +31,9 @@ public class SystemConfiguration extends ZooBasedConfiguration {
 
   private final RuntimeFixedProperties runtimeFixedProps;
 
-  public SystemConfiguration(ServerContext context, SystemPropKey propCacheKey,
+  public SystemConfiguration(ServerContext context, SystemPropKey propStoreKey,
       AccumuloConfiguration parent) {
-    super(log, context, propCacheKey, parent);
+    super(log, context, propStoreKey, parent);
     runtimeFixedProps = new RuntimeFixedProperties(getSnapshot(), context.getSiteConfiguration());
   }
 
diff --git a/server/base/src/main/java/org/apache/accumulo/server/conf/ZooBasedConfiguration.java b/server/base/src/main/java/org/apache/accumulo/server/conf/ZooBasedConfiguration.java
index 2a847af26b..9f5c57719b 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/conf/ZooBasedConfiguration.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/conf/ZooBasedConfiguration.java
@@ -26,7 +26,7 @@ import java.util.function.Predicate;
 import org.apache.accumulo.core.conf.AccumuloConfiguration;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.server.ServerContext;
-import org.apache.accumulo.server.conf.store.PropCacheKey;
+import org.apache.accumulo.server.conf.store.PropStoreKey;
 import org.apache.accumulo.server.conf.util.PropSnapshot;
 import org.checkerframework.checker.nullness.qual.NonNull;
 import org.checkerframework.checker.nullness.qual.Nullable;
@@ -49,23 +49,23 @@ public class ZooBasedConfiguration extends AccumuloConfiguration {
 
   protected final Logger log;
   private final AccumuloConfiguration parent;
-  private final PropCacheKey<?> propCacheKey;
+  private final PropStoreKey<?> propStoreKey;
 
   private final PropSnapshot propSnapshot;
 
-  public ZooBasedConfiguration(Logger log, ServerContext context, PropCacheKey<?> propCacheKey,
+  public ZooBasedConfiguration(Logger log, ServerContext context, PropStoreKey<?> propStoreKey,
       AccumuloConfiguration parent) {
     this.log = requireNonNull(log, "a Logger must be supplied");
     requireNonNull(context, "the context cannot be null");
-    this.propCacheKey = requireNonNull(propCacheKey, "a PropCacheId must be supplied");
+    this.propStoreKey = requireNonNull(propStoreKey, "a PropCacheId must be supplied");
     this.parent = requireNonNull(parent, "An AccumuloConfiguration parent must be supplied");
 
-    propSnapshot = PropSnapshot.create(propCacheKey, context.getPropStore());
+    propSnapshot = PropSnapshot.create(propStoreKey, context.getPropStore());
   }
 
   @VisibleForTesting
-  public void zkChangeEvent(PropCacheKey<?> propCacheKey) {
-    propSnapshot.zkChangeEvent(propCacheKey);
+  public void zkChangeEvent(PropStoreKey<?> propStoreKey) {
+    propSnapshot.zkChangeEvent(propStoreKey);
   }
 
   public long getDataVersion() {
@@ -93,7 +93,7 @@ public class ZooBasedConfiguration extends AccumuloConfiguration {
       count += dataVersion;
     }
 
-    log.trace("update count result for: {} - data version: {} update: {}", propCacheKey,
+    log.trace("update count result for: {} - data version: {} update: {}", propStoreKey,
         dataVersion, count);
     return count;
   }
@@ -103,8 +103,8 @@ public class ZooBasedConfiguration extends AccumuloConfiguration {
     return parent;
   }
 
-  public PropCacheKey<?> getPropCacheKey() {
-    return propCacheKey;
+  public PropStoreKey<?> getPropStoreKey() {
+    return propStoreKey;
   }
 
   @Override
@@ -128,7 +128,7 @@ public class ZooBasedConfiguration extends AccumuloConfiguration {
 
     Map<String,String> theseProps = getSnapshot();
 
-    log.trace("getProperties() for: {} filter: {}, have: {}, passed: {}", getPropCacheKey(), filter,
+    log.trace("getProperties() for: {} filter: {}, have: {}, passed: {}", getPropStoreKey(), filter,
         theseProps, props);
 
     for (Map.Entry<String,String> p : theseProps.entrySet()) {
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 b13fb4d8b8..49fd279a02 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
@@ -28,7 +28,7 @@ import org.apache.accumulo.fate.zookeeper.ZooUtil;
 import org.apache.accumulo.server.ServerContext;
 import org.checkerframework.checker.nullness.qual.NonNull;
 
-public class NamespacePropKey extends PropCacheKey<NamespaceId> {
+public class NamespacePropKey extends PropStoreKey<NamespaceId> {
 
   private NamespacePropKey(final InstanceId instanceId, final String path, final NamespaceId id) {
     super(instanceId, path, id);
diff --git a/server/base/src/main/java/org/apache/accumulo/server/conf/store/PropCache.java b/server/base/src/main/java/org/apache/accumulo/server/conf/store/PropCache.java
index 9d50189a17..9099e7fb85 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/conf/store/PropCache.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/conf/store/PropCache.java
@@ -24,13 +24,10 @@ import org.checkerframework.checker.nullness.qual.Nullable;
 public interface PropCache {
 
   @Nullable
-  VersionedProperties get(final PropCacheKey<?> propCacheKey);
+  VersionedProperties get(final PropStoreKey<?> propStoreKey);
 
-  void remove(final PropCacheKey<?> propCacheKey);
+  void remove(final PropStoreKey<?> propStoreKey);
 
   void removeAll();
 
-  @Nullable
-  VersionedProperties getWithoutCaching(PropCacheKey<?> propCacheKey);
-
 }
diff --git a/server/base/src/main/java/org/apache/accumulo/server/conf/store/PropChangeListener.java b/server/base/src/main/java/org/apache/accumulo/server/conf/store/PropChangeListener.java
index 4e4b010cf2..409629951d 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/conf/store/PropChangeListener.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/conf/store/PropChangeListener.java
@@ -23,18 +23,18 @@ public interface PropChangeListener {
   /**
    * Signal that a ZooKeeper data change event occurred and that the data has changed.
    */
-  void zkChangeEvent(final PropCacheKey<?> propCacheKey);
+  void zkChangeEvent(final PropStoreKey<?> propStoreKey);
 
   /**
    * Signal that a cache change event occurred - cache change events occur on eviction or
    * invalidation of the cache entry. The underlying data may or may not have changed.
    */
-  void cacheChangeEvent(final PropCacheKey<?> propCacheKey);
+  void cacheChangeEvent(final PropStoreKey<?> propStoreKey);
 
   /**
    * Signal that the node had been deleted from ZooKeeper.
    */
-  void deleteEvent(final PropCacheKey<?> propCacheKey);
+  void deleteEvent(final PropStoreKey<?> propStoreKey);
 
   /**
    * A ZooKeeper connection event (session closed, expired...) and that
diff --git a/server/base/src/main/java/org/apache/accumulo/server/conf/store/PropStore.java b/server/base/src/main/java/org/apache/accumulo/server/conf/store/PropStore.java
index 8315c22c33..582cafb5b9 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/conf/store/PropStore.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/conf/store/PropStore.java
@@ -23,32 +23,33 @@ import java.util.Map;
 
 import org.apache.accumulo.server.conf.codec.VersionedProperties;
 import org.checkerframework.checker.nullness.qual.NonNull;
+import org.checkerframework.checker.nullness.qual.Nullable;
 
 public interface PropStore {
 
   /**
    * Test that a node for properties exists without throwing a KeeperException.
    *
-   * @param propCacheKey
+   * @param propStoreKey
    *          the prop cache key
    * @return true if the property node exists, false otherwise.
    * @throws IllegalStateException
    *           if the check fails due to interrupt.
    */
-  boolean exists(PropCacheKey<?> propCacheKey);
+  boolean exists(PropStoreKey<?> propStoreKey);
 
   /**
    * Create an initial entry for the PropCacheId. If properties already exist, they are not
    * modified.
    *
-   * @param propCacheKey
+   * @param propStoreKey
    *          the prop cache key
    * @param props
    *          a map of property k,v pairs
    * @throws IllegalStateException
    *           if the updates fails because of an underlying store exception
    */
-  void create(PropCacheKey<?> propCacheKey, Map<String,String> props);
+  void create(PropStoreKey<?> propStoreKey, Map<String,String> props);
 
   /**
    *
@@ -60,42 +61,42 @@ public interface PropStore {
    *           not exist for the propCacheId
    */
   @NonNull
-  VersionedProperties get(PropCacheKey<?> propCacheId);
+  VersionedProperties get(PropStoreKey<?> propCacheId);
 
   /**
    * Adds or updates current properties. If the property currently exists it is overwritten,
    * otherwise it is added.
    *
-   * @param propCacheKey
+   * @param propStoreKey
    *          the prop cache key
    * @param props
    *          a map of property k,v pairs
    * @throws IllegalStateException
    *           if the values cannot be written or if an underlying store exception occurs.
    */
-  void putAll(PropCacheKey<?> propCacheKey, Map<String,String> props);
+  void putAll(PropStoreKey<?> propStoreKey, Map<String,String> props);
 
   /**
    * Delete the store node from the underlying store.
    *
-   * @param propCacheKey
+   * @param propStoreKey
    *          the prop cache key
    * @throws IllegalStateException
    *           if the updates fails because of an underlying store exception
    */
-  void delete(PropCacheKey<?> propCacheKey);
+  void delete(PropStoreKey<?> propStoreKey);
 
   /**
    * Deletes individual properties specified by the set of keys.
    *
-   * @param propCacheKey
+   * @param propStoreKey
    *          the prop cache key
    * @param keys
    *          a set of keys.
    * @throws IllegalStateException
    *           if the values cannot be deleted or if an underlying store exception occurs.
    */
-  void removeProperties(PropCacheKey<?> propCacheKey, Collection<String> keys);
+  void removeProperties(PropStoreKey<?> propStoreKey, Collection<String> keys);
 
   /**
    * External processes can register for notifications if the properties change. Normally processes
@@ -107,11 +108,16 @@ public interface PropStore {
    * but listeners should not perform lengthy operations on the notification to prevent delaying
    * other listeners from receive timely notification of the changes detected.
    *
-   * @param propCacheKey
+   * @param propStoreKey
    *          the prop cache key
    * @param listener
    *          a listener
    */
-  void registerAsListener(PropCacheKey<?> propCacheKey, PropChangeListener listener);
+  void registerAsListener(PropStoreKey<?> propStoreKey, PropChangeListener listener);
+
+  PropCache getCache();
+
+  @Nullable
+  VersionedProperties getWithoutCaching(PropStoreKey<?> propStoreKey);
 
 }
diff --git a/server/base/src/main/java/org/apache/accumulo/server/conf/store/PropCacheKey.java b/server/base/src/main/java/org/apache/accumulo/server/conf/store/PropStoreKey.java
similarity index 90%
rename from server/base/src/main/java/org/apache/accumulo/server/conf/store/PropCacheKey.java
rename to server/base/src/main/java/org/apache/accumulo/server/conf/store/PropStoreKey.java
index 171a3f8274..c3fe88c9b8 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/conf/store/PropCacheKey.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/conf/store/PropStoreKey.java
@@ -41,12 +41,12 @@ import org.slf4j.LoggerFactory;
  * Provides utility methods from constructing different id based on type and methods to parse a
  * ZooKeeper path and return a prop cache id.
  */
-public abstract class PropCacheKey<ID_TYPE extends AbstractId<ID_TYPE>>
-    implements Comparable<PropCacheKey<ID_TYPE>> {
+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(PropCacheKey.class);
+  private static final Logger log = LoggerFactory.getLogger(PropStoreKey.class);
 
   // indices for path.split();
   public static final int TYPE_TOKEN_POSITION = 3;
@@ -62,7 +62,7 @@ public abstract class PropCacheKey<ID_TYPE extends AbstractId<ID_TYPE>>
 
   private final String path;
 
-  protected PropCacheKey(final InstanceId instanceId, final String path, final ID_TYPE id) {
+  protected PropStoreKey(final InstanceId instanceId, final String path, final ID_TYPE id) {
     this.instanceId = instanceId;
     this.path = path;
     this.id = id;
@@ -87,7 +87,7 @@ public abstract class PropCacheKey<ID_TYPE extends AbstractId<ID_TYPE>>
    *          the path
    * @return the prop cache id
    */
-  public static @Nullable PropCacheKey<?> fromPath(final String path) {
+  public static @Nullable PropStoreKey<?> fromPath(final String path) {
     String[] tokens = path.split("/");
 
     InstanceId instanceId;
@@ -111,8 +111,8 @@ public abstract class PropCacheKey<ID_TYPE extends AbstractId<ID_TYPE>>
   }
 
   @Override
-  public int compareTo(@NonNull PropCacheKey<ID_TYPE> other) {
-    return Comparator.comparing(PropCacheKey<ID_TYPE>::getPath).compare(this, other);
+  public int compareTo(@NonNull PropStoreKey<ID_TYPE> other) {
+    return Comparator.comparing(PropStoreKey<ID_TYPE>::getPath).compare(this, other);
   }
 
   @Override
@@ -121,7 +121,7 @@ public abstract class PropCacheKey<ID_TYPE extends AbstractId<ID_TYPE>>
       return true;
     if (o == null || getClass() != o.getClass())
       return false;
-    PropCacheKey<?> that = (PropCacheKey<?>) o;
+    PropStoreKey<?> that = (PropStoreKey<?>) o;
     if (getId().getClass() != that.getId().getClass()) {
       return false;
     }
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 f960f4a242..369b0a4a8b 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
@@ -25,7 +25,7 @@ import org.apache.accumulo.fate.zookeeper.ZooUtil;
 import org.apache.accumulo.server.ServerContext;
 import org.checkerframework.checker.nullness.qual.NonNull;
 
-public class SystemPropKey extends PropCacheKey<InstanceId> {
+public class SystemPropKey extends PropStoreKey<InstanceId> {
 
   private SystemPropKey(final InstanceId instanceId, final String path) {
     super(instanceId, path, instanceId);
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 d8ee686ca3..061725401f 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
@@ -27,7 +27,7 @@ import org.apache.accumulo.fate.zookeeper.ZooUtil;
 import org.apache.accumulo.server.ServerContext;
 import org.checkerframework.checker.nullness.qual.NonNull;
 
-public class TablePropKey extends PropCacheKey<TableId> {
+public class TablePropKey extends PropStoreKey<TableId> {
 
   public static TablePropKey of(final ServerContext context, final TableId tableId) {
     return of(context.getInstanceID(), tableId);
diff --git a/server/base/src/main/java/org/apache/accumulo/server/conf/store/impl/PropCacheCaffeineImpl.java b/server/base/src/main/java/org/apache/accumulo/server/conf/store/impl/PropCacheCaffeineImpl.java
index 6433a51a4c..e5c84b65cb 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/conf/store/impl/PropCacheCaffeineImpl.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/conf/store/impl/PropCacheCaffeineImpl.java
@@ -25,7 +25,7 @@ import java.util.concurrent.TimeUnit;
 import org.apache.accumulo.core.util.threads.ThreadPools;
 import org.apache.accumulo.server.conf.codec.VersionedProperties;
 import org.apache.accumulo.server.conf.store.PropCache;
-import org.apache.accumulo.server.conf.store.PropCacheKey;
+import org.apache.accumulo.server.conf.store.PropStoreKey;
 import org.checkerframework.checker.nullness.qual.Nullable;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -47,9 +47,9 @@ public class PropCacheCaffeineImpl implements PropCache {
 
   private final PropStoreMetrics metrics;
 
-  private final LoadingCache<PropCacheKey<?>,VersionedProperties> cache;
+  private final LoadingCache<PropStoreKey<?>,VersionedProperties> cache;
 
-  private PropCacheCaffeineImpl(final CacheLoader<PropCacheKey<?>,VersionedProperties> cacheLoader,
+  private PropCacheCaffeineImpl(final CacheLoader<PropStoreKey<?>,VersionedProperties> cacheLoader,
       final PropStoreMetrics metrics, final Ticker ticker, boolean runTasksInline) {
     this.metrics = metrics;
     var builder = Caffeine.newBuilder().refreshAfterWrite(REFRESH_MIN, BASE_TIME_UNITS)
@@ -69,28 +69,28 @@ public class PropCacheCaffeineImpl implements PropCache {
     return metrics;
   }
 
-  void evictionNotifier(PropCacheKey<?> propCacheKey, VersionedProperties value,
+  void evictionNotifier(PropStoreKey<?> propStoreKey, VersionedProperties value,
       RemovalCause cause) {
-    log.trace("Evicted: ID: {} was evicted from cache. Reason: {}", propCacheKey, cause);
+    log.trace("Evicted: ID: {} was evicted from cache. Reason: {}", propStoreKey, cause);
     metrics.incrEviction();
   }
 
   @Override
-  public @Nullable VersionedProperties get(PropCacheKey<?> propCacheKey) {
-    log.trace("Called get() for {}", propCacheKey);
+  public @Nullable VersionedProperties get(PropStoreKey<?> propStoreKey) {
+    log.trace("Called get() for {}", propStoreKey);
     try {
-      return cache.get(propCacheKey);
+      return cache.get(propStoreKey);
     } catch (Exception ex) {
-      log.info("Cache failed to retrieve properties for: " + propCacheKey, ex);
+      log.info("Cache failed to retrieve properties for: " + propStoreKey, ex);
       metrics.incrZkError();
       return null;
     }
   }
 
   @Override
-  public void remove(PropCacheKey<?> propCacheKey) {
-    log.trace("clear {} from cache", propCacheKey);
-    cache.invalidate(propCacheKey);
+  public void remove(PropStoreKey<?> propStoreKey) {
+    log.trace("clear {} from cache", propStoreKey);
+    cache.invalidate(propStoreKey);
   }
 
   @Override
@@ -105,13 +105,12 @@ public class PropCacheCaffeineImpl implements PropCache {
    * processing so there is no reason to store them in the cache at this time. If they are used, a
    * normal cache get will load the property into the cache.
    *
-   * @param propCacheKey
+   * @param propStoreKey
    *          the property id
    * @return the version properties if cached, otherwise return null.
    */
-  @Override
-  public @Nullable VersionedProperties getWithoutCaching(PropCacheKey<?> propCacheKey) {
-    return cache.getIfPresent(propCacheKey);
+  public @Nullable VersionedProperties getWithoutCaching(PropStoreKey<?> propStoreKey) {
+    return cache.getIfPresent(propStoreKey);
   }
 
   public static class Builder {
diff --git a/server/base/src/main/java/org/apache/accumulo/server/conf/store/impl/PropStoreEventTask.java b/server/base/src/main/java/org/apache/accumulo/server/conf/store/impl/PropStoreEventTask.java
index c87e4e8807..182d50cccc 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/conf/store/impl/PropStoreEventTask.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/conf/store/impl/PropStoreEventTask.java
@@ -20,8 +20,8 @@ package org.apache.accumulo.server.conf.store.impl;
 
 import java.util.Set;
 
-import org.apache.accumulo.server.conf.store.PropCacheKey;
 import org.apache.accumulo.server.conf.store.PropChangeListener;
+import org.apache.accumulo.server.conf.store.PropStoreKey;
 
 /**
  * Provides a simple runnable base task for notifying listeners for PropStore event change
@@ -29,7 +29,7 @@ import org.apache.accumulo.server.conf.store.PropChangeListener;
  */
 public abstract class PropStoreEventTask implements Runnable {
 
-  private final PropCacheKey<?> propCacheKey;
+  private final PropStoreKey<?> propStoreKey;
   private final Set<PropChangeListener> listeners;
 
   /**
@@ -39,60 +39,60 @@ public abstract class PropStoreEventTask implements Runnable {
    *          the set of listeners.
    */
   private PropStoreEventTask(final Set<PropChangeListener> listeners) {
-    this.propCacheKey = null;
+    this.propStoreKey = null;
     this.listeners = listeners;
   }
 
   /**
    * Used when listeners for the specified prop cahe key should receive a notification.
    *
-   * @param propCacheKey
+   * @param propStoreKey
    *          the prop cache key
    * @param listeners
    *          the set of listeners
    */
-  private PropStoreEventTask(final PropCacheKey<?> propCacheKey,
+  private PropStoreEventTask(final PropStoreKey<?> propStoreKey,
       final Set<PropChangeListener> listeners) {
-    this.propCacheKey = propCacheKey;
+    this.propStoreKey = propStoreKey;
     this.listeners = listeners;
   }
 
   public static class PropStoreZkChangeEventTask extends PropStoreEventTask {
 
-    PropStoreZkChangeEventTask(final PropCacheKey<?> propCacheKey,
+    PropStoreZkChangeEventTask(final PropStoreKey<?> propStoreKey,
         final Set<PropChangeListener> listeners) {
-      super(propCacheKey, listeners);
+      super(propStoreKey, listeners);
     }
 
     @Override
     public void run() {
-      super.listeners.forEach(listener -> listener.zkChangeEvent(super.propCacheKey));
+      super.listeners.forEach(listener -> listener.zkChangeEvent(super.propStoreKey));
     }
   }
 
   public static class PropStoreCacheChangeEventTask extends PropStoreEventTask {
 
-    PropStoreCacheChangeEventTask(final PropCacheKey<?> propCacheKey,
+    PropStoreCacheChangeEventTask(final PropStoreKey<?> propStoreKey,
         final Set<PropChangeListener> listeners) {
-      super(propCacheKey, listeners);
+      super(propStoreKey, listeners);
     }
 
     @Override
     public void run() {
-      super.listeners.forEach(listener -> listener.cacheChangeEvent(super.propCacheKey));
+      super.listeners.forEach(listener -> listener.cacheChangeEvent(super.propStoreKey));
     }
   }
 
   public static class PropStoreDeleteEventTask extends PropStoreEventTask {
 
-    PropStoreDeleteEventTask(final PropCacheKey<?> propCacheKey,
+    PropStoreDeleteEventTask(final PropStoreKey<?> propStoreKey,
         final Set<PropChangeListener> listeners) {
-      super(propCacheKey, listeners);
+      super(propStoreKey, listeners);
     }
 
     @Override
     public void run() {
-      super.listeners.forEach(listener -> listener.deleteEvent(super.propCacheKey));
+      super.listeners.forEach(listener -> listener.deleteEvent(super.propStoreKey));
     }
   }
 
diff --git a/server/base/src/main/java/org/apache/accumulo/server/conf/store/impl/PropStoreWatcher.java b/server/base/src/main/java/org/apache/accumulo/server/conf/store/impl/PropStoreWatcher.java
index 5e8a6e7ab0..dd84d17317 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/conf/store/impl/PropStoreWatcher.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/conf/store/impl/PropStoreWatcher.java
@@ -30,8 +30,8 @@ import java.util.concurrent.ExecutorService;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
 
 import org.apache.accumulo.core.util.threads.ThreadPools;
-import org.apache.accumulo.server.conf.store.PropCacheKey;
 import org.apache.accumulo.server.conf.store.PropChangeListener;
+import org.apache.accumulo.server.conf.store.PropStoreKey;
 import org.apache.zookeeper.WatchedEvent;
 import org.apache.zookeeper.Watcher;
 import org.checkerframework.checker.nullness.qual.NonNull;
@@ -65,7 +65,7 @@ public class PropStoreWatcher implements Watcher {
   private final ReentrantReadWriteLock.WriteLock listenerWriteLock = listenerLock.writeLock();
 
   // access should be guarded by acquiring the listener read or write lock
-  private final Map<PropCacheKey<?>,Set<PropChangeListener>> listeners = new HashMap<>();
+  private final Map<PropStoreKey<?>,Set<PropChangeListener>> listeners = new HashMap<>();
 
   private final ReadyMonitor zkReadyMonitor;
 
@@ -73,11 +73,11 @@ public class PropStoreWatcher implements Watcher {
     this.zkReadyMonitor = zkReadyMonitor;
   }
 
-  public void registerListener(final PropCacheKey<?> propCacheKey,
+  public void registerListener(final PropStoreKey<?> propStoreKey,
       final PropChangeListener listener) {
     listenerWriteLock.lock();
     try {
-      Set<PropChangeListener> set = listeners.computeIfAbsent(propCacheKey, s -> new HashSet<>());
+      Set<PropChangeListener> set = listeners.computeIfAbsent(propStoreKey, s -> new HashSet<>());
       set.add(listener);
     } finally {
       listenerWriteLock.unlock();
@@ -97,28 +97,28 @@ public class PropStoreWatcher implements Watcher {
   public void process(final WatchedEvent event) {
 
     String path;
-    PropCacheKey<?> propCacheKey;
+    PropStoreKey<?> propStoreKey;
     switch (event.getType()) {
       case NodeDataChanged:
         path = event.getPath();
         log.trace("handle change event for path: {}", path);
-        propCacheKey = PropCacheKey.fromPath(path);
-        if (propCacheKey != null) {
-          signalZkChangeEvent(propCacheKey);
+        propStoreKey = PropStoreKey.fromPath(path);
+        if (propStoreKey != null) {
+          signalZkChangeEvent(propStoreKey);
         }
         break;
       case NodeDeleted:
         path = event.getPath();
         log.trace("handle delete event for path: {}", path);
-        propCacheKey = PropCacheKey.fromPath(path);
-        if (propCacheKey != null) {
+        propStoreKey = PropStoreKey.fromPath(path);
+        if (propStoreKey != null) {
           // notify listeners
-          Set<PropChangeListener> snapshot = getListenerSnapshot(propCacheKey);
+          Set<PropChangeListener> snapshot = getListenerSnapshot(propStoreKey);
           if (snapshot != null) {
             executorService
-                .execute(new PropStoreEventTask.PropStoreDeleteEventTask(propCacheKey, snapshot));
+                .execute(new PropStoreEventTask.PropStoreDeleteEventTask(propStoreKey, snapshot));
           }
-          listenerCleanup(propCacheKey);
+          listenerCleanup(propStoreKey);
         }
         break;
       case None:
@@ -160,36 +160,36 @@ public class PropStoreWatcher implements Watcher {
   }
 
   /**
-   * Execute a task to notify registered listeners that the propCacheKey node received an event
+   * Execute a task to notify registered listeners that the propStoreKey node received an event
    * notification from ZooKeeper and should be updated. The process can be initiated either by a
    * ZooKeeper notification or a change detected in the cache based on a ZooKeeper event.
    *
-   * @param propCacheKey
+   * @param propStoreKey
    *          the cache id
    */
-  public void signalZkChangeEvent(@NonNull final PropCacheKey<?> propCacheKey) {
-    log.trace("signal ZooKeeper change event: {}", propCacheKey);
-    Set<PropChangeListener> snapshot = getListenerSnapshot(propCacheKey);
+  public void signalZkChangeEvent(@NonNull final PropStoreKey<?> propStoreKey) {
+    log.trace("signal ZooKeeper change event: {}", propStoreKey);
+    Set<PropChangeListener> snapshot = getListenerSnapshot(propStoreKey);
     log.trace("Sending change event to: {}", snapshot);
     if (snapshot != null) {
       executorService
-          .execute(new PropStoreEventTask.PropStoreZkChangeEventTask(propCacheKey, snapshot));
+          .execute(new PropStoreEventTask.PropStoreZkChangeEventTask(propStoreKey, snapshot));
     }
   }
 
   /**
-   * Execute a task to notify registered listeners that the propCacheKey node change was detected
+   * Execute a task to notify registered listeners that the propStoreKey node change was detected
    * should be updated.
    *
-   * @param propCacheKey
+   * @param propStoreKey
    *          the cache id
    */
-  public void signalCacheChangeEvent(final PropCacheKey<?> propCacheKey) {
-    log.trace("cache change event: {}", propCacheKey);
-    Set<PropChangeListener> snapshot = getListenerSnapshot(propCacheKey);
+  public void signalCacheChangeEvent(final PropStoreKey<?> propStoreKey) {
+    log.trace("cache change event: {}", propStoreKey);
+    Set<PropChangeListener> snapshot = getListenerSnapshot(propStoreKey);
     if (snapshot != null) {
       executorService
-          .execute(new PropStoreEventTask.PropStoreCacheChangeEventTask(propCacheKey, snapshot));
+          .execute(new PropStoreEventTask.PropStoreCacheChangeEventTask(propStoreKey, snapshot));
     }
   }
 
@@ -197,13 +197,13 @@ public class PropStoreWatcher implements Watcher {
    * Clean-up the active listeners set when an entry is removed from the cache, remove it from the
    * active listeners.
    *
-   * @param propCacheKey
+   * @param propStoreKey
    *          the cache id
    */
-  public void listenerCleanup(final PropCacheKey<?> propCacheKey) {
+  public void listenerCleanup(final PropStoreKey<?> propStoreKey) {
     listenerWriteLock.lock();
     try {
-      listeners.remove(propCacheKey);
+      listeners.remove(propStoreKey);
     } finally {
       listenerWriteLock.unlock();
     }
@@ -213,16 +213,16 @@ public class PropStoreWatcher implements Watcher {
    * Get an immutable snapshot of the listeners for a prop cache id. The set is intended for
    * notification of changes for a specific prop cache id.
    *
-   * @param propCacheKey
+   * @param propStoreKey
    *          the prop cache id
    * @return an immutable copy of listeners.
    */
-  private Set<PropChangeListener> getListenerSnapshot(final PropCacheKey<?> propCacheKey) {
+  private Set<PropChangeListener> getListenerSnapshot(final PropStoreKey<?> propStoreKey) {
 
     Set<PropChangeListener> snapshot = null;
     listenerReadLock.lock();
     try {
-      Set<PropChangeListener> set = listeners.get(propCacheKey);
+      Set<PropChangeListener> set = listeners.get(propStoreKey);
       if (set != null) {
         snapshot = Set.copyOf(set);
       }
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 803463e135..4761d66620 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
@@ -27,7 +27,7 @@ import java.util.concurrent.TimeUnit;
 import org.apache.accumulo.fate.zookeeper.ZooReaderWriter;
 import org.apache.accumulo.server.conf.codec.VersionedPropCodec;
 import org.apache.accumulo.server.conf.codec.VersionedProperties;
-import org.apache.accumulo.server.conf.store.PropCacheKey;
+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.Nullable;
@@ -36,7 +36,7 @@ import org.slf4j.LoggerFactory;
 
 import com.github.benmanes.caffeine.cache.CacheLoader;
 
-public class ZooPropLoader implements CacheLoader<PropCacheKey<?>,VersionedProperties> {
+public class ZooPropLoader implements CacheLoader<PropStoreKey<?>,VersionedProperties> {
 
   private static final Logger log = LoggerFactory.getLogger(ZooPropLoader.class);
 
@@ -55,14 +55,14 @@ public class ZooPropLoader implements CacheLoader<PropCacheKey<?>,VersionedPrope
   }
 
   @Override
-  public @Nullable VersionedProperties load(PropCacheKey<?> propCacheKey) {
+  public @Nullable VersionedProperties load(PropStoreKey<?> propStoreKey) {
     try {
-      log.trace("load called for {}", propCacheKey);
+      log.trace("load called for {}", propStoreKey);
 
       long startNanos = System.nanoTime();
 
       Stat stat = new Stat();
-      byte[] bytes = zrw.getData(propCacheKey.getPath(), propStoreWatcher, stat);
+      byte[] bytes = zrw.getData(propStoreKey.getPath(), propStoreWatcher, stat);
       VersionedProperties vProps = propCodec.fromBytes(stat.getVersion(), bytes);
 
       metrics.addLoadTime(
@@ -71,41 +71,41 @@ public class ZooPropLoader implements CacheLoader<PropCacheKey<?>,VersionedPrope
       return vProps;
     } catch (KeeperException.NoNodeException ex) {
       metrics.incrZkError();
-      log.debug("property node for {} does not exist - it may be being created", propCacheKey);
-      propStoreWatcher.signalZkChangeEvent(propCacheKey);
+      log.debug("property node for {} does not exist - it may be being created", propStoreKey);
+      propStoreWatcher.signalZkChangeEvent(propStoreKey);
       return null;
     } catch (Exception ex) {
       metrics.incrZkError();
-      log.info("Failed to load properties for: {} from ZooKeeper, returning null", propCacheKey,
+      log.info("Failed to load properties for: {} from ZooKeeper, returning null", propStoreKey,
           ex);
-      propStoreWatcher.signalZkChangeEvent(propCacheKey);
+      propStoreWatcher.signalZkChangeEvent(propStoreKey);
       return null;
     }
   }
 
   @Override
-  public CompletableFuture<? extends VersionedProperties> asyncLoad(PropCacheKey<?> propCacheKey,
+  public CompletableFuture<? extends VersionedProperties> asyncLoad(PropStoreKey<?> propStoreKey,
       Executor executor) throws Exception {
-    log.trace("asyncLoad called for key: {}", propCacheKey);
-    return CacheLoader.super.asyncLoad(propCacheKey, executor);
+    log.trace("asyncLoad called for key: {}", propStoreKey);
+    return CacheLoader.super.asyncLoad(propStoreKey, executor);
   }
 
   @Override
-  public CompletableFuture<VersionedProperties> asyncReload(PropCacheKey<?> propCacheKey,
+  public CompletableFuture<VersionedProperties> asyncReload(PropStoreKey<?> propStoreKey,
       VersionedProperties oldValue, Executor executor) throws Exception {
-    log.trace("asyncReload called for key: {}", propCacheKey);
+    log.trace("asyncReload called for key: {}", propStoreKey);
     metrics.incrRefresh();
 
-    return CompletableFuture.supplyAsync(() -> loadIfDifferentVersion(propCacheKey, oldValue),
+    return CompletableFuture.supplyAsync(() -> loadIfDifferentVersion(propStoreKey, oldValue),
         executor);
   }
 
   @Override
-  public @Nullable VersionedProperties reload(PropCacheKey<?> propCacheKey,
+  public @Nullable VersionedProperties reload(PropStoreKey<?> propStoreKey,
       VersionedProperties oldValue) throws Exception {
-    log.trace("reload called for: {}", propCacheKey);
+    log.trace("reload called for: {}", propStoreKey);
     metrics.incrRefresh();
-    return loadIfDifferentVersion(propCacheKey, oldValue);
+    return loadIfDifferentVersion(propStoreKey, oldValue);
   }
 
   /**
@@ -122,7 +122,7 @@ public class ZooPropLoader implements CacheLoader<PropCacheKey<?>,VersionedPrope
    * @return versioned properties that match the values stored in ZooKeeper, or null if the
    *         properties cannot be retrieved.
    */
-  private @Nullable VersionedProperties loadIfDifferentVersion(PropCacheKey<?> propCacheId,
+  private @Nullable VersionedProperties loadIfDifferentVersion(PropStoreKey<?> propCacheId,
       VersionedProperties currentValue) {
     requireNonNull(propCacheId, "propCacheId cannot be null");
     try {
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 c523574de3..073a7dfc1e 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
@@ -34,9 +34,9 @@ import org.apache.accumulo.fate.zookeeper.ZooUtil;
 import org.apache.accumulo.server.conf.codec.VersionedPropCodec;
 import org.apache.accumulo.server.conf.codec.VersionedProperties;
 import org.apache.accumulo.server.conf.store.PropCache;
-import org.apache.accumulo.server.conf.store.PropCacheKey;
 import org.apache.accumulo.server.conf.store.PropChangeListener;
 import org.apache.accumulo.server.conf.store.PropStore;
+import org.apache.accumulo.server.conf.store.PropStoreKey;
 import org.apache.accumulo.server.conf.store.SystemPropKey;
 import org.apache.accumulo.server.conf.util.ConfigTransformer;
 import org.apache.zookeeper.KeeperException;
@@ -55,7 +55,7 @@ public class ZooPropStore implements PropStore, PropChangeListener {
 
   private final ZooReaderWriter zrw;
   private final PropStoreWatcher propStoreWatcher;
-  private final PropCache cache;
+  private final PropCacheCaffeineImpl cache;
   private final PropStoreMetrics cacheMetrics = new PropStoreMetrics();
   private final ReadyMonitor zkReadyMon;
 
@@ -129,9 +129,9 @@ public class ZooPropStore implements PropStore, PropChangeListener {
   }
 
   @Override
-  public boolean exists(final PropCacheKey<?> propCacheKey) {
+  public boolean exists(final PropStoreKey<?> propStoreKey) {
     try {
-      if (zrw.exists(propCacheKey.getPath())) {
+      if (zrw.exists(propStoreKey.getPath())) {
         return true;
       }
     } catch (KeeperException ex) {
@@ -148,14 +148,14 @@ public class ZooPropStore implements PropStore, PropChangeListener {
   }
 
   @Override
-  public void create(PropCacheKey<?> propCacheKey, Map<String,String> props) {
+  public void create(PropStoreKey<?> propStoreKey, Map<String,String> props) {
 
     try {
       VersionedProperties vProps = new VersionedProperties(props);
-      String path = propCacheKey.getPath();
+      String path = propStoreKey.getPath();
       zrw.putPrivatePersistentData(path, codec.toBytes(vProps), ZooUtil.NodeExistsPolicy.FAIL);
     } catch (IOException | KeeperException | InterruptedException ex) {
-      throw new IllegalStateException("Failed to serialize properties for " + propCacheKey, ex);
+      throw new IllegalStateException("Failed to serialize properties for " + propStoreKey, ex);
     }
   }
 
@@ -164,28 +164,28 @@ public class ZooPropStore implements PropStore, PropChangeListener {
    * legacy properties exist, they will be converted to the new storage form and naming convention.
    * The legacy properties are deleted once the new node format is written.
    *
-   * @param propCacheKey
+   * @param propStoreKey
    *          the prop cache key
    * @return The versioned properties.
    * @throws IllegalStateException
    *           if the updates fails because of an underlying store exception
    */
   @Override
-  public @NonNull VersionedProperties get(final PropCacheKey<?> propCacheKey) {
+  public @NonNull VersionedProperties get(final PropStoreKey<?> propStoreKey) {
     checkZkConnection(); // if ZK not connected, block, do not just return a cached value.
-    propStoreWatcher.registerListener(propCacheKey, this);
+    propStoreWatcher.registerListener(propStoreKey, this);
 
-    var props = cache.get(propCacheKey);
+    var props = cache.get(propStoreKey);
     if (props != null) {
       return props;
     }
 
-    if (propCacheKey instanceof SystemPropKey) {
-      return new ConfigTransformer(zrw, codec, propStoreWatcher).transform(propCacheKey);
+    if (propStoreKey instanceof SystemPropKey) {
+      return new ConfigTransformer(zrw, codec, propStoreWatcher).transform(propStoreKey);
     }
 
     throw new IllegalStateException(
-        "Invalid request for " + propCacheKey + ", the property node does not exist");
+        "Invalid request for " + propStoreKey + ", the property node does not exist");
   }
 
   /**
@@ -193,7 +193,7 @@ public class ZooPropStore implements PropStore, PropChangeListener {
    * directly from ZooKeeper. This allows utilities access when there is a ZooKeeper, may there may
    * not be a full instance running. All exception handling is left to the caller.
    *
-   * @param propCacheKey
+   * @param propStoreKey
    *          the prop cache key
    * @param watcher
    *          a prop store watcher that will receive / handle ZooKeeper events.
@@ -207,12 +207,12 @@ public class ZooPropStore implements PropStore, PropChangeListener {
    * @throws InterruptedException
    *           if the ZooKeeper read was interrupted.
    */
-  public static @Nullable VersionedProperties readFromZk(final PropCacheKey<?> propCacheKey,
+  public static @Nullable VersionedProperties readFromZk(final PropStoreKey<?> propStoreKey,
       final PropStoreWatcher watcher, final ZooReader zooReader)
       throws IOException, KeeperException, InterruptedException {
     try {
       Stat stat = new Stat();
-      byte[] bytes = zooReader.getData(propCacheKey.getPath(), watcher, stat);
+      byte[] bytes = zooReader.getData(propStoreKey.getPath(), watcher, stat);
       return codec.fromBytes(stat.getVersion(), bytes);
     } catch (KeeperException.NoNodeException ex) {
       // ignore no node - allow other exceptions to propagate
@@ -231,7 +231,7 @@ public class ZooPropStore implements PropStore, PropChangeListener {
    * value(s) will be the set to the values provided by the last thread to complete. The order is
    * indeterminate.
    *
-   * @param propCacheKey
+   * @param propStoreKey
    *          the prop cache id
    * @param props
    *          a map of property k,v pairs
@@ -239,77 +239,77 @@ public class ZooPropStore implements PropStore, PropChangeListener {
    *           if the values cannot be written or if an underlying store exception occurs.
    */
   @Override
-  public void putAll(@NonNull PropCacheKey<?> propCacheKey, @NonNull Map<String,String> props) {
+  public void putAll(@NonNull PropStoreKey<?> propStoreKey, @NonNull Map<String,String> props) {
     if (props.isEmpty()) {
       return; // no props - noop
     }
-    mutateVersionedProps(propCacheKey, VersionedProperties::addOrUpdate, props);
+    mutateVersionedProps(propStoreKey, VersionedProperties::addOrUpdate, props);
   }
 
   @Override
-  public void removeProperties(@NonNull PropCacheKey<?> propCacheKey,
+  public void removeProperties(@NonNull PropStoreKey<?> propStoreKey,
       @NonNull Collection<String> keys) {
     if (keys.isEmpty()) {
       return; // no keys - noop.
     }
-    mutateVersionedProps(propCacheKey, VersionedProperties::remove, keys);
+    mutateVersionedProps(propStoreKey, VersionedProperties::remove, keys);
   }
 
   @Override
-  public void delete(@NonNull PropCacheKey<?> propCacheKey) {
-    Objects.requireNonNull(propCacheKey, "prop store delete() - Must provide propCacheId");
+  public void delete(@NonNull PropStoreKey<?> propStoreKey) {
+    Objects.requireNonNull(propStoreKey, "prop store delete() - Must provide propCacheId");
     try {
-      log.trace("called delete() for: {}", propCacheKey);
-      final String path = propCacheKey.getPath();
+      log.trace("called delete() for: {}", propStoreKey);
+      final String path = propStoreKey.getPath();
       zrw.delete(path);
-      cache.remove(propCacheKey);
+      cache.remove(propStoreKey);
     } catch (KeeperException | InterruptedException ex) {
-      throw new IllegalStateException("Failed to delete properties for propCacheId " + propCacheKey,
+      throw new IllegalStateException("Failed to delete properties for propCacheId " + propStoreKey,
           ex);
     }
   }
 
-  private <T> void mutateVersionedProps(PropCacheKey<?> propCacheKey,
+  private <T> void mutateVersionedProps(PropStoreKey<?> propStoreKey,
       BiFunction<VersionedProperties,T,VersionedProperties> action, T changes) {
 
-    log.trace("mutateVersionedProps called for: {}", propCacheKey);
+    log.trace("mutateVersionedProps called for: {}", propStoreKey);
 
     try {
 
-      VersionedProperties vProps = cache.getWithoutCaching(propCacheKey);
+      VersionedProperties vProps = cache.getWithoutCaching(propStoreKey);
       if (vProps == null) {
-        vProps = readPropsFromZk(propCacheKey);
+        vProps = readPropsFromZk(propStoreKey);
       }
 
       for (int attempts = 3; attempts > 0; --attempts) {
 
         VersionedProperties updates = action.apply(vProps, changes);
 
-        if (zrw.overwritePersistentData(propCacheKey.getPath(), codec.toBytes(updates),
+        if (zrw.overwritePersistentData(propStoreKey.getPath(), codec.toBytes(updates),
             (int) updates.getDataVersion())) {
           return;
         }
         Thread.sleep(20); // small pause to get thread to yield.
         // re-read from zookeeper to ensure the latest version.
-        vProps = readPropsFromZk(propCacheKey);
+        vProps = readPropsFromZk(propStoreKey);
       }
       throw new IllegalStateException(
-          "failed to remove properties to zooKeeper for " + propCacheKey, null);
+          "failed to remove properties to zooKeeper for " + propStoreKey, null);
     } catch (IllegalArgumentException | IOException ex) {
       throw new IllegalStateException(
-          "Codec failed to decode / encode properties for " + propCacheKey, ex);
+          "Codec failed to decode / encode properties for " + propStoreKey, ex);
     } catch (InterruptedException | KeeperException ex) {
       if (ex instanceof InterruptedException) {
         Thread.currentThread().interrupt();
       }
       throw new IllegalStateException(
-          "failed to remove properties to zooKeeper for " + propCacheKey, ex);
+          "failed to remove properties to zooKeeper for " + propStoreKey, ex);
     }
   }
 
   @Override
-  public void registerAsListener(PropCacheKey<?> propCacheKey, PropChangeListener listener) {
-    propStoreWatcher.registerListener(propCacheKey, listener);
+  public void registerAsListener(PropStoreKey<?> propStoreKey, PropChangeListener listener) {
+    propStoreWatcher.registerListener(propStoreKey, listener);
   }
 
   private void checkZkConnection() {
@@ -325,9 +325,9 @@ public class ZooPropStore implements PropStore, PropChangeListener {
   }
 
   @Override
-  public void zkChangeEvent(PropCacheKey<?> propCacheKey) {
-    log.trace("Received change event from ZooKeeper for: {} removed from cache", propCacheKey);
-    cache.remove(propCacheKey);
+  public void zkChangeEvent(PropStoreKey<?> propStoreKey) {
+    log.trace("Received change event from ZooKeeper for: {} removed from cache", propStoreKey);
+    cache.remove(propStoreKey);
   }
 
   /**
@@ -335,18 +335,18 @@ public class ZooPropStore implements PropStore, PropChangeListener {
    * change is also sent to external listeners of the need to take action, but for the prop store,
    * no additional action is required.
    *
-   * @param propCacheKey
+   * @param propStoreKey
    *          the prop cache id.
    */
   @Override
-  public void cacheChangeEvent(PropCacheKey<?> propCacheKey) {
-    log.trace("zkChangeEvent: {}", propCacheKey);
+  public void cacheChangeEvent(PropStoreKey<?> propStoreKey) {
+    log.trace("zkChangeEvent: {}", propStoreKey);
   }
 
   @Override
-  public void deleteEvent(PropCacheKey<?> propCacheKey) {
-    log.trace("deleteEvent: {}", propCacheKey);
-    cache.remove(propCacheKey);
+  public void deleteEvent(PropStoreKey<?> propStoreKey) {
+    log.trace("deleteEvent: {}", propStoreKey);
+    cache.remove(propStoreKey);
   }
 
   @Override
@@ -358,8 +358,8 @@ public class ZooPropStore implements PropStore, PropChangeListener {
   /**
    * Read and decode property node from ZooKeeper.
    *
-   * @param propCacheKey
-   *          the propCacheKey
+   * @param propStoreKey
+   *          the propStoreKey
    * @return the decoded properties.
    * @throws KeeperException
    *           if a ZooKeeper exception occurs to allow caller to decide on action.
@@ -370,15 +370,25 @@ public class ZooPropStore implements PropStore, PropChangeListener {
    *           if an interrupt occurs. The interrupt status is reasserted and usually best to not
    *           otherwise try to handle the exception.
    */
-  private VersionedProperties readPropsFromZk(PropCacheKey<?> propCacheKey)
+  private VersionedProperties readPropsFromZk(PropStoreKey<?> propStoreKey)
       throws KeeperException, IOException {
     try {
       Stat stat = new Stat();
-      byte[] bytes = zrw.getData(propCacheKey.getPath(), stat);
+      byte[] bytes = zrw.getData(propStoreKey.getPath(), stat);
       return codec.fromBytes(stat.getVersion(), bytes);
     } catch (InterruptedException ex) {
       Thread.currentThread().interrupt();
       throw new IllegalStateException("Interrupt received during ZooKeeper read", ex);
     }
   }
+
+  @Override
+  public PropCache getCache() {
+    return cache;
+  }
+
+  @Override
+  public @Nullable VersionedProperties getWithoutCaching(PropStoreKey<?> propStoreKey) {
+    return cache.getWithoutCaching(propStoreKey);
+  }
 }
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 73fa29324d..8912250eb5 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
@@ -40,7 +40,7 @@ import org.apache.accumulo.fate.zookeeper.ZooReaderWriter;
 import org.apache.accumulo.fate.zookeeper.ZooUtil;
 import org.apache.accumulo.server.conf.codec.VersionedPropCodec;
 import org.apache.accumulo.server.conf.codec.VersionedProperties;
-import org.apache.accumulo.server.conf.store.PropCacheKey;
+import org.apache.accumulo.server.conf.store.PropStoreKey;
 import org.apache.accumulo.server.conf.store.SystemPropKey;
 import org.apache.accumulo.server.conf.store.impl.PropStoreWatcher;
 import org.apache.accumulo.server.conf.store.impl.ZooPropStore;
@@ -104,27 +104,27 @@ public class ConfigTransformer {
    *
    * @return the encoded properties.
    */
-  public VersionedProperties transform(final PropCacheKey<?> propCacheKey) {
-    TransformToken token = TransformToken.createToken(propCacheKey, zrw);
-    return transform(propCacheKey, token);
+  public VersionedProperties transform(final PropStoreKey<?> propStoreKey) {
+    TransformToken token = TransformToken.createToken(propStoreKey, zrw);
+    return transform(propStoreKey, token);
   }
 
   // Allow external (mocked) TransformToken to be used
   @VisibleForTesting
-  VersionedProperties transform(final PropCacheKey<?> propCacheKey, final TransformToken token) {
+  VersionedProperties transform(final PropStoreKey<?> propStoreKey, final TransformToken token) {
 
-    log.info("checking for legacy property upgrade transform for {}", propCacheKey);
+    log.info("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(propCacheKey, propStoreWatcher, zrw);
+      results = ZooPropStore.readFromZk(propStoreKey, propStoreWatcher, zrw);
       if (results != null) {
         log.debug(
             "Found existing node at {}. skipping legacy prop conversion - version: {}, timestamp: {}",
-            propCacheKey, results.getDataVersion(), results.getTimestamp());
+            propStoreKey, results.getDataVersion(), results.getTimestamp());
         return results;
       }
 
@@ -134,67 +134,67 @@ public class ConfigTransformer {
           retry.waitForNextAttempt();
           // look and return node if created while trying to token.
           log.trace("own the token - look for existing encoded node at: {}",
-              propCacheKey.getPath());
-          results = ZooPropStore.readFromZk(propCacheKey, propStoreWatcher, zrw);
+              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: {}",
-                propCacheKey, results.getDataVersion(), results.getTimestamp());
+                propStoreKey, results.getDataVersion(), results.getTimestamp());
             return results;
           }
           // still does not exist - try again.
           token.getTokenOwnership();
         } catch (InterruptedException ex) {
           Thread.currentThread().interrupt();
-          throw new IllegalStateException("Failed to hold transform token for " + propCacheKey, ex);
+          throw new IllegalStateException("Failed to hold transform token for " + propStoreKey, ex);
         } catch (IllegalStateException ex) {
-          throw new IllegalStateException("Failed to hold transform token for " + propCacheKey, ex);
+          throw new IllegalStateException("Failed to hold transform token for " + propStoreKey, ex);
         }
       }
 
-      Set<LegacyPropNode> upgradeNodes = readLegacyProps(propCacheKey);
+      Set<LegacyPropNode> upgradeNodes = readLegacyProps(propStoreKey);
       if (upgradeNodes == null) {
         log.info("Found existing node after reading legacy props {}, skipping conversion",
-            propCacheKey);
-        results = ZooPropStore.readFromZk(propCacheKey, propStoreWatcher, zrw);
+            propStoreKey);
+        results = ZooPropStore.readFromZk(propStoreKey, propStoreWatcher, zrw);
         if (results != null) {
           return results;
         }
       }
 
-      upgradeNodes = convertDeprecatedProps(propCacheKey, upgradeNodes);
+      upgradeNodes = convertDeprecatedProps(propStoreKey, upgradeNodes);
 
-      results = writeConverted(propCacheKey, upgradeNodes);
+      results = writeConverted(propStoreKey, upgradeNodes);
 
       if (results == null) {
-        throw new IllegalStateException("Could not create properties for " + propCacheKey);
+        throw new IllegalStateException("Could not create properties for " + propStoreKey);
       }
 
       // validate token still valid before deletion.
       if (!token.validateToken()) {
         throw new IllegalStateException(
-            "legacy conversion failed. Lost transform token for " + propCacheKey);
+            "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", propCacheKey,
+      log.debug("property transform for {} took {} ms", propStoreKey,
           new DurationFormat(Duration.between(start, Instant.now()).toMillis(), ""));
 
       return results;
 
     } catch (Exception ex) {
-      log.info("Exception on upgrading legacy properties for: " + propCacheKey, ex);
+      log.info("Exception on upgrading legacy properties for: " + propStoreKey, ex);
     } finally {
       token.releaseToken();
     }
     return null;
   }
 
-  private Set<LegacyPropNode> convertDeprecatedProps(PropCacheKey<?> propCacheKey,
+  private Set<LegacyPropNode> convertDeprecatedProps(PropStoreKey<?> propStoreKey,
       Set<LegacyPropNode> upgradeNodes) {
 
-    if (!(propCacheKey instanceof SystemPropKey)) {
+    if (!(propStoreKey instanceof SystemPropKey)) {
       return upgradeNodes;
     }
 
@@ -212,7 +212,7 @@ public class ConfigTransformer {
     return renamedNodes;
   }
 
-  private @Nullable Set<LegacyPropNode> readLegacyProps(PropCacheKey<?> propCacheKey) {
+  private @Nullable Set<LegacyPropNode> readLegacyProps(PropStoreKey<?> propStoreKey) {
 
     Set<LegacyPropNode> legacyProps = new TreeSet<>();
 
@@ -220,17 +220,17 @@ public class ConfigTransformer {
     var tokenName = TransformToken.TRANSFORM_TOKEN.substring(1);
 
     try {
-      var keyBasePath = propCacheKey.getBasePath();
+      var keyBasePath = propStoreKey.getBasePath();
       List<String> childNames = zrw.getChildren(keyBasePath);
       for (String propName : childNames) {
-        log.trace("processing ZooKeeper child node: {} for: {}", propName, propCacheKey);
+        log.trace("processing ZooKeeper child node: {} for: {}", propName, propStoreKey);
         if (tokenName.equals(propName)) {
           continue;
         }
-        if (PropCacheKey.PROP_NODE_NAME.equals(propName)) {
+        if (PropStoreKey.PROP_NODE_NAME.equals(propName)) {
           log.debug(
               "encoded property node exists for {}. Legacy conversion ignoring conversion of this node",
-              propCacheKey);
+              propStoreKey);
           return null;
         }
         log.trace("Adding: {} to list for legacy conversion", propName);
@@ -278,26 +278,26 @@ public class ConfigTransformer {
     return errorCount;
   }
 
-  private @Nullable VersionedProperties writeConverted(final PropCacheKey<?> propCacheKey,
+  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 = propCacheKey.getPath();
+    String path = propStoreKey.getPath();
     try {
       try {
         zrw.putPrivatePersistentData(path, codec.toBytes(vProps), ZooUtil.NodeExistsPolicy.FAIL);
       } catch (KeeperException.NodeExistsException ex) {
-        vProps = ZooPropStore.readFromZk(propCacheKey, propStoreWatcher, zrw);
+        vProps = ZooPropStore.readFromZk(propStoreKey, propStoreWatcher, zrw);
       }
     } catch (InterruptedException | IOException | KeeperException ex) {
       if (ex instanceof InterruptedException) {
         Thread.currentThread().interrupt();
       }
       throw new IllegalStateException(
-          "failed to create node for " + propCacheKey + " on conversion", ex);
+          "failed to create node for " + propStoreKey + " on conversion", ex);
     }
-    if (!validateWrite(propCacheKey, vProps)) {
+    if (!validateWrite(propStoreKey, vProps)) {
       // failed validation
       return null;
     }
@@ -305,20 +305,20 @@ public class ConfigTransformer {
     return vProps;
   }
 
-  private boolean validateWrite(final PropCacheKey<?> propCacheKey,
+  private boolean validateWrite(final PropStoreKey<?> propStoreKey,
       final VersionedProperties vProps) {
     try {
-      Stat stat = zrw.getStatus(propCacheKey.getPath(), propStoreWatcher);
+      Stat stat = zrw.getStatus(propStoreKey.getPath(), propStoreWatcher);
       if (stat == null) {
         throw new IllegalStateException(
-            "failed to get stat to validate created node for " + propCacheKey);
+            "failed to get stat to validate created node for " + propStoreKey);
       }
       return stat.getVersion() == vProps.getDataVersion();
     } catch (KeeperException ex) {
-      throw new IllegalStateException("failed to validate created node for " + propCacheKey, ex);
+      throw new IllegalStateException("failed to validate created node for " + propStoreKey, ex);
     } catch (InterruptedException ex) {
       Thread.currentThread().interrupt();
-      throw new IllegalStateException("failed to validate created node for " + propCacheKey, 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/PropSnapshot.java b/server/base/src/main/java/org/apache/accumulo/server/conf/util/PropSnapshot.java
index 28cc7371c0..f1aa2f6dd9 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/conf/util/PropSnapshot.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/conf/util/PropSnapshot.java
@@ -24,9 +24,9 @@ import java.util.concurrent.locks.Lock;
 import java.util.concurrent.locks.ReentrantLock;
 
 import org.apache.accumulo.server.conf.codec.VersionedProperties;
-import org.apache.accumulo.server.conf.store.PropCacheKey;
 import org.apache.accumulo.server.conf.store.PropChangeListener;
 import org.apache.accumulo.server.conf.store.PropStore;
+import org.apache.accumulo.server.conf.store.PropStoreKey;
 import org.checkerframework.checker.nullness.qual.NonNull;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -46,17 +46,17 @@ public class PropSnapshot implements PropChangeListener {
   private final Lock updateLock = new ReentrantLock();
   private final AtomicBoolean needsUpdate = new AtomicBoolean(true);
   private final AtomicReference<VersionedProperties> vPropRef = new AtomicReference<>();
-  private final PropCacheKey<?> propCacheKey;
+  private final PropStoreKey<?> propStoreKey;
   private final PropStore propStore;
 
-  public static PropSnapshot create(final PropCacheKey<?> propCacheKey, final PropStore propStore) {
-    var ps = new PropSnapshot(propCacheKey, propStore);
-    propStore.registerAsListener(propCacheKey, ps);
+  public static PropSnapshot create(final PropStoreKey<?> propStoreKey, final PropStore propStore) {
+    var ps = new PropSnapshot(propStoreKey, propStore);
+    propStore.registerAsListener(propStoreKey, ps);
     return ps;
   }
 
-  private PropSnapshot(final PropCacheKey<?> propCacheKey, final PropStore propStore) {
-    this.propCacheKey = propCacheKey;
+  private PropSnapshot(final PropStoreKey<?> propStoreKey, final PropStore propStore) {
+    this.propStoreKey = propStoreKey;
     this.propStore = propStore;
   }
 
@@ -102,7 +102,7 @@ public class PropSnapshot implements PropChangeListener {
       if (!needsUpdate.get()) {
         return;
       }
-      var vProps = propStore.get(propCacheKey);
+      var vProps = propStore.get(propStoreKey);
       vPropRef.set(vProps);
       needsUpdate.set(false);
     } finally {
@@ -111,24 +111,24 @@ public class PropSnapshot implements PropChangeListener {
   }
 
   @Override
-  public void zkChangeEvent(final PropCacheKey<?> eventPropKey) {
-    if (propCacheKey.equals(eventPropKey)) {
+  public void zkChangeEvent(final PropStoreKey<?> eventPropKey) {
+    if (propStoreKey.equals(eventPropKey)) {
       requireUpdate();
     }
   }
 
   @Override
-  public void cacheChangeEvent(final PropCacheKey<?> eventPropKey) {
-    if (propCacheKey.equals(eventPropKey)) {
+  public void cacheChangeEvent(final PropStoreKey<?> eventPropKey) {
+    if (propStoreKey.equals(eventPropKey)) {
       requireUpdate();
     }
   }
 
   @Override
-  public void deleteEvent(final PropCacheKey<?> eventPropKey) {
-    if (propCacheKey.equals(eventPropKey)) {
+  public void deleteEvent(final PropStoreKey<?> eventPropKey) {
+    if (propStoreKey.equals(eventPropKey)) {
       requireUpdate();
-      log.debug("Received property delete event for {}", propCacheKey);
+      log.debug("Received property delete event for {}", propStoreKey);
     }
   }
 
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 943fc8fa7a..c7eb855530 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,7 @@ import java.util.Objects;
 import java.util.UUID;
 
 import org.apache.accumulo.fate.zookeeper.ZooReaderWriter;
-import org.apache.accumulo.server.conf.store.PropCacheKey;
+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,7 +55,7 @@ public class TransformToken {
   private final ZooReaderWriter zrw;
   private boolean haveToken = false;
 
-  private TransformToken(final @NonNull PropCacheKey<?> key, final ZooReaderWriter zrw) {
+  private TransformToken(final @NonNull PropStoreKey<?> key, final ZooReaderWriter zrw) {
     path = key.getBasePath() + TRANSFORM_TOKEN;
     this.zrw = zrw;
 
@@ -70,7 +70,7 @@ public class TransformToken {
    * to be created by the thread that created the lock, or try calling to {@code lock} to succeed
    *
    * @param key
-   *          a PropCacheKey that defines the storage location of the created lock and the
+   *          a PropStoreKey that defines the storage location of the created lock and the
    *          associated property nodes.
    * @param zrw
    *          a ZooReaderWriter
@@ -78,7 +78,7 @@ public class TransformToken {
    * @throws IllegalStateException
    *           is the lock creation fails due to an underlying ZooKeeper exception.
    */
-  public static TransformToken createToken(final @NonNull PropCacheKey<?> key,
+  public static TransformToken createToken(final @NonNull PropStoreKey<?> key,
       final ZooReaderWriter zrw) {
     return new TransformToken(key, zrw);
   }
diff --git a/server/base/src/test/java/org/apache/accumulo/server/conf/NamespaceConfigurationTest.java b/server/base/src/test/java/org/apache/accumulo/server/conf/NamespaceConfigurationTest.java
index c57d4248de..621e10f42f 100644
--- a/server/base/src/test/java/org/apache/accumulo/server/conf/NamespaceConfigurationTest.java
+++ b/server/base/src/test/java/org/apache/accumulo/server/conf/NamespaceConfigurationTest.java
@@ -75,10 +75,10 @@ public class NamespaceConfigurationTest {
     parent = createMock(AccumuloConfiguration.class);
     reset(propStore);
 
-    var nsCacheKey = NamespacePropKey.of(iid, NSID);
-    expect(propStore.get(eq(nsCacheKey))).andReturn(new VersionedProperties(123, Instant.now(),
+    var nsPropStoreKey = NamespacePropKey.of(iid, NSID);
+    expect(propStore.get(eq(nsPropStoreKey))).andReturn(new VersionedProperties(123, Instant.now(),
         Map.of(Property.INSTANCE_SECRET.getKey(), "sekrit"))).anyTimes();
-    propStore.registerAsListener(eq(nsCacheKey), anyObject());
+    propStore.registerAsListener(eq(nsPropStoreKey), anyObject());
     expectLastCall().anyTimes();
 
     replay(propStore, context);
diff --git a/server/base/src/test/java/org/apache/accumulo/server/conf/store/PropCacheKeyTest.java b/server/base/src/test/java/org/apache/accumulo/server/conf/store/PropStoreKeyTest.java
similarity index 92%
rename from server/base/src/test/java/org/apache/accumulo/server/conf/store/PropCacheKeyTest.java
rename to server/base/src/test/java/org/apache/accumulo/server/conf/store/PropStoreKeyTest.java
index 39982dc5fb..15de7736c2 100644
--- a/server/base/src/test/java/org/apache/accumulo/server/conf/store/PropCacheKeyTest.java
+++ b/server/base/src/test/java/org/apache/accumulo/server/conf/store/PropStoreKeyTest.java
@@ -21,7 +21,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;
-import static org.apache.accumulo.server.conf.store.PropCacheKey.PROP_NODE_NAME;
+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;
@@ -40,8 +40,8 @@ import org.junit.jupiter.api.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-public class PropCacheKeyTest {
-  private static final Logger log = LoggerFactory.getLogger(PropCacheKeyTest.class);
+public class PropStoreKeyTest {
+  private static final Logger log = LoggerFactory.getLogger(PropStoreKeyTest.class);
 
   private final InstanceId instanceId = InstanceId.of(UUID.randomUUID());
 
@@ -98,18 +98,18 @@ public class PropCacheKeyTest {
 
     var iid = "3f9976c6-3bf1-41ab-9751-1b0a9be3551d";
 
-    PropCacheKey<?> t1 =
-        PropCacheKey.fromPath("/accumulo/" + iid + "/tables/t1/conf/encoded_props");
+    PropStoreKey<?> t1 =
+        PropStoreKey.fromPath("/accumulo/" + iid + "/tables/t1/conf/encoded_props");
     assertNotNull(t1);
     assertEquals(TableId.of("t1"), t1.getId());
 
-    PropCacheKey<?> n1 =
-        PropCacheKey.fromPath("/accumulo/" + iid + "/namespaces/n1/conf/encoded_props");
+    PropStoreKey<?> n1 =
+        PropStoreKey.fromPath("/accumulo/" + iid + "/namespaces/n1/conf/encoded_props");
     assertNotNull(n1);
     assertEquals(NamespaceId.of("n1"), n1.getId());
     assertNotNull(n1.getId());
 
-    PropCacheKey<?> s1 = PropCacheKey.fromPath("/accumulo/" + iid + "/config/encoded_props");
+    PropStoreKey<?> s1 = PropStoreKey.fromPath("/accumulo/" + iid + "/config/encoded_props");
     assertNotNull(s1);
     // system config returns instance id as id placeholder
     assertEquals(iid, s1.getId().canonical());
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 59d28c3c77..02ad89061c 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
@@ -43,9 +43,9 @@ import org.apache.accumulo.fate.zookeeper.ZooReaderWriter;
 import org.apache.accumulo.server.ServerContext;
 import org.apache.accumulo.server.conf.codec.VersionedPropCodec;
 import org.apache.accumulo.server.conf.codec.VersionedProperties;
-import org.apache.accumulo.server.conf.store.PropCacheKey;
 import org.apache.accumulo.server.conf.store.PropChangeListener;
 import org.apache.accumulo.server.conf.store.PropStore;
+import org.apache.accumulo.server.conf.store.PropStoreKey;
 import org.apache.accumulo.server.conf.store.TablePropKey;
 import org.apache.zookeeper.WatchedEvent;
 import org.apache.zookeeper.Watcher;
@@ -280,17 +280,17 @@ public class PropStoreEventTest {
     private int connectionEventCount = 0;
 
     @Override
-    public void zkChangeEvent(PropCacheKey<?> propCacheKey) {
+    public void zkChangeEvent(PropStoreKey<?> propStoreKey) {
       zkChangeEventCount++;
     }
 
     @Override
-    public void cacheChangeEvent(PropCacheKey<?> propCacheKey) {
+    public void cacheChangeEvent(PropStoreKey<?> propStoreKey) {
       cacheChangeEventCount++;
     }
 
     @Override
-    public void deleteEvent(PropCacheKey<?> propCacheKey) {
+    public void deleteEvent(PropStoreKey<?> propStoreKey) {
       deleteEventCount++;
     }
 
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 ba376dc22d..2487513c13 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
@@ -51,7 +51,7 @@ import org.apache.accumulo.fate.zookeeper.ZooReaderWriter;
 import org.apache.accumulo.server.ServerContext;
 import org.apache.accumulo.server.conf.codec.VersionedPropCodec;
 import org.apache.accumulo.server.conf.codec.VersionedProperties;
-import org.apache.accumulo.server.conf.store.PropCacheKey;
+import org.apache.accumulo.server.conf.store.PropStoreKey;
 import org.apache.accumulo.server.conf.store.SystemPropKey;
 import org.apache.accumulo.server.conf.store.TablePropKey;
 import org.apache.zookeeper.KeeperException;
@@ -70,7 +70,7 @@ public class ZooPropLoaderTest {
   private PropCacheCaffeineImplTest.TestTicker ticker;
   private InstanceId instanceId;
   private ServerContext context;
-  private PropCacheKey<?> propCacheKey;
+  private PropStoreKey<?> propStoreKey;
   private VersionedPropCodec propCodec;
 
   // mocks
@@ -85,7 +85,7 @@ public class ZooPropLoaderTest {
     ticker = new PropCacheCaffeineImplTest.TestTicker();
     instanceId = InstanceId.of(UUID.randomUUID());
 
-    propCacheKey = TablePropKey.of(instanceId, TableId.of("1abc"));
+    propStoreKey = TablePropKey.of(instanceId, TableId.of("1abc"));
     propCodec = VersionedPropCodec.getDefault();
 
     // mocks
@@ -113,7 +113,7 @@ public class ZooPropLoaderTest {
 
     VersionedProperties defaultProps = new VersionedProperties();
 
-    expect(zrw.getData(eq(propCacheKey.getPath()), anyObject(), anyObject()))
+    expect(zrw.getData(eq(propStoreKey.getPath()), anyObject(), anyObject()))
         .andReturn(propCodec.toBytes(defaultProps)).anyTimes();
 
     cacheMetrics.addLoadTime(anyLong());
@@ -121,7 +121,7 @@ public class ZooPropLoaderTest {
 
     replay(context, zrw, propStoreWatcher, cacheMetrics);
 
-    assertNotNull(loader.load(propCacheKey));
+    assertNotNull(loader.load(propStoreKey));
   }
 
   // from cache loader
@@ -137,9 +137,9 @@ public class ZooPropLoaderTest {
 
     VersionedProperties defaultProps = new VersionedProperties();
 
-    expect(zrw.getStatus(propCacheKey.getPath())).andThrow(new KeeperException.NoNodeException())
+    expect(zrw.getStatus(propStoreKey.getPath())).andThrow(new KeeperException.NoNodeException())
         .anyTimes();
-    expect(zrw.getData(eq(propCacheKey.getPath()), anyObject(), anyObject()))
+    expect(zrw.getData(eq(propStoreKey.getPath()), anyObject(), anyObject()))
         .andReturn(propCodec.toBytes(defaultProps)).once();
 
     cacheMetrics.addLoadTime(anyLong());
@@ -151,11 +151,11 @@ public class ZooPropLoaderTest {
         new PropCacheCaffeineImpl.Builder(loader, cacheMetrics).forTests(ticker).build();
 
     // load into cache
-    assertNotNull(cache.get(propCacheKey));
+    assertNotNull(cache.get(propStoreKey));
 
     // read cached entry - load count should not change.
     ticker.advance(1, TimeUnit.MINUTES);
-    assertNotNull(cache.get(propCacheKey));
+    assertNotNull(cache.get(propStoreKey));
   }
 
   @Test
@@ -174,10 +174,10 @@ public class ZooPropLoaderTest {
   @Test
   public void loadFailTest() throws Exception {
 
-    expect(zrw.getData(eq(propCacheKey.getPath()), anyObject(), anyObject()))
+    expect(zrw.getData(eq(propStoreKey.getPath()), anyObject(), anyObject()))
         .andThrow(new KeeperException.NoNodeException("force no node exception")).once();
 
-    propStoreWatcher.signalZkChangeEvent(eq(propCacheKey));
+    propStoreWatcher.signalZkChangeEvent(eq(propStoreKey));
     expectLastCall();
 
     cacheMetrics.incrZkError();
@@ -188,7 +188,7 @@ public class ZooPropLoaderTest {
     PropCacheCaffeineImpl cache =
         new PropCacheCaffeineImpl.Builder(loader, cacheMetrics).forTests(ticker).build();
 
-    assertNull(cache.get(propCacheKey));
+    assertNull(cache.get(propStoreKey));
 
     log.debug("Metrics: {}", cacheMetrics);
   }
@@ -203,7 +203,7 @@ public class ZooPropLoaderTest {
 
     VersionedProperties defaultProps = new VersionedProperties();
 
-    expect(zrw.getData(eq(propCacheKey.getPath()), anyObject(), anyObject()))
+    expect(zrw.getData(eq(propStoreKey.getPath()), anyObject(), anyObject()))
         .andReturn(propCodec.toBytes(defaultProps)).times(2);
 
     cacheMetrics.addLoadTime(anyLong());
@@ -218,11 +218,11 @@ public class ZooPropLoaderTest {
         new PropCacheCaffeineImpl.Builder(loader, cacheMetrics).forTests(ticker).build();
 
     // load cache
-    assertNotNull(cache.get(propCacheKey));
+    assertNotNull(cache.get(propStoreKey));
 
     ticker.advance(70, TimeUnit.MINUTES);
 
-    assertNotNull(cache.get(propCacheKey));
+    assertNotNull(cache.get(propStoreKey));
 
   }
 
@@ -238,14 +238,14 @@ public class ZooPropLoaderTest {
 
     final VersionedProperties defaultProps = new VersionedProperties();
 
-    expect(zrw.getData(eq(propCacheKey.getPath()), anyObject(), anyObject()))
+    expect(zrw.getData(eq(propStoreKey.getPath()), anyObject(), anyObject()))
         .andReturn(propCodec.toBytes(defaultProps)).once();
 
     Stat stat = new Stat();
     stat.setVersion(123); // set different version so reload triggered
-    expect(zrw.getStatus(propCacheKey.getPath())).andReturn(stat).once();
+    expect(zrw.getStatus(propStoreKey.getPath())).andReturn(stat).once();
 
-    expect(zrw.getData(eq(propCacheKey.getPath()), anyObject(), anyObject()))
+    expect(zrw.getData(eq(propStoreKey.getPath()), anyObject(), anyObject()))
         .andThrow(new KeeperException.NoNodeException("forced no node")).anyTimes();
 
     propStoreWatcher.signalZkChangeEvent(anyObject());
@@ -269,19 +269,19 @@ public class ZooPropLoaderTest {
         new PropCacheCaffeineImpl.Builder(loader, cacheMetrics).forTests(ticker).build();
 
     // prime cache
-    assertNotNull(cache.get(propCacheKey));
+    assertNotNull(cache.get(propStoreKey));
 
     ticker.advance(5, TimeUnit.MINUTES);
 
     // read cached value
-    assertNotNull(cache.get(propCacheKey));
+    assertNotNull(cache.get(propStoreKey));
 
     // advance so refresh called.
     ticker.advance(20, TimeUnit.MINUTES);
 
-    assertNotNull(cache.get(propCacheKey));
+    assertNotNull(cache.get(propStoreKey));
 
-    assertNull(cache.get(propCacheKey));
+    assertNull(cache.get(propStoreKey));
   }
 
   @Test
@@ -292,7 +292,7 @@ public class ZooPropLoaderTest {
     PropCacheCaffeineImpl cache =
         new PropCacheCaffeineImpl.Builder(loader, cacheMetrics).forTests(ticker).build();
 
-    assertNull(cache.getWithoutCaching(propCacheKey));
+    assertNull(cache.getWithoutCaching(propStoreKey));
 
   }
 
@@ -366,7 +366,7 @@ public class ZooPropLoaderTest {
         new PropCacheCaffeineImpl.Builder(loader, cacheMetrics).forTests(ticker).build();
 
     // load into cache
-    assertNull(cache.getWithoutCaching(propCacheKey));
+    assertNull(cache.getWithoutCaching(propStoreKey));
   }
 
   @Test
@@ -376,7 +376,7 @@ public class ZooPropLoaderTest {
 
     // first call loads cache
     Capture<Stat> stat = newCapture();
-    expect(zrw.getData(eq(propCacheKey.getPath()), anyObject(), capture(stat))).andAnswer(() -> {
+    expect(zrw.getData(eq(propStoreKey.getPath()), anyObject(), capture(stat))).andAnswer(() -> {
       Stat s = stat.getValue();
       s.setCtime(System.currentTimeMillis());
       s.setMtime(System.currentTimeMillis());
@@ -388,7 +388,7 @@ public class ZooPropLoaderTest {
 
     Stat expectedStat = new Stat();
     expectedStat.setVersion(0);
-    expect(zrw.getStatus(propCacheKey.getPath())).andReturn(expectedStat).times(2);
+    expect(zrw.getStatus(propStoreKey.getPath())).andReturn(expectedStat).times(2);
 
     cacheMetrics.addLoadTime(anyLong());
     expectLastCall().times(1);
@@ -401,23 +401,23 @@ public class ZooPropLoaderTest {
         new PropCacheCaffeineImpl.Builder(loader, cacheMetrics).forTests(ticker).build();
 
     // load cache
-    log.debug("received: {}", cache.get(propCacheKey));
+    log.debug("received: {}", cache.get(propStoreKey));
 
     ticker.advance(REFRESH_MIN + 1, TimeUnit.MINUTES);
 
-    assertNotNull(cache.get(propCacheKey));
+    assertNotNull(cache.get(propStoreKey));
 
     ticker.advance(REFRESH_MIN / 2, TimeUnit.MINUTES);
 
-    assertNotNull(cache.get(propCacheKey));
+    assertNotNull(cache.get(propStoreKey));
 
     ticker.advance(REFRESH_MIN + 1, TimeUnit.MINUTES);
 
-    assertNotNull(cache.get(propCacheKey));
+    assertNotNull(cache.get(propStoreKey));
 
     ticker.advance(1, TimeUnit.MINUTES);
 
-    assertNotNull(cache.get(propCacheKey));
+    assertNotNull(cache.get(propStoreKey));
 
   }
 
@@ -433,7 +433,7 @@ public class ZooPropLoaderTest {
 
     Capture<Stat> stat = newCapture();
 
-    expect(zrw.getData(eq(propCacheKey.getPath()), capture(propStoreWatcherCapture), capture(stat)))
+    expect(zrw.getData(eq(propStoreKey.getPath()), capture(propStoreWatcherCapture), capture(stat)))
         .andAnswer(() -> {
           Stat s = stat.getValue();
           s.setCtime(System.currentTimeMillis());
@@ -447,12 +447,12 @@ public class ZooPropLoaderTest {
     // make it look like version on ZK has advanced.
     Stat stat2 = new Stat();
     stat2.setVersion(initialVersion + 3); // initSysProps 123, on write 124
-    expect(zrw.getStatus(propCacheKey.getPath())).andReturn(stat2).once();
+    expect(zrw.getStatus(propStoreKey.getPath())).andReturn(stat2).once();
 
     Capture<Stat> stat3 = newCapture();
 
     expect(
-        zrw.getData(eq(propCacheKey.getPath()), capture(propStoreWatcherCapture), capture(stat3)))
+        zrw.getData(eq(propStoreKey.getPath()), capture(propStoreWatcherCapture), capture(stat3)))
             .andAnswer(() -> {
               Stat s = stat3.getValue();
               s.setCtime(System.currentTimeMillis());
@@ -463,7 +463,7 @@ public class ZooPropLoaderTest {
                   Map.of(Property.TABLE_SPLIT_THRESHOLD.getKey(), "12G")));
             }).once();
 
-    propStoreWatcher.signalCacheChangeEvent(eq(propCacheKey));
+    propStoreWatcher.signalCacheChangeEvent(eq(propStoreKey));
     expectLastCall();
 
     cacheMetrics.addLoadTime(anyLong());
@@ -481,18 +481,18 @@ public class ZooPropLoaderTest {
         new PropCacheCaffeineImpl.Builder(loader, cacheMetrics).forTests(ticker).build();
 
     // prime cache
-    var origProps = cache.get(propCacheKey);
+    var origProps = cache.get(propStoreKey);
     assertNotNull(origProps);
     assertEquals("7G", origProps.asMap().get(Property.TABLE_SPLIT_THRESHOLD.getKey()));
 
     ticker.advance(REFRESH_MIN + 1, TimeUnit.MINUTES);
     // first call after refresh return original and schedules update
-    var originalProps = cache.get(propCacheKey);
+    var originalProps = cache.get(propStoreKey);
     assertNotNull(originalProps);
     assertNotNull(originalProps.asMap().get(Property.TABLE_SPLIT_THRESHOLD.getKey()));
 
     // refresh should have loaded updated value;
-    var updatedProps = cache.get(propCacheKey);
+    var updatedProps = cache.get(propStoreKey);
     log.debug("Updated props: {}", updatedProps == null ? "null" : updatedProps.print(true));
 
     assertNotNull(updatedProps);
@@ -520,7 +520,7 @@ public class ZooPropLoaderTest {
     Capture<Stat> stat = newCapture();
 
     // first call loads cache
-    expect(zrw.getData(eq(propCacheKey.getPath()), anyObject(), capture(stat))).andAnswer(() -> {
+    expect(zrw.getData(eq(propStoreKey.getPath()), anyObject(), capture(stat))).andAnswer(() -> {
       Stat s = stat.getValue();
       s.setCtime(System.currentTimeMillis());
       s.setMtime(System.currentTimeMillis());
@@ -534,7 +534,7 @@ public class ZooPropLoaderTest {
     stat2.setMtime(System.currentTimeMillis());
     stat2.setVersion(expectedVersion);
 
-    expect(zrw.getStatus(propCacheKey.getPath())).andReturn(stat2).once();
+    expect(zrw.getStatus(propStoreKey.getPath())).andReturn(stat2).once();
 
     cacheMetrics.addLoadTime(anyLong());
     expectLastCall().times(1);
@@ -547,15 +547,15 @@ public class ZooPropLoaderTest {
         new PropCacheCaffeineImpl.Builder(loader, cacheMetrics).forTests(ticker).build();
 
     // prime cache
-    cache.get(propCacheKey);
+    cache.get(propStoreKey);
 
     ticker.advance(30, TimeUnit.MINUTES);
 
-    VersionedProperties vPropsRead = cache.get(propCacheKey);
+    VersionedProperties vPropsRead = cache.get(propStoreKey);
 
     assertNotNull(vPropsRead);
 
-    cache.get(propCacheKey);
+    cache.get(propStoreKey);
 
     verify(mockProps);
   }
@@ -572,7 +572,7 @@ public class ZooPropLoaderTest {
 
     // first call loads cache
     Capture<Stat> stat = newCapture();
-    expect(zrw.getData(eq(propCacheKey.getPath()), anyObject(), capture(stat))).andAnswer(() -> {
+    expect(zrw.getData(eq(propStoreKey.getPath()), anyObject(), capture(stat))).andAnswer(() -> {
       Stat s = stat.getValue();
       s.setCtime(System.currentTimeMillis());
       s.setMtime(System.currentTimeMillis());
@@ -584,13 +584,13 @@ public class ZooPropLoaderTest {
 
     Stat expectedStat = new Stat();
     expectedStat.setVersion(0);
-    expect(zrw.getStatus(propCacheKey.getPath()))
+    expect(zrw.getStatus(propStoreKey.getPath()))
         .andThrow(new KeeperException.NoNodeException("force no node exception")).once();
 
-    propStoreWatcher.signalZkChangeEvent(eq(propCacheKey));
+    propStoreWatcher.signalZkChangeEvent(eq(propStoreKey));
     expectLastCall().anyTimes();
 
-    expect(zrw.getData(eq(propCacheKey.getPath()), anyObject(), anyObject()))
+    expect(zrw.getData(eq(propStoreKey.getPath()), anyObject(), anyObject()))
         .andThrow(new KeeperException.NoNodeException("force no node exception")).once();
 
     cacheMetrics.addLoadTime(anyLong());
@@ -606,13 +606,13 @@ public class ZooPropLoaderTest {
         new PropCacheCaffeineImpl.Builder(loader, cacheMetrics).forTests(ticker).build();
 
     // load cache
-    log.debug("received: {}", cache.get(propCacheKey));
+    log.debug("received: {}", cache.get(propStoreKey));
 
     ticker.advance(REFRESH_MIN + 1, TimeUnit.MINUTES);
 
-    assertNotNull(cache.get(propCacheKey)); // returns current and queues async refresh
+    assertNotNull(cache.get(propStoreKey)); // returns current and queues async refresh
 
-    assertNull(cache.get(propCacheKey)); // on exception, the loader should return null
+    assertNull(cache.get(propStoreKey)); // on exception, the loader should return null
 
   }
 
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 745ecdcf7c..b9d5ff3182 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
@@ -85,17 +85,17 @@ public class ZooPropStoreTest {
   @Test
   public void create() throws Exception {
 
-    var propCacheKey = TablePropKey.of(instanceId, TableId.of("propCacheKey"));
+    var propStoreKey = TablePropKey.of(instanceId, TableId.of("propStoreKey"));
 
     Capture<byte[]> bytes = newCapture();
-    expect(zrw.putPrivatePersistentData(eq(propCacheKey.getPath()), capture(bytes), anyObject()))
+    expect(zrw.putPrivatePersistentData(eq(propStoreKey.getPath()), capture(bytes), anyObject()))
         .andReturn(true).once();
 
     replay(context, zrw);
 
     PropStore propStore = ZooPropStore.initialize(instanceId, zrw);
 
-    propStore.create(propCacheKey,
+    propStore.create(propStoreKey,
         Map.of(TABLE_BULK_MAX_TABLETS.getKey(), "1234", TABLE_FILE_BLOCK_SIZE.getKey(), "512M"));
 
     var decoded = propCodec.fromBytes(0, bytes.getValue());
@@ -114,22 +114,22 @@ public class ZooPropStoreTest {
   @Test
   public void getTest() throws Exception {
 
-    var propCacheKey = TablePropKey.of(instanceId, TableId.of("propCacheKey"));
+    var propStoreKey = TablePropKey.of(instanceId, TableId.of("propStoreKey"));
 
     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(propCacheKey.getPath()), anyObject(PropStoreWatcher.class), anyObject()))
+    expect(zrw.getData(eq(propStoreKey.getPath()), anyObject(PropStoreWatcher.class), anyObject()))
         .andReturn(VersionedPropCodec.getDefault().toBytes(vProps)).once();
 
     replay(context, zrw);
 
     PropStore propStore = ZooPropStore.initialize(instanceId, zrw);
 
-    assertNotNull(propStore.get(propCacheKey)); // first call will fetch from ZooKeeper
-    assertNotNull(propStore.get(propCacheKey)); // next call will fetch from cache.
+    assertNotNull(propStore.get(propStoreKey)); // first call will fetch from ZooKeeper
+    assertNotNull(propStore.get(propStoreKey)); // next call will fetch from cache.
 
-    var p = propStore.get(propCacheKey);
+    var p = propStore.get(propStoreKey);
 
     assertEquals("true", p.asMap().get(Property.TABLE_BLOOM_ENABLED.getKey()));
   }
@@ -137,7 +137,7 @@ public class ZooPropStoreTest {
   @Test
   public void versionTest() throws Exception {
 
-    var propCacheKey = TablePropKey.of(instanceId, TableId.of("table1"));
+    var propStoreKey = TablePropKey.of(instanceId, TableId.of("table1"));
     Map<String,String> props =
         Map.of(TABLE_BULK_MAX_TABLETS.getKey(), "1234", TABLE_FILE_BLOCK_SIZE.getKey(), "512M");
 
@@ -146,7 +146,7 @@ public class ZooPropStoreTest {
 
     // force version mismatch between zk and props.
     var expectedVersion = 99;
-    expect(zrw.getData(eq(propCacheKey.getPath()), capture(propStoreWatcherCapture), capture(stat)))
+    expect(zrw.getData(eq(propStoreKey.getPath()), capture(propStoreWatcherCapture), capture(stat)))
         .andAnswer(() -> {
           Stat s = stat.getValue();
           s.setCtime(System.currentTimeMillis());
@@ -159,7 +159,7 @@ public class ZooPropStoreTest {
     replay(context, zrw);
 
     PropStore propStore = ZooPropStore.initialize(instanceId, zrw);
-    var vProps = propStore.get(propCacheKey);
+    var vProps = propStore.get(propStoreKey);
     assertNotNull(vProps);
     assertEquals(expectedVersion, vProps.getDataVersion());
   }
@@ -173,17 +173,17 @@ public class ZooPropStoreTest {
   @Test
   public void putAllTest() throws Exception {
 
-    var propCacheKey = TablePropKey.of(instanceId, TableId.of("table1"));
+    var propStoreKey = TablePropKey.of(instanceId, TableId.of("table1"));
 
     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(propCacheKey.getPath()), anyObject(Stat.class)))
+    expect(zrw.getData(eq(propStoreKey.getPath()), anyObject(Stat.class)))
         .andReturn(propCodec.toBytes(initialProps)).once();
 
     Capture<byte[]> bytes = newCapture();
-    expect(zrw.overwritePersistentData(eq(propCacheKey.getPath()), capture(bytes), eq(0)))
+    expect(zrw.overwritePersistentData(eq(propStoreKey.getPath()), capture(bytes), eq(0)))
         .andAnswer(() -> {
           var stored = propCodec.fromBytes(0, bytes.getValue());
           assertEquals(3, stored.asMap().size());
@@ -203,7 +203,7 @@ public class ZooPropStoreTest {
     Map<String,String> updateProps =
         Map.of(TABLE_BULK_MAX_TABLETS.getKey(), "4321", TABLE_SPLIT_THRESHOLD.getKey(), "123M");
 
-    propStore.putAll(propCacheKey, updateProps);
+    propStore.putAll(propStoreKey, updateProps);
 
     verify(zrw);
   }
@@ -211,7 +211,7 @@ public class ZooPropStoreTest {
   @Test
   public void removeTest() throws Exception {
 
-    var propCacheKey = TablePropKey.of(instanceId, TableId.of("table1"));
+    var propStoreKey = TablePropKey.of(instanceId, TableId.of("table1"));
 
     var initialProps = new VersionedProperties(123, Instant.now(),
         Map.of(TABLE_BULK_MAX_TABLETS.getKey(), "1234", TABLE_FILE_BLOCK_SIZE.getKey(), "512M"));
@@ -219,7 +219,7 @@ public class ZooPropStoreTest {
     // not cached - will load from ZooKeeper
     Capture<Stat> stat = newCapture();
 
-    expect(zrw.getData(eq(propCacheKey.getPath()), capture(stat))).andAnswer(() -> {
+    expect(zrw.getData(eq(propStoreKey.getPath()), capture(stat))).andAnswer(() -> {
       Stat s = stat.getValue();
       s.setVersion(123);
       stat.setValue(s);
@@ -227,7 +227,7 @@ public class ZooPropStoreTest {
     }).once();
 
     Capture<byte[]> bytes = newCapture();
-    expect(zrw.overwritePersistentData(eq(propCacheKey.getPath()), capture(bytes), eq(123)))
+    expect(zrw.overwritePersistentData(eq(propStoreKey.getPath()), capture(bytes), eq(123)))
         .andAnswer(() -> {
           var stored = propCodec.fromBytes(124, bytes.getValue());
           assertEquals(1, stored.asMap().size());
@@ -247,18 +247,18 @@ public class ZooPropStoreTest {
     Set<String> deleteNames =
         Set.of(TABLE_BULK_MAX_TABLETS.getKey(), TABLE_SPLIT_THRESHOLD.getKey());
 
-    propStore.removeProperties(propCacheKey, deleteNames);
+    propStore.removeProperties(propStoreKey, deleteNames);
     verify(zrw);
   }
 
   @Test
   public void removeWithExceptionsTest() throws Exception {
 
-    var propCacheKey = TablePropKey.of(instanceId, TableId.of("table1"));
+    var propStoreKey = TablePropKey.of(instanceId, TableId.of("table1"));
 
     // return "bad data"
     Capture<Stat> stat = newCapture();
-    expect(zrw.getData(eq(propCacheKey.getPath()), capture(stat))).andAnswer(() -> {
+    expect(zrw.getData(eq(propStoreKey.getPath()), capture(stat))).andAnswer(() -> {
       Stat s = stat.getValue();
       s.setCtime(System.currentTimeMillis());
       s.setMtime(System.currentTimeMillis());
@@ -269,9 +269,9 @@ public class ZooPropStoreTest {
     }).once();
 
     // mock throwing exceptions
-    expect(zrw.getData(eq(propCacheKey.getPath()), anyObject(Stat.class)))
+    expect(zrw.getData(eq(propStoreKey.getPath()), anyObject(Stat.class)))
         .andThrow(new KeeperException.NoNodeException("mock forced no node")).once();
-    expect(zrw.getData(eq(propCacheKey.getPath()), anyObject(Stat.class)))
+    expect(zrw.getData(eq(propStoreKey.getPath()), anyObject(Stat.class)))
         .andThrow(new InterruptedException("mock forced interrupt exception")).once();
 
     replay(context, zrw);
@@ -283,13 +283,13 @@ public class ZooPropStoreTest {
 
     // Parse error converted to IllegalStateException
     assertThrows(IllegalStateException.class,
-        () -> propStore.removeProperties(propCacheKey, deleteNames));
+        () -> propStore.removeProperties(propStoreKey, deleteNames));
     // ZK exception converted to IllegalStateException
     assertThrows(IllegalStateException.class,
-        () -> propStore.removeProperties(propCacheKey, deleteNames));
+        () -> propStore.removeProperties(propStoreKey, deleteNames));
     // InterruptException converted to IllegalStateException
     assertThrows(IllegalStateException.class,
-        () -> propStore.removeProperties(propCacheKey, deleteNames));
+        () -> propStore.removeProperties(propStoreKey, deleteNames));
   }
 
   @Test
@@ -376,28 +376,28 @@ public class ZooPropStoreTest {
   @Test
   public void deleteTest() throws Exception {
 
-    var propCacheKey = TablePropKey.of(instanceId, TableId.of("propCacheKey"));
+    var propStoreKey = TablePropKey.of(instanceId, TableId.of("propStoreKey"));
 
     var vProps = new VersionedProperties(Map.of(Property.TABLE_BLOOM_ENABLED.getKey(), "true"));
 
     // expect first call to load cache.
-    expect(zrw.getData(eq(propCacheKey.getPath()), anyObject(PropStoreWatcher.class), anyObject()))
+    expect(zrw.getData(eq(propStoreKey.getPath()), anyObject(PropStoreWatcher.class), anyObject()))
         .andReturn(VersionedPropCodec.getDefault().toBytes(vProps)).once();
 
-    zrw.delete(eq(propCacheKey.getPath()));
+    zrw.delete(eq(propStoreKey.getPath()));
     expectLastCall().once();
 
     replay(context, zrw);
 
     PropStore propStore = ZooPropStore.initialize(instanceId, zrw);
 
-    assertNotNull(propStore.get(propCacheKey)); // first call will fetch from ZooKeeper
-    assertNotNull(propStore.get(propCacheKey)); // next call will fetch from cache.
-    var p = propStore.get(propCacheKey);
+    assertNotNull(propStore.get(propStoreKey)); // first call will fetch from ZooKeeper
+    assertNotNull(propStore.get(propStoreKey)); // next call will fetch from cache.
+    var p = propStore.get(propStoreKey);
 
     assertEquals("true", p.asMap().get(Property.TABLE_BLOOM_ENABLED.getKey()));
 
-    propStore.delete(propCacheKey);
+    propStore.delete(propStoreKey);
     Thread.sleep(50);
   }
 }
diff --git a/test/src/main/java/org/apache/accumulo/test/conf/store/PropCacheCaffeineImplZkIT.java b/test/src/main/java/org/apache/accumulo/test/conf/store/PropCacheCaffeineImplZkIT.java
index 2f0b9034dd..27b1094b75 100644
--- a/test/src/main/java/org/apache/accumulo/test/conf/store/PropCacheCaffeineImplZkIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/conf/store/PropCacheCaffeineImplZkIT.java
@@ -151,8 +151,8 @@ public class PropCacheCaffeineImplZkIT {
     VersionedProperties vProps = new VersionedProperties(props);
 
     // directly create prop node - simulate existing properties.
-    var propCacheKey = TablePropKey.of(INSTANCE_ID, tIdA);
-    var created = zrw.putPersistentData(propCacheKey.getPath(),
+    var propStoreKey = TablePropKey.of(INSTANCE_ID, tIdA);
+    var created = zrw.putPersistentData(propStoreKey.getPath(),
         VersionedPropCodec.getDefault().toBytes(vProps), ZooUtil.NodeExistsPolicy.FAIL);
 
     assertTrue(created, "expected properties to be created");
@@ -168,7 +168,7 @@ public class PropCacheCaffeineImplZkIT {
     PropCacheCaffeineImpl cache =
         new PropCacheCaffeineImpl.Builder(propLoader, cacheMetrics).build();
 
-    VersionedProperties readProps = cache.get(propCacheKey);
+    VersionedProperties readProps = cache.get(propStoreKey);
 
     if (readProps == null) {
       fail("Received null for versioned properties");
diff --git a/test/src/main/java/org/apache/accumulo/test/conf/store/PropStoreZooKeeperIT.java b/test/src/main/java/org/apache/accumulo/test/conf/store/PropStoreZooKeeperIT.java
index c6f2d4b92d..a107498b24 100644
--- a/test/src/main/java/org/apache/accumulo/test/conf/store/PropStoreZooKeeperIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/conf/store/PropStoreZooKeeperIT.java
@@ -43,9 +43,9 @@ import org.apache.accumulo.fate.zookeeper.ZooUtil;
 import org.apache.accumulo.server.ServerContext;
 import org.apache.accumulo.server.conf.codec.VersionedPropCodec;
 import org.apache.accumulo.server.conf.codec.VersionedProperties;
-import org.apache.accumulo.server.conf.store.PropCacheKey;
 import org.apache.accumulo.server.conf.store.PropChangeListener;
 import org.apache.accumulo.server.conf.store.PropStore;
+import org.apache.accumulo.server.conf.store.PropStoreKey;
 import org.apache.accumulo.server.conf.store.TablePropKey;
 import org.apache.accumulo.server.conf.store.impl.ZooPropStore;
 import org.apache.accumulo.test.zookeeper.ZooKeeperTestingServer;
@@ -383,22 +383,22 @@ public class PropStoreZooKeeperIT {
 
   private static class TestChangeListener implements PropChangeListener {
 
-    private final Map<PropCacheKey<?>,Integer> changeCounts = new ConcurrentHashMap<>();
-    private final Map<PropCacheKey<?>,Integer> deleteCounts = new ConcurrentHashMap<>();
+    private final Map<PropStoreKey<?>,Integer> changeCounts = new ConcurrentHashMap<>();
+    private final Map<PropStoreKey<?>,Integer> deleteCounts = new ConcurrentHashMap<>();
 
     @Override
-    public void zkChangeEvent(PropCacheKey<?> propCacheKey) {
-      changeCounts.merge(propCacheKey, 1, Integer::sum);
+    public void zkChangeEvent(PropStoreKey<?> propStoreKey) {
+      changeCounts.merge(propStoreKey, 1, Integer::sum);
     }
 
     @Override
-    public void cacheChangeEvent(PropCacheKey<?> propCacheKey) {
-      changeCounts.merge(propCacheKey, 1, Integer::sum);
+    public void cacheChangeEvent(PropStoreKey<?> propStoreKey) {
+      changeCounts.merge(propStoreKey, 1, Integer::sum);
     }
 
     @Override
-    public void deleteEvent(PropCacheKey<?> propCacheKey) {
-      deleteCounts.merge(propCacheKey, 1, Integer::sum);
+    public void deleteEvent(PropStoreKey<?> propStoreKey) {
+      deleteCounts.merge(propStoreKey, 1, Integer::sum);
     }
 
     @Override
@@ -406,11 +406,11 @@ public class PropStoreZooKeeperIT {
 
     }
 
-    public Map<PropCacheKey<?>,Integer> getChangeCounts() {
+    public Map<PropStoreKey<?>,Integer> getChangeCounts() {
       return changeCounts;
     }
 
-    public Map<PropCacheKey<?>,Integer> getDeleteCounts() {
+    public Map<PropStoreKey<?>,Integer> getDeleteCounts() {
       return deleteCounts;
     }
   }
diff --git a/test/src/main/java/org/apache/accumulo/test/conf/store/ZooBasedConfigIT.java b/test/src/main/java/org/apache/accumulo/test/conf/store/ZooBasedConfigIT.java
index 47e59cc76a..cccc8c1f40 100644
--- a/test/src/main/java/org/apache/accumulo/test/conf/store/ZooBasedConfigIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/conf/store/ZooBasedConfigIT.java
@@ -51,9 +51,9 @@ import org.apache.accumulo.server.conf.NamespaceConfiguration;
 import org.apache.accumulo.server.conf.SystemConfiguration;
 import org.apache.accumulo.server.conf.ZooBasedConfiguration;
 import org.apache.accumulo.server.conf.store.NamespacePropKey;
-import org.apache.accumulo.server.conf.store.PropCacheKey;
 import org.apache.accumulo.server.conf.store.PropChangeListener;
 import org.apache.accumulo.server.conf.store.PropStore;
+import org.apache.accumulo.server.conf.store.PropStoreKey;
 import org.apache.accumulo.server.conf.store.SystemPropKey;
 import org.apache.accumulo.server.conf.store.TablePropKey;
 import org.apache.accumulo.server.conf.store.impl.ZooPropStore;
@@ -339,20 +339,20 @@ public class ZooBasedConfigIT {
     }
 
     @Override
-    public void zkChangeEvent(PropCacheKey<?> propCacheKey) {
-      log.debug("Received zkChangeEvent for {}", propCacheKey);
+    public void zkChangeEvent(PropStoreKey<?> propStoreKey) {
+      log.debug("Received zkChangeEvent for {}", propStoreKey);
       zkChangeCount.incrementAndGet();
     }
 
     @Override
-    public void cacheChangeEvent(PropCacheKey<?> propCacheKey) {
-      log.debug("Received cacheChangeEvent for {}", propCacheKey);
+    public void cacheChangeEvent(PropStoreKey<?> propStoreKey) {
+      log.debug("Received cacheChangeEvent for {}", propStoreKey);
       cacheChangeCount.incrementAndGet();
     }
 
     @Override
-    public void deleteEvent(PropCacheKey<?> propCacheKey) {
-      log.debug("Received deleteEvent for: {}", propCacheKey);
+    public void deleteEvent(PropStoreKey<?> propStoreKey) {
+      log.debug("Received deleteEvent for: {}", propStoreKey);
       deleteCount.incrementAndGet();
     }